DL-163: clean up direct zookeeper and bookkeeper usage and use metadata/data store abstraction

- introduce NamespaceDriver class to manage metadata/data store abstractions
- clean up direct zookeeper and bookkeeper usage and use metadata/data store abstraction
- separate MetadataAccessor from DistributedLogManager
- remove deprecated classes
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
index 0f4d3d4..072c3ef 100644
--- 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
@@ -25,7 +25,7 @@
 import com.twitter.distributedlog.LogSegmentMetadata;
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import java.io.IOException;
 import java.util.List;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
index b9d0365..cebbc33 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
@@ -594,8 +594,6 @@
                 }
                 lastProcessTime.reset().start();
 
-                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
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
index a6b5fd2..9432e8a 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
@@ -24,7 +24,6 @@
 import com.twitter.distributedlog.exceptions.WriteCancelledException;
 import com.twitter.distributedlog.exceptions.WriteException;
 import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.stats.OpStatsListener;
 import com.twitter.distributedlog.util.FailpointUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.util.Future;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
index d20cc6a..a3959b0 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
@@ -20,12 +20,6 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Ticker;
-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.callback.LogSegmentListener;
 import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
 import com.twitter.distributedlog.exceptions.AlreadyClosedException;
@@ -34,27 +28,22 @@
 import com.twitter.distributedlog.exceptions.UnexpectedException;
 import com.twitter.distributedlog.function.CloseAsyncCloseableFunction;
 import com.twitter.distributedlog.function.GetVersionedValueFunction;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
 import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
 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.impl.metadata.ZKLogStreamMetadataStore;
 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.BKDLConfig;
 import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
 import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
 import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.subscription.ZKSubscriptionStateStore;
-import com.twitter.distributedlog.subscription.ZKSubscriptionsStore;
-import com.twitter.distributedlog.util.ConfUtils;
+import com.twitter.distributedlog.util.Allocator;
 import com.twitter.distributedlog.util.DLUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.MonitoredFuturePool;
@@ -65,24 +54,20 @@
 import com.twitter.distributedlog.util.Utils;
 import com.twitter.util.ExceptionalFunction;
 import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.ExecutorServiceFuturePool;
 import com.twitter.util.Function;
 import com.twitter.util.Future;
-import com.twitter.util.FuturePool;
 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.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
 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;
@@ -91,6 +76,9 @@
 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>
@@ -112,7 +100,7 @@
  * for details.
  * </ul>
  */
-class BKDistributedLogManager extends ZKMetadataAccessor implements DistributedLogManager {
+class BKDistributedLogManager implements DistributedLogManager {
     static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class);
 
     static final Function<LogRecordWithDLSN, Long> RECORD_2_TXID_FUNCTION =
@@ -131,44 +119,28 @@
                 }
             };
 
+    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 OrderedScheduler readAheadScheduler;
-    private boolean ownExecutor;
     private final FeatureProvider featureProvider;
+    private final AsyncFailureInjector failureInjector;
     private final StatsLogger statsLogger;
     private final StatsLogger perLogStatsLogger;
     final AlertStatsLogger alertStatsLogger;
 
-    // log stream metadata stores
-    private final LogStreamMetadataStore writerMetadataStore;
-    private final LogStreamMetadataStore readerMetadataStore;
     // log segment metadata cache
     private final LogSegmentMetadataCache logSegmentMetadataCache;
 
-    // bookkeeper clients
-    // 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 final BookKeeperClientBuilder writerBKCBuilder;
-    private final BookKeeperClient writerBKC;
-    private final LogSegmentEntryStore writerEntryStore;
-    private final boolean ownWriterBKC;
-    private final BookKeeperClientBuilder readerBKCBuilder;
-    private final BookKeeperClient readerBKC;
-    private final LogSegmentEntryStore readerEntryStore;
-    private final boolean ownReaderBKC;
-
     //
     // Writer Related Variables
     //
-    private final LedgerAllocator ledgerAllocator;
     private final PermitLimiter writeLimiter;
 
     //
@@ -176,92 +148,26 @@
     ///
     // read handler for listener.
     private BKLogReadHandler readHandlerForListener = null;
-    private FuturePool readerFuturePool = null;
     private final PendingReaders pendingReaders;
 
-    // Failure Injector
-    private final AsyncFailureInjector failureInjector;
-
-    /**
-     * Create a DLM for testing.
-     *
-     * @param name log name
-     * @param conf distributedlog configuration
-     * @param uri uri location for the log
-     * @param writerZKCBuilder zookeeper builder for writers
-     * @param readerZKCBuilder zookeeper builder for readers
-     * @param zkcForWriterBKC zookeeper builder for bookkeeper shared by writers
-     * @param zkcForReaderBKC zookeeper builder for bookkeeper shared by readers
-     * @param writerBKCBuilder bookkeeper builder for writers
-     * @param readerBKCBuilder bookkeeper builder for readers
-     * @param featureProvider provider to offer features
-     * @param writeLimiter write limiter
-     * @param statsLogger stats logger to receive stats
-     * @throws IOException
-     */
-    BKDistributedLogManager(String name,
-                            DistributedLogConfiguration conf,
-                            URI uri,
-                            ZooKeeperClientBuilder writerZKCBuilder,
-                            ZooKeeperClientBuilder readerZKCBuilder,
-                            ZooKeeperClient zkcForWriterBKC,
-                            ZooKeeperClient zkcForReaderBKC,
-                            BookKeeperClientBuilder writerBKCBuilder,
-                            BookKeeperClientBuilder readerBKCBuilder,
-                            FeatureProvider featureProvider,
-                            PermitLimiter writeLimiter,
-                            StatsLogger statsLogger) throws IOException {
-        this(name,
-             conf,
-             ConfUtils.getConstDynConf(conf),
-             uri,
-             writerZKCBuilder,
-             readerZKCBuilder,
-             zkcForWriterBKC,
-             zkcForReaderBKC,
-             writerBKCBuilder,
-             readerBKCBuilder,
-             null,
-             null,
-             new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
-             OrderedScheduler.newBuilder().name("BKDL-" + name).corePoolSize(1).build(),
-             null,
-             null,
-             null,
-             DistributedLogConstants.UNKNOWN_CLIENT_ID,
-             DistributedLogConstants.LOCAL_REGION_ID,
-             null,
-             writeLimiter,
-             featureProvider,
-             statsLogger,
-             NullStatsLogger.INSTANCE);
-        this.ownExecutor = true;
-    }
+    // 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 writerZKCBuilder zookeeper builder for writers
-     * @param readerZKCBuilder zookeeper builder for readers
-     * @param zkcForWriterBKC zookeeper builder for bookkeeper shared by writers
-     * @param zkcForReaderBKC zookeeper builder for bookkeeper shared by readers
-     * @param writerBKCBuilder bookkeeper builder for writers
-     * @param readerBKCBuilder bookkeeper builder for readers
-     * @param writerMetadataStore writer metadata store
-     * @param readerMetadataStore reader metadata store
+     * @param driver namespace driver
+     * @param logSegmentMetadataCache log segment metadata cache
      * @param scheduler ordered scheduled used by readers and writers
-     * @param readAheadScheduler readAhead scheduler used by readers
-     * @param channelFactory client socket channel factory to build bookkeeper clients
-     * @param requestTimer request timer to build bookkeeper clients
      * @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 ledgerAllocator ledger allocator to allocate ledgers
-     * @param featureProvider provider to offer features
      * @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
@@ -270,148 +176,48 @@
                             DistributedLogConfiguration conf,
                             DynamicDistributedLogConfiguration dynConf,
                             URI uri,
-                            ZooKeeperClientBuilder writerZKCBuilder,
-                            ZooKeeperClientBuilder readerZKCBuilder,
-                            ZooKeeperClient zkcForWriterBKC,
-                            ZooKeeperClient zkcForReaderBKC,
-                            BookKeeperClientBuilder writerBKCBuilder,
-                            BookKeeperClientBuilder readerBKCBuilder,
-                            LogStreamMetadataStore writerMetadataStore,
-                            LogStreamMetadataStore readerMetadataStore,
+                            NamespaceDriver driver,
                             LogSegmentMetadataCache logSegmentMetadataCache,
                             OrderedScheduler scheduler,
-                            OrderedScheduler readAheadScheduler,
-                            ClientSocketChannelFactory channelFactory,
-                            HashedWheelTimer requestTimer,
                             String clientId,
                             Integer regionId,
-                            LedgerAllocator ledgerAllocator,
                             PermitLimiter writeLimiter,
                             FeatureProvider featureProvider,
+                            AsyncFailureInjector failureInjector,
                             StatsLogger statsLogger,
-                            StatsLogger perLogStatsLogger) throws IOException {
-        super(name, conf, uri, writerZKCBuilder, readerZKCBuilder, 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.readAheadScheduler = null == readAheadScheduler ? scheduler : readAheadScheduler;
         this.statsLogger = statsLogger;
         this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
-        this.ownExecutor = false;
         this.pendingReaders = new PendingReaders(scheduler);
         this.regionId = regionId;
         this.clientId = clientId;
         this.streamIdentifier = conf.getUnpartitionedStreamName();
-        this.ledgerAllocator = ledgerAllocator;
         this.writeLimiter = writeLimiter;
-
-        // Failure Injection
-        this.failureInjector = AsyncRandomFailureInjector.newBuilder()
-                .injectDelays(conf.getEIInjectReadAheadDelay(),
-                              conf.getEIInjectReadAheadDelayPercent(),
-                              conf.getEIInjectMaxReadAheadDelayMs())
-                .injectErrors(false, 10)
-                .injectStops(conf.getEIInjectReadAheadStall(), 10)
-                .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
-                .build();
-
-        if (null == writerMetadataStore) {
-            this.writerMetadataStore = new ZKLogStreamMetadataStore(
-                    clientId,
-                    conf,
-                    writerZKC,
-                    scheduler,
-                    statsLogger);
-        } else {
-            this.writerMetadataStore = writerMetadataStore;
-        }
-        if (null == readerMetadataStore) {
-            this.readerMetadataStore = new ZKLogStreamMetadataStore(
-                    clientId,
-                    conf,
-                    readerZKC,
-                    scheduler,
-                    statsLogger);
-        } else {
-            this.readerMetadataStore = readerMetadataStore;
-        }
-        this.logSegmentMetadataCache = logSegmentMetadataCache;
-
-        // create the bkc for writers
-        if (null == writerBKCBuilder) {
-            // resolve uri
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, uri);
-            BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-            this.writerBKCBuilder = BookKeeperClientBuilder.newBuilder()
-                    .dlConfig(conf)
-                    .name(String.format("bk:%s:dlm_writer_shared", name))
-                    .ledgersPath(bkdlConfig.getBkLedgersPath())
-                    .channelFactory(channelFactory)
-                    .requestTimer(requestTimer)
-                    .statsLogger(statsLogger);
-            if (null == zkcForWriterBKC) {
-                this.writerBKCBuilder.zkServers(bkdlConfig.getBkZkServersForWriter());
-            } else {
-                this.writerBKCBuilder.zkc(zkcForWriterBKC);
-            }
-            this.ownWriterBKC = true;
-        } else {
-            this.writerBKCBuilder = writerBKCBuilder;
-            this.ownWriterBKC = false;
-        }
-        this.writerBKC = this.writerBKCBuilder.build();
-        this.writerEntryStore = new BKLogSegmentEntryStore(
-                conf,
-                writerBKC,
-                scheduler,
-                statsLogger,
-                failureInjector);
-
-        // create the bkc for readers
-        if (null == readerBKCBuilder) {
-            // resolve uri
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, uri);
-            BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-            if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
-                this.readerBKCBuilder = this.writerBKCBuilder;
-                this.ownReaderBKC = false;
-            } else {
-                this.readerBKCBuilder = BookKeeperClientBuilder.newBuilder()
-                        .dlConfig(conf)
-                        .name(String.format("bk:%s:dlm_reader_shared", name))
-                        .ledgersPath(bkdlConfig.getBkLedgersPath())
-                        .channelFactory(channelFactory)
-                        .requestTimer(requestTimer)
-                        .statsLogger(statsLogger);
-                if (null == zkcForReaderBKC) {
-                    this.readerBKCBuilder.zkServers(bkdlConfig.getBkZkServersForReader());
-                } else {
-                    this.readerBKCBuilder.zkc(zkcForReaderBKC);
-                }
-                this.ownReaderBKC = true;
-            }
-        } else {
-            this.readerBKCBuilder = readerBKCBuilder;
-            this.ownReaderBKC = false;
-        }
-        this.readerBKC = this.readerBKCBuilder.build();
-        this.readerEntryStore = new BKLogSegmentEntryStore(
-                conf,
-                readerBKC,
-                scheduler,
-                statsLogger,
-                failureInjector);
-
         // Feature Provider
         this.featureProvider = featureProvider;
-
+        // Failure Injector
+        this.failureInjector = failureInjector;
         // Stats
         this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
+        this.resourcesCloseable = resourcesCloseable;
     }
 
-    @VisibleForTesting
-    LogStreamMetadataStore getWriterMetadataStore() {
-        return writerMetadataStore;
+    @Override
+    public String getStreamName() {
+        return name;
+    }
+
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return driver;
     }
 
     URI getUri() {
@@ -426,23 +232,22 @@
         return scheduler;
     }
 
+    AsyncFailureInjector getFailureInjector() {
+        return failureInjector;
+    }
+
+    //
+    // Test Methods
+    //
+
     @VisibleForTesting
-    BookKeeperClient getWriterBKC() {
-        return this.writerBKC;
+    LogStreamMetadataStore getWriterMetadataStore() {
+        return driver.getLogStreamMetadataStore(WRITER);
     }
 
     @VisibleForTesting
-    BookKeeperClient getReaderBKC() {
-        return this.readerBKC;
-    }
-
     LogSegmentEntryStore getReaderEntryStore() {
-        return this.readerEntryStore;
-    }
-
-    @VisibleForTesting
-    FuturePool getReaderFuturePool() {
-        return this.readerFuturePool;
+        return driver.getLogSegmentEntryStore(READER);
     }
 
     @VisibleForTesting
@@ -450,10 +255,6 @@
         return this.featureProvider;
     }
 
-    AsyncFailureInjector getFailureInjector() {
-        return this.failureInjector;
-    }
-
     private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
             boolean create, LogSegmentListener listener) {
         if (null == readHandlerForListener && create) {
@@ -502,12 +303,6 @@
                 throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
             }
         }
-        if (null != writerBKC) {
-            writerBKC.checkClosedOrInError();
-        }
-        if (null != readerBKC) {
-            readerBKC.checkClosedOrInError();
-        }
     }
 
     // Create Read Handler
@@ -538,9 +333,9 @@
                 subscriberId,
                 conf,
                 dynConf,
-                readerMetadataStore,
+                driver.getLogStreamMetadataStore(READER),
                 logSegmentMetadataCache,
-                readerEntryStore,
+                driver.getLogSegmentEntryStore(READER),
                 scheduler,
                 alertStatsLogger,
                 statsLogger,
@@ -552,23 +347,7 @@
 
     // Create Ledger Allocator
 
-    LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata) throws IOException {
-        LedgerAllocator ledgerAllocatorDelegator;
-        if (!dynConf.getEnableLedgerAllocatorPool()) {
-            QuorumConfigProvider quorumConfigProvider =
-                    new DynamicQuorumConfigProvider(dynConf);
-            LedgerAllocator allocator = new SimpleLedgerAllocator(
-                    logMetadata.getAllocationPath(),
-                    logMetadata.getAllocationData(),
-                    quorumConfigProvider,
-                    writerZKC,
-                    writerBKC);
-            ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true);
-        } else {
-            ledgerAllocatorDelegator = ledgerAllocator;
-        }
-        return ledgerAllocatorDelegator;
-    }
+
 
     // Create Write Handler
 
@@ -578,14 +357,12 @@
     }
 
     Future<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
-        boolean ownAllocator = null == ledgerAllocator;
-
         // Fetching Log Metadata (create if not exists)
-        return writerMetadataStore.getLog(
+        return driver.getLogStreamMetadataStore(WRITER).getLog(
                 uri,
                 name,
-                ownAllocator,
-                conf.getCreateStreamIfNotExists() || ownAllocator
+                true,
+                conf.getCreateStreamIfNotExists()
         ).flatMap(new AbstractFunction1<LogMetadataForWriter, Future<BKLogWriteHandler>>() {
             @Override
             public Future<BKLogWriteHandler> apply(LogMetadataForWriter logMetadata) {
@@ -602,16 +379,17 @@
         // Build the locks
         DistributedLock lock;
         if (conf.isWriteLockEnabled()) {
-            lock = writerMetadataStore.createWriteLock(logMetadata);
+            lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
         } else {
             lock = NopDistributedLock.INSTANCE;
         }
-        // Build the ledger allocator
-        LedgerAllocator allocator;
+
+        Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
         try {
-            allocator = createLedgerAllocator(logMetadata);
-        } catch (IOException e) {
-            FutureUtils.setException(createPromise, e);
+            segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
+                    .newLogSegmentAllocator(logMetadata, dynConf);
+        } catch (IOException ioe) {
+            FutureUtils.setException(createPromise, ioe);
             return;
         }
 
@@ -619,11 +397,11 @@
         final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
                 logMetadata,
                 conf,
-                writerMetadataStore,
+                driver.getLogStreamMetadataStore(WRITER),
                 logSegmentMetadataCache,
-                writerEntryStore,
+                driver.getLogSegmentEntryStore(WRITER),
                 scheduler,
-                allocator,
+                segmentAllocator,
                 statsLogger,
                 perLogStatsLogger,
                 alertStatsLogger,
@@ -657,12 +435,11 @@
     }
 
     PermitManager getLogSegmentRollingPermitManager() {
-        return writerMetadataStore.getPermitManager();
+        return driver.getLogStreamMetadataStore(WRITER).getPermitManager();
     }
 
     <T> Future<T> processReaderOperation(final Function<BKLogReadHandler, Future<T>> func) {
-        initializeFuturePool(false);
-        return readerFuturePool.apply(new ExceptionalFunction0<BKLogReadHandler>() {
+        return scheduler.apply(new ExceptionalFunction0<BKLogReadHandler>() {
             @Override
             public BKLogReadHandler applyE() throws Throwable {
                 return getReadHandlerAndRegisterListener(true, null);
@@ -822,7 +599,7 @@
                 fromTxnId,
                 segmentIdx,
                 segments,
-                readerEntryStore
+                driver.getLogSegmentEntryStore(READER)
         );
     }
 
@@ -1010,8 +787,9 @@
                 LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.",
                         subscriberId.get(), name);
                 // we acquired lock
-                final SubscriptionStateStore stateStore = getSubscriptionStateStore(subscriberId.get());
-                return stateStore.getLastCommitPosition().map(new ExceptionalFunction<DLSN, AsyncLogReader>() {
+                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 {}.",
@@ -1226,7 +1004,8 @@
      */
     @Override
     public void delete() throws IOException {
-        FutureUtils.result(writerMetadataStore.deleteLog(uri, getStreamName()));
+        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+                .deleteLog(uri, getStreamName()));
     }
 
     /**
@@ -1297,43 +1076,10 @@
             readHandlerToClose = readHandlerForListener;
         }
 
-        // NOTE: the resources {scheduler, writerBKC, readerBKC} are mostly from namespace instance.
-        //       so they are not blocking call except tests.
-        AsyncCloseable resourcesCloseable = new AsyncCloseable() {
-            @Override
-            public Future<Void> asyncClose() {
-                int schedTimeout = conf.getSchedulerShutdownTimeoutMs();
-
-                // Clean up executor state.
-                if (ownExecutor) {
-                    SchedulerUtils.shutdownScheduler(scheduler, schedTimeout, TimeUnit.MILLISECONDS);
-                    LOG.info("Stopped BKDL executor service for {}.", name);
-
-                    if (scheduler != readAheadScheduler) {
-                        SchedulerUtils.shutdownScheduler(readAheadScheduler, schedTimeout, TimeUnit.MILLISECONDS);
-                        LOG.info("Stopped BKDL ReadAhead Executor Service for {}.", name);
-                    }
-                }
-                if (ownWriterBKC) {
-                    writerBKC.close();
-                }
-                if (ownReaderBKC) {
-                    readerBKC.close();
-                }
-                return Future.Void();
-            }
-        };
-
         Future<Void> closeResult = Utils.closeSequence(null, true,
                 readHandlerToClose,
                 pendingReaders,
-                resourcesCloseable,
-                new AsyncCloseable() {
-                    @Override
-                    public Future<Void> asyncClose() {
-                        return BKDistributedLogManager.super.asyncClose();
-                    }
-                });
+                resourcesCloseable.or(AsyncCloseable.NULL));
         closeResult.proxyTo(closeFuture);
         return closeFuture;
     }
@@ -1343,70 +1089,18 @@
         FutureUtils.result(asyncClose());
     }
 
-    private FuturePool buildFuturePool(ExecutorService executorService,
-                                       StatsLogger statsLogger) {
-        FuturePool futurePool = new ExecutorServiceFuturePool(executorService);
-        return new MonitoredFuturePool(
-                futurePool,
-                statsLogger,
-                conf.getEnableTaskExecutionStats(),
-                conf.getTaskExecutionWarnTimeMicros());
-    }
-
-    private void initializeFuturePool(boolean ordered) {
-        // ownExecutor is a single threaded thread pool
-        if (null == readerFuturePool) {
-            readerFuturePool = buildFuturePool(
-                    scheduler, statsLogger.scope("reader_future_pool"));
-        }
-    }
-
     @Override
     public String toString() {
-        return String.format("DLM:%s:%s", getZKPath(), getStreamName());
+        return String.format("DLM:%s:%s", getUri(), getStreamName());
     }
 
     public void raiseAlert(String msg, Object... args) {
         alertStatsLogger.raise(msg, args);
     }
 
-    /**
-     * Get the subscription state storage provided by the distributed log manager
-     *
-     * @param subscriberId - Application specific Id associated with the subscriber
-     * @return Subscription state store
-     */
-    @Override
-    @Deprecated
-    public SubscriptionStateStore getSubscriptionStateStore(String subscriberId) {
-        return getSubscriptionStateStoreInternal(conf.getUnpartitionedStreamName(), subscriberId);
-    }
-
-    /**
-     * Get the subscription state storage provided by the distributed log manager
-     *
-     * @param streamIdentifier - Identifier associated with the stream
-     * @param subscriberId - Application specific Id associated with the subscriber
-     * @return Subscription state store
-     */
-    private SubscriptionStateStore getSubscriptionStateStoreInternal(String streamIdentifier, String subscriberId) {
-        return new ZKSubscriptionStateStore(writerZKC,
-                LogMetadataForReader.getSubscriberPath(uri, name, streamIdentifier, subscriberId));
-    }
-
     @Override
     public SubscriptionsStore getSubscriptionsStore() {
-        return getSubscriptionsStoreInternal(conf.getUnpartitionedStreamName());
+        return driver.getSubscriptionsStore(getStreamName());
     }
 
-    /**
-     * Get the subscription state storage provided by the distributed log manager
-     *
-     * @param streamIdentifier - Identifier associated with the stream
-     * @return Subscriptions store
-     */
-    private SubscriptionsStore getSubscriptionsStoreInternal(String streamIdentifier) {
-        return new ZKSubscriptionsStore(writerZKC,
-                LogMetadataForReader.getSubscribersPath(uri, name, streamIdentifier));
-    }
 }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
index 1a23228..a8b1f77 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
@@ -17,71 +17,39 @@
  */
 package com.twitter.distributedlog;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Ticker;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption;
 import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.acl.DefaultAccessControlManager;
-import com.twitter.distributedlog.acl.ZKAccessControlManager;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
 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.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.impl.ZKLogMetadataStore;
-import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.io.AsyncCloseable;
 import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
 import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.DLUtils;
 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.SimplePermitLimiter;
 import com.twitter.distributedlog.util.Utils;
 import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-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.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.InetAddress;
 import java.net.URI;
-import java.util.Collection;
-import java.util.HashMap;
 import java.util.Iterator;
-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.impl.BKDLUtils.*;
+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
@@ -119,344 +87,57 @@
 public class BKDistributedLogNamespace implements DistributedLogNamespace {
     static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
 
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    public static class Builder {
-        private DistributedLogConfiguration _conf = null;
-        private URI _uri = null;
-        private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
-        private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
-        private FeatureProvider _featureProvider = new SettableFeatureProvider("", 0);
-        private String _clientId = DistributedLogConstants.UNKNOWN_CLIENT_ID;
-        private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
-
-        private Builder() {}
-
-        public Builder conf(DistributedLogConfiguration conf) {
-            this._conf = conf;
-            return this;
-        }
-
-        public Builder uri(URI uri) {
-            this._uri = uri;
-            return this;
-        }
-
-        public Builder statsLogger(StatsLogger statsLogger) {
-            this._statsLogger = statsLogger;
-            return this;
-        }
-
-        public Builder perLogStatsLogger(StatsLogger perLogStatsLogger) {
-            this._perLogStatsLogger = perLogStatsLogger;
-            return this;
-        }
-
-        public Builder featureProvider(FeatureProvider featureProvider) {
-            this._featureProvider = featureProvider;
-            return this;
-        }
-
-        public Builder clientId(String clientId) {
-            this._clientId = clientId;
-            return this;
-        }
-
-        public Builder regionId(int regionId) {
-            this._regionId = regionId;
-            return this;
-        }
-
-        @SuppressWarnings("deprecation")
-        public BKDistributedLogNamespace build()
-                throws IOException, NullPointerException, IllegalArgumentException {
-            Preconditions.checkNotNull(_conf, "No DistributedLog Configuration");
-            Preconditions.checkNotNull(_uri, "No DistributedLog URI");
-            Preconditions.checkNotNull(_featureProvider, "No Feature Provider");
-            Preconditions.checkNotNull(_statsLogger, "No Stats Logger");
-            Preconditions.checkNotNull(_featureProvider, "No Feature Provider");
-            Preconditions.checkNotNull(_clientId, "No Client ID");
-            // validate conf and uri
-            validateConfAndURI(_conf, _uri);
-
-            // Build the namespace zookeeper client
-            ZooKeeperClientBuilder nsZkcBuilder = createDLZKClientBuilder(
-                    String.format("dlzk:%s:factory_writer_shared", _uri),
-                    _conf,
-                    DLUtils.getZKServersFromDLUri(_uri),
-                    _statsLogger.scope("dlzk_factory_writer_shared"));
-            ZooKeeperClient nsZkc = nsZkcBuilder.build();
-
-            // Resolve namespace binding
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(nsZkc, _uri);
-
-            // Backward Compatible to enable per log stats by configuration settings
-            StatsLogger perLogStatsLogger = _perLogStatsLogger;
-            if (perLogStatsLogger == NullStatsLogger.INSTANCE &&
-                    _conf.getEnablePerStreamStat()) {
-                perLogStatsLogger = _statsLogger.scope("stream");
-            }
-
-            return new BKDistributedLogNamespace(
-                    _conf,
-                    _uri,
-                    _featureProvider,
-                    _statsLogger,
-                    perLogStatsLogger,
-                    _clientId,
-                    _regionId,
-                    nsZkcBuilder,
-                    nsZkc,
-                    bkdlConfig);
-        }
-    }
-
-    static interface ZooKeeperClientHandler<T> {
-        T handle(ZooKeeperClient zkc) throws IOException;
-    }
-
-    /**
-     * Run given <i>handler</i> by providing an available new zookeeper client
-     *
-     * @param handler
-     *          Handler to process with provided zookeeper client.
-     * @param conf
-     *          Distributedlog Configuration.
-     * @param namespace
-     *          Distributedlog Namespace.
-     */
-    private static <T> T withZooKeeperClient(ZooKeeperClientHandler<T> handler,
-                                             DistributedLogConfiguration conf,
-                                             URI namespace) throws IOException {
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .name(String.format("dlzk:%s:factory_static", namespace))
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .uri(namespace)
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getZKRequestRateLimit())
-                .zkAclId(conf.getZkAclId())
-                .build();
-        try {
-            return handler.handle(zkc);
-        } finally {
-            zkc.close();
-        }
-    }
-
-    private static String getHostIpLockClientId() {
-        try {
-            return InetAddress.getLocalHost().toString();
-        } catch(Exception ex) {
-            return DistributedLogConstants.UNKNOWN_CLIENT_ID;
-        }
-    }
-
     private final String clientId;
     private final int regionId;
     private final DistributedLogConfiguration conf;
     private final URI namespace;
-    private final BKDLConfig bkdlConfig;
+    // namespace driver
+    private final NamespaceDriver driver;
+    // resources
     private final OrderedScheduler scheduler;
-    private final OrderedScheduler readAheadExecutor;
-    private final ClientSocketChannelFactory channelFactory;
-    private final HashedWheelTimer requestTimer;
-    // 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 final ZooKeeperClientBuilder sharedWriterZKCBuilderForDL;
-    private final ZooKeeperClient sharedWriterZKCForDL;
-    private final ZooKeeperClientBuilder sharedReaderZKCBuilderForDL;
-    private final ZooKeeperClient sharedReaderZKCForDL;
-    private ZooKeeperClientBuilder sharedWriterZKCBuilderForBK = null;
-    private ZooKeeperClient sharedWriterZKCForBK = null;
-    private ZooKeeperClientBuilder sharedReaderZKCBuilderForBK = null;
-    private ZooKeeperClient sharedReaderZKCForBK = null;
-    // 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 final BookKeeperClientBuilder sharedWriterBKCBuilder;
-    private final BookKeeperClient writerBKC;
-    private final BookKeeperClientBuilder sharedReaderBKCBuilder;
-    private final BookKeeperClient readerBKC;
-    // ledger allocator
-    private final LedgerAllocator allocator;
-    // access control manager
-    private AccessControlManager accessControlManager;
-    // log metadata store
-    private final LogMetadataStore metadataStore;
+    private final PermitLimiter writeLimiter;
+    private final AsyncFailureInjector failureInjector;
     // log segment metadata store
     private final LogSegmentMetadataCache logSegmentMetadataCache;
-    private final LogStreamMetadataStore writerStreamMetadataStore;
-    private final LogStreamMetadataStore readerStreamMetadataStore;
-
     // feature provider
     private final FeatureProvider featureProvider;
-
     // Stats Loggers
     private final StatsLogger statsLogger;
     private final StatsLogger perLogStatsLogger;
 
-    protected AtomicBoolean closed = new AtomicBoolean(false);
+    protected final AtomicBoolean closed = new AtomicBoolean(false);
 
-    private final PermitLimiter writeLimiter;
-
-    private BKDistributedLogNamespace(
+    public BKDistributedLogNamespace(
             DistributedLogConfiguration conf,
             URI uri,
+            NamespaceDriver driver,
+            OrderedScheduler scheduler,
             FeatureProvider featureProvider,
+            PermitLimiter writeLimiter,
+            AsyncFailureInjector failureInjector,
             StatsLogger statsLogger,
             StatsLogger perLogStatsLogger,
             String clientId,
-            int regionId,
-            ZooKeeperClientBuilder nsZkcBuilder,
-            ZooKeeperClient nsZkc,
-            BKDLConfig bkdlConfig)
-            throws IOException, IllegalArgumentException {
+            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;
-        this.bkdlConfig = bkdlConfig;
-        if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
-            this.clientId = getHostIpLockClientId();
-        } else {
-            this.clientId = clientId;
-        }
 
-        // Build resources
-        StatsLogger schedulerStatsLogger = statsLogger.scope("factory").scope("thread_pool");
-        this.scheduler = OrderedScheduler.newBuilder()
-                .name("DLM-" + uri.getPath())
-                .corePoolSize(conf.getNumWorkerThreads())
-                .statsLogger(schedulerStatsLogger)
-                .perExecutorStatsLogger(schedulerStatsLogger)
-                .traceTaskExecution(conf.getEnableTaskExecutionStats())
-                .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
-                .build();
-        if (conf.getNumReadAheadWorkerThreads() > 0) {
-            this.readAheadExecutor = OrderedScheduler.newBuilder()
-                    .name("DLM-" + uri.getPath() + "-readahead-executor")
-                    .corePoolSize(conf.getNumReadAheadWorkerThreads())
-                    .statsLogger(statsLogger.scope("factory").scope("readahead_thread_pool"))
-                    .traceTaskExecution(conf.getTraceReadAheadDeliveryLatency())
-                    .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
-                    .build();
-            LOG.info("Created dedicated readahead executor : threads = {}", conf.getNumReadAheadWorkerThreads());
-        } else {
-            this.readAheadExecutor = this.scheduler;
-            LOG.info("Used shared executor for readahead.");
-        }
-
-        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 zookeeper client for writers
-        this.sharedWriterZKCBuilderForDL = nsZkcBuilder;
-        this.sharedWriterZKCForDL = nsZkc;
-
-        // Build zookeeper client for readers
-        if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
-            this.sharedReaderZKCBuilderForDL = this.sharedWriterZKCBuilderForDL;
-        } else {
-            this.sharedReaderZKCBuilderForDL = createDLZKClientBuilder(
-                    String.format("dlzk:%s:factory_reader_shared", namespace),
-                    conf,
-                    bkdlConfig.getDlZkServersForReader(),
-                    statsLogger.scope("dlzk_factory_reader_shared"));
-        }
-        this.sharedReaderZKCForDL = this.sharedReaderZKCBuilderForDL.build();
-
-        // Build bookkeeper client for writers
-        this.sharedWriterBKCBuilder = createBKCBuilder(
-                String.format("bk:%s:factory_writer_shared", namespace),
-                conf,
-                bkdlConfig.getBkZkServersForWriter(),
-                bkdlConfig.getBkLedgersPath(),
-                Optional.of(featureProvider.scope("bkc")));
-        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(),
-                Optional.<FeatureProvider>absent());
-        }
-        this.readerBKC = this.sharedReaderBKCBuilder.build();
-
-        if (conf.getGlobalOutstandingWriteLimit() < 0) {
-            this.writeLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
-        } else {
-            Feature disableWriteLimitFeature = featureProvider.getFeature(
-                CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
-            this.writeLimiter = new SimplePermitLimiter(
-                conf.getOutstandingWriteLimitDarkmode(),
-                conf.getGlobalOutstandingWriteLimit(),
-                statsLogger.scope("writeLimiter"),
-                true /* singleton */,
-                disableWriteLimitFeature);
-        }
-
-        // propagate bkdlConfig to configuration
-        BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-
-        // Build the allocator
-        if (conf.getEnableLedgerAllocatorPool()) {
-            String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
-            allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize(),
-                    conf, sharedWriterZKCForDL, writerBKC, scheduler);
-            if (null != allocator) {
-                allocator.start();
-            }
-            LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
-        } else {
-            allocator = null;
-        }
-
-        // log metadata store
-        if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
-            this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, sharedReaderZKCForDL, scheduler);
-        } else {
-            this.metadataStore = new ZKLogMetadataStore(conf, namespace, sharedReaderZKCForDL, scheduler);
-        }
-
-        // create log stream metadata store
-        this.writerStreamMetadataStore =
-                new ZKLogStreamMetadataStore(
-                        clientId,
-                        conf,
-                        sharedWriterZKCForDL,
-                        scheduler,
-                        statsLogger);
-        this.readerStreamMetadataStore =
-                new ZKLogStreamMetadataStore(
-                        clientId,
-                        conf,
-                        sharedReaderZKCForDL,
-                        scheduler,
-                        statsLogger);
         // create a log segment metadata cache
         this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
+    }
 
-        LOG.info("Constructed BK DistributedLogNamespace : clientId = {}, regionId = {}, federated = {}.",
-                new Object[] { clientId, regionId, bkdlConfig.isFederatedNamespace() });
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return driver;
     }
 
     //
@@ -468,8 +149,8 @@
             throws InvalidStreamNameException, IOException {
         checkState();
         validateName(logName);
-        URI uri = FutureUtils.result(metadataStore.createLog(logName));
-        FutureUtils.result(writerStreamMetadataStore.getLog(uri, logName, true, true));
+        URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
+        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
     }
 
     @Override
@@ -477,17 +158,15 @@
             throws InvalidStreamNameException, LogNotFoundException, IOException {
         checkState();
         validateName(logName);
-        Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
         if (!uri.isPresent()) {
             throw new LogNotFoundException("Log " + logName + " isn't found.");
         }
-        DistributedLogManager dlm = createDistributedLogManager(
+        DistributedLogManager dlm = openLogInternal(
                 uri.get(),
                 logName,
-                ClientSharingOption.SharedClients,
                 Optional.<DistributedLogConfiguration>absent(),
-                Optional.<DynamicDistributedLogConfiguration>absent(),
-                Optional.<StatsLogger>absent());
+                Optional.<DynamicDistributedLogConfiguration>absent());
         dlm.delete();
     }
 
@@ -508,27 +187,26 @@
             throws InvalidStreamNameException, IOException {
         checkState();
         validateName(logName);
-        Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
         if (!uri.isPresent()) {
             throw new LogNotFoundException("Log " + logName + " isn't found.");
         }
-        return createDistributedLogManager(
+        return openLogInternal(
                 uri.get(),
                 logName,
-                ClientSharingOption.SharedClients,
                 logConf,
-                dynamicLogConf,
-                perStreamStatsLogger);
+                dynamicLogConf);
     }
 
     @Override
     public boolean logExists(String logName)
         throws IOException, IllegalArgumentException {
         checkState();
-        Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
         if (uri.isPresent()) {
             try {
-                FutureUtils.result(writerStreamMetadataStore.logExists(uri.get(), logName));
+                FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+                        .logExists(uri.get(), logName));
                 return true;
             } catch (LogNotFoundException lnfe) {
                 return false;
@@ -541,240 +219,18 @@
     @Override
     public Iterator<String> getLogs() throws IOException {
         checkState();
-        return FutureUtils.result(metadataStore.getLogs());
+        return FutureUtils.result(driver.getLogMetadataStore().getLogs());
     }
 
     @Override
     public void registerNamespaceListener(NamespaceListener listener) {
-        metadataStore.registerNamespaceListener(listener);
+        driver.getLogMetadataStore().registerNamespaceListener(listener);
     }
 
     @Override
     public synchronized AccessControlManager createAccessControlManager() throws IOException {
         checkState();
-        if (null == accessControlManager) {
-            String aclRootPath = bkdlConfig.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, sharedReaderZKCForDL,
-                        zkRootPath, scheduler);
-                LOG.info("Created zk based access control manager @ {} for {}",
-                        zkRootPath, namespace);
-            }
-        }
-        return accessControlManager;
-    }
-
-    //
-    // Legacy methods
-    //
-
-    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;
-    }
-
-    public static ZooKeeperClientBuilder createDLZKClientBuilder(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 static ZooKeeperClientBuilder createBKZKClientBuilder(String zkcName,
-                                                                  DistributedLogConfiguration conf,
-                                                                  String zkServers,
-                                                                  StatsLogger statsLogger) {
-        RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                    conf.getBKClientZKRetryBackoffStartMillis(),
-                    conf.getBKClientZKRetryBackoffMaxMillis(),
-                    conf.getBKClientZKNumRetries());
-        ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
-                .name(zkcName)
-                .sessionTimeoutMs(conf.getBKClientZKSessionTimeoutMilliSeconds())
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getBKClientZKRequestRateLimit())
-                .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.getBKClientZKNumRetries(),
-                conf.getBKClientZKSessionTimeoutMilliSeconds(), conf.getBKClientZKRetryBackoffStartMillis(),
-                conf.getBKClientZKRetryBackoffMaxMillis(), conf.getZkAclId() });
-        return builder;
-    }
-
-    private BookKeeperClientBuilder createBKCBuilder(String bkcName,
-                                                     DistributedLogConfiguration conf,
-                                                     String zkServers,
-                                                     String ledgersPath,
-                                                     Optional<FeatureProvider> featureProviderOptional) {
-        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;
-    }
-
-    @VisibleForTesting
-    public ZooKeeperClient getSharedWriterZKCForDL() {
-        return sharedWriterZKCForDL;
-    }
-
-    @VisibleForTesting
-    public BookKeeperClient getReaderBKC() {
-        return readerBKC;
-    }
-
-    @VisibleForTesting
-    public LogStreamMetadataStore getWriterStreamMetadataStore() {
-        return writerStreamMetadataStore;
-    }
-
-    @VisibleForTesting
-    public LedgerAllocator getLedgerAllocator() {
-        return allocator;
-    }
-
-    /**
-     * Run given <i>handler</i> by providing an available zookeeper client.
-     *
-     * @param handler
-     *          Handler to process with provided zookeeper client.
-     * @return result processed by handler.
-     * @throws IOException
-     */
-    private <T> T withZooKeeperClient(ZooKeeperClientHandler<T> handler) throws IOException {
-        checkState();
-        return handler.handle(sharedWriterZKCForDL);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with default shared clients.
-     *
-     * @param nameOfLogStream
-     *          name of log stream
-     * @return distributedlog manager
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManagerWithSharedClients(String nameOfLogStream)
-        throws InvalidStreamNameException, IOException {
-        return createDistributedLogManager(nameOfLogStream, ClientSharingOption.SharedClients);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
-     *
-     * @param nameOfLogStream
-     *          name of log stream.
-     * @param clientSharingOption
-     *          specifies if the ZK/BK clients are shared
-     * @return distributedlog manager instance.
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManager(
-            String nameOfLogStream,
-            ClientSharingOption clientSharingOption)
-        throws InvalidStreamNameException, IOException {
-        Optional<DistributedLogConfiguration> logConfiguration = Optional.absent();
-        Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration = Optional.absent();
-        return createDistributedLogManager(
-                nameOfLogStream,
-                clientSharingOption,
-                logConfiguration,
-                dynamicLogConfiguration);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
-     * Override whitelisted stream-level configuration settings with settings found in
-     * <i>logConfiguration</i>.
-     *
-     *
-     * @param nameOfLogStream
-     *          name of log stream.
-     * @param clientSharingOption
-     *          specifies if the ZK/BK clients are shared
-     * @param logConfiguration
-     *          stream configuration overrides.
-     * @param dynamicLogConfiguration
-     *          dynamic stream configuration overrides.
-     * @return distributedlog manager instance.
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManager(
-            String nameOfLogStream,
-            ClientSharingOption clientSharingOption,
-            Optional<DistributedLogConfiguration> logConfiguration,
-            Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
-        throws InvalidStreamNameException, IOException {
-        if (bkdlConfig.isFederatedNamespace()) {
-            throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
-        }
-        return createDistributedLogManager(
-                namespace,
-                nameOfLogStream,
-                clientSharingOption,
-                logConfiguration,
-                dynamicLogConfiguration,
-                Optional.<StatsLogger>absent()
-        );
+        return driver.getAccessControlManager();
     }
 
     /**
@@ -784,8 +240,6 @@
      *          location to store the log
      * @param nameOfLogStream
      *          name of the log
-     * @param clientSharingOption
-     *          client sharing option
      * @param logConfiguration
      *          optional stream configuration
      * @param dynamicLogConfiguration
@@ -794,13 +248,11 @@
      * @throws InvalidStreamNameException if the stream name is invalid
      * @throws IOException
      */
-    protected DistributedLogManager createDistributedLogManager(
+    protected DistributedLogManager openLogInternal(
             URI uri,
             String nameOfLogStream,
-            ClientSharingOption clientSharingOption,
             Optional<DistributedLogConfiguration> logConfiguration,
-            Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration,
-            Optional<StatsLogger> perStreamStatsLogger)
+            Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
         throws InvalidStreamNameException, IOException {
         // Make sure the name is well formed
         checkState();
@@ -817,172 +269,34 @@
             dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
         }
 
-        ZooKeeperClientBuilder writerZKCBuilderForDL = null;
-        ZooKeeperClientBuilder readerZKCBuilderForDL = null;
-        ZooKeeperClient writerZKCForBK = null;
-        ZooKeeperClient readerZKCForBK = null;
-        BookKeeperClientBuilder writerBKCBuilder = null;
-        BookKeeperClientBuilder readerBKCBuilder = null;
-
-        switch(clientSharingOption) {
-            case SharedClients:
-                writerZKCBuilderForDL = sharedWriterZKCBuilderForDL;
-                readerZKCBuilderForDL = sharedReaderZKCBuilderForDL;
-                writerBKCBuilder = sharedWriterBKCBuilder;
-                readerBKCBuilder = sharedReaderBKCBuilder;
-                break;
-            case SharedZKClientPerStreamBKClient:
-                writerZKCBuilderForDL = sharedWriterZKCBuilderForDL;
-                readerZKCBuilderForDL = sharedReaderZKCBuilderForDL;
-                synchronized (this) {
-                    if (null == this.sharedWriterZKCForBK) {
-                        this.sharedWriterZKCBuilderForBK = createBKZKClientBuilder(
-                            String.format("bkzk:%s:factory_writer_shared", uri),
-                            mergedConfiguration,
-                            bkdlConfig.getBkZkServersForWriter(),
-                            statsLogger.scope("bkzk_factory_writer_shared"));
-                        this.sharedWriterZKCForBK = this.sharedWriterZKCBuilderForBK.build();
-                    }
-                    if (null == this.sharedReaderZKCForBK) {
-                        if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
-                            this.sharedReaderZKCBuilderForBK = this.sharedWriterZKCBuilderForBK;
-                        } else {
-                            this.sharedReaderZKCBuilderForBK = createBKZKClientBuilder(
-                                String.format("bkzk:%s:factory_reader_shared", uri),
-                                mergedConfiguration,
-                                bkdlConfig.getBkZkServersForReader(),
-                                statsLogger.scope("bkzk_factory_reader_shared"));
-                        }
-                        this.sharedReaderZKCForBK = this.sharedReaderZKCBuilderForBK.build();
-                    }
-                    writerZKCForBK = this.sharedWriterZKCForBK;
-                    readerZKCForBK = this.sharedReaderZKCForBK;
-                }
-                break;
-        }
-
-        LedgerAllocator dlmLedgerAlloctor = null;
-        if (ClientSharingOption.SharedClients == clientSharingOption) {
-            dlmLedgerAlloctor = this.allocator;
-        }
-        // if there's a specified perStreamStatsLogger, user it, otherwise use the default one.
-        StatsLogger perLogStatsLogger = perStreamStatsLogger.or(this.perLogStatsLogger);
-
         return new BKDistributedLogManager(
                 nameOfLogStream,                    /* Log Name */
                 mergedConfiguration,                /* Configuration */
                 dynConf,                            /* Dynamic Configuration */
-                uri,                                /* Namespace */
-                writerZKCBuilderForDL,              /* ZKC Builder for DL Writer */
-                readerZKCBuilderForDL,              /* ZKC Builder for DL Reader */
-                writerZKCForBK,                     /* ZKC for BookKeeper for DL Writers */
-                readerZKCForBK,                     /* ZKC for BookKeeper for DL Readers */
-                writerBKCBuilder,                   /* BookKeeper Builder for DL Writers */
-                readerBKCBuilder,                   /* BookKeeper Builder for DL Readers */
-                writerStreamMetadataStore,         /* Log Segment Metadata Store for DL Writers */
-                readerStreamMetadataStore,         /* Log Segment Metadata Store for DL Readers */
+                uri,                                /* Namespace URI */
+                driver,                             /* Namespace Driver */
                 logSegmentMetadataCache,            /* Log Segment Metadata Cache */
                 scheduler,                          /* DL scheduler */
-                readAheadExecutor,                  /* Read Aheader Executor */
-                channelFactory,                     /* Netty Channel Factory */
-                requestTimer,                       /* Request Timer */
                 clientId,                           /* Client Id */
                 regionId,                           /* Region Id */
-                dlmLedgerAlloctor,                  /* Ledger Allocator */
                 writeLimiter,                       /* Write Limiter */
                 featureProvider.scope("dl"),        /* Feature Provider */
+                failureInjector,                    /* Failure Injector */
                 statsLogger,                        /* Stats Logger */
-                perLogStatsLogger                   /* Per Log Stats Logger */
+                perLogStatsLogger,                  /* Per Log Stats Logger */
+                Optional.<AsyncCloseable>absent()   /* shared resources, we don't need to close any resources in dlm */
         );
     }
 
-    public MetadataAccessor createMetadataAccessor(String nameOfMetadataNode)
-            throws InvalidStreamNameException, IOException {
-        if (bkdlConfig.isFederatedNamespace()) {
-            throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
-        }
-        checkState();
-        validateName(nameOfMetadataNode);
-        return new ZKMetadataAccessor(nameOfMetadataNode, conf, namespace,
-                sharedWriterZKCBuilderForDL, sharedReaderZKCBuilderForDL, statsLogger);
-    }
-
-    public Collection<String> enumerateAllLogsInNamespace()
-        throws IOException, IllegalArgumentException {
-        if (bkdlConfig.isFederatedNamespace()) {
-            throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
-        }
-        return Sets.newHashSet(getLogs());
-    }
-
-    public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
-        throws IOException, IllegalArgumentException {
-        if (bkdlConfig.isFederatedNamespace()) {
-            throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
-        }
-        return withZooKeeperClient(new ZooKeeperClientHandler<Map<String, byte[]>>() {
-            @Override
-            public Map<String, byte[]> handle(ZooKeeperClient zkc) throws IOException {
-                return enumerateLogsWithMetadataInternal(zkc, conf, namespace);
-            }
-        });
-    }
-
-    private static void validateInput(DistributedLogConfiguration conf, URI uri, String nameOfStream)
-        throws IllegalArgumentException, InvalidStreamNameException {
-        validateConfAndURI(conf, uri);
-        validateName(nameOfStream);
-    }
-
-    public static Map<String, byte[]> enumerateLogsWithMetadataInNamespace(final DistributedLogConfiguration conf, final URI uri)
-        throws IOException, IllegalArgumentException {
-        return withZooKeeperClient(new ZooKeeperClientHandler<Map<String, byte[]>>() {
-            @Override
-            public Map<String, byte[]> handle(ZooKeeperClient zkc) throws IOException {
-                return enumerateLogsWithMetadataInternal(zkc, conf, uri);
-            }
-        }, conf, uri);
-    }
-
-    private static Map<String, byte[]> enumerateLogsWithMetadataInternal(ZooKeeperClient zkc,
-                                                                         DistributedLogConfiguration conf, URI uri)
-        throws IOException, IllegalArgumentException {
-        validateConfAndURI(conf, uri);
-        String namespaceRootPath = uri.getPath();
-        HashMap<String, byte[]> result = new HashMap<String, byte[]>();
-        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;
-    }
-
+    /**
+     * Check the namespace state.
+     *
+     * @throws IOException
+     */
     private void checkState() throws IOException {
         if (closed.get()) {
-            LOG.error("BKDistributedLogNamespace {} is already closed", namespace);
-            throw new AlreadyClosedException("Namespace " + namespace + " is already closed");
+            LOG.error("BK namespace {} is already closed", namespace);
+            throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
         }
     }
 
@@ -991,58 +305,16 @@
      */
     @Override
     public void close() {
-        ZooKeeperClient writerZKC;
-        ZooKeeperClient readerZKC;
-        AccessControlManager acm;
-        if (closed.compareAndSet(false, true)) {
-            writerZKC = sharedWriterZKCForBK;
-            readerZKC = sharedReaderZKCForBK;
-            acm = accessControlManager;
-        } else {
+        if (!closed.compareAndSet(false, true)) {
             return;
         }
-        if (null != acm) {
-            acm.close();
-            LOG.info("Access Control Manager Stopped.");
-        }
-
-        // Close the allocator
-        if (null != allocator) {
-            Utils.closeQuietly(allocator);
-            LOG.info("Ledger Allocator stopped.");
-        }
-
+        // shutdown the driver
+        Utils.close(driver);
+        // close the write limiter
         this.writeLimiter.close();
-
-        // Shutdown log segment metadata stores
-        Utils.close(writerStreamMetadataStore);
-        Utils.close(readerStreamMetadataStore);
-
         // Shutdown the schedulers
         SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
-            TimeUnit.MILLISECONDS);
-        LOG.info("Executor Service Stopped.");
-        if (scheduler != readAheadExecutor) {
-            SchedulerUtils.shutdownScheduler(readAheadExecutor, conf.getSchedulerShutdownTimeoutMs(),
                 TimeUnit.MILLISECONDS);
-            LOG.info("ReadAhead Executor Service Stopped.");
-        }
-
-        writerBKC.close();
-        readerBKC.close();
-        sharedWriterZKCForDL.close();
-        sharedReaderZKCForDL.close();
-
-        // Close shared zookeeper clients for bk
-        if (null != writerZKC) {
-            writerZKC.close();
-        }
-        if (null != readerZKC) {
-            readerZKC.close();
-        }
-        channelFactory.releaseExternalResources();
-        LOG.info("Release external resources used by channel factory.");
-        requestTimer.stop();
-        LOG.info("Stopped request timer");
+        LOG.info("Executor Service Stopped.");
     }
 }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
index 25b25e2..2486297 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
@@ -20,7 +20,6 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
-import com.twitter.distributedlog.bk.LedgerAllocator;
 import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
 import com.twitter.distributedlog.exceptions.DLIllegalStateException;
 import com.twitter.distributedlog.exceptions.EndOfStreamException;
@@ -29,8 +28,8 @@
 import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
 import com.twitter.distributedlog.exceptions.UnexpectedException;
 import com.twitter.distributedlog.function.GetLastTxIdFunction;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
 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;
@@ -41,6 +40,7 @@
 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;
@@ -53,7 +53,6 @@
 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.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
@@ -88,9 +87,22 @@
 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 LedgerAllocator ledgerAllocator;
     protected final MaxTxId maxTxId;
     protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
     protected final boolean validateLogSegmentSequenceNumber;
@@ -154,7 +166,7 @@
                       LogSegmentMetadataCache metadataCache,
                       LogSegmentEntryStore entryStore,
                       OrderedScheduler scheduler,
-                      LedgerAllocator allocator,
+                      Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
                       StatsLogger statsLogger,
                       StatsLogger perLogStatsLogger,
                       AlertStatsLogger alertStatsLogger,
@@ -174,11 +186,11 @@
                 alertStatsLogger,
                 clientId);
         this.logMetadataForWriter = logMetadata;
+        this.logSegmentAllocator = segmentAllocator;
         this.perLogStatsLogger = perLogStatsLogger;
         this.writeLimiter = writeLimiter;
         this.featureProvider = featureProvider;
         this.dynConf = dynConf;
-        this.ledgerAllocator = allocator;
         this.lock = lock;
         this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
 
@@ -523,7 +535,7 @@
         }
 
         try {
-            ledgerAllocator.allocate();
+            logSegmentAllocator.allocate();
         } catch (IOException e) {
             // failed to issue an allocation request
             failStartLogSegment(promise, bestEffort, e);
@@ -541,25 +553,16 @@
             return;
         }
 
-        ledgerAllocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
-            @Override
-            public void onCommit(LedgerHandle lh) {
-                // no-op
-            }
+        logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
+                .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
 
             @Override
-            public void onAbort(Throwable t) {
-                // no-op
-            }
-        }).addEventListener(new FutureEventListener<LedgerHandle>() {
-
-            @Override
-            public void onSuccess(LedgerHandle lh) {
+            public void onSuccess(LogSegmentEntryWriter entryWriter) {
                 // try-obtain succeed
                 createInprogressLogSegment(
                         txn,
                         txId,
-                        lh,
+                        entryWriter,
                         bestEffort,
                         promise);
             }
@@ -586,7 +589,7 @@
     // just leak from the allocation pool - hence cause "No Ledger Allocator"
     private void createInprogressLogSegment(Transaction<Object> txn,
                                             final long txId,
-                                            final LedgerHandle lh,
+                                            final LogSegmentEntryWriter entryWriter,
                                             boolean bestEffort,
                                             final Promise<BKLogSegmentWriter> promise) {
         final long logSegmentSeqNo;
@@ -601,13 +604,15 @@
             return;
         }
 
-        final String inprogressZnodePath = inprogressZNode(lh.getId(), txId, logSegmentSeqNo);
+        final String inprogressZnodePath = inprogressZNode(
+                entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
         final LogSegmentMetadata l =
             new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
-                conf.getDLLedgerMetadataLayoutVersion(), lh.getId(), txId)
+                conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
                     .setLogSegmentSequenceNo(logSegmentSeqNo)
                     .setRegionId(regionId)
-                    .setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
+                    .setEnvelopeEntries(
+                            LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
                     .build();
 
         // Create an inprogress segment
@@ -631,7 +636,7 @@
                             l.getSegmentName(),
                             conf,
                             conf.getDLLedgerMetadataLayoutVersion(),
-                            new BKLogSegmentEntryWriter(lh),
+                            entryWriter,
                             lock,
                             txId,
                             logSegmentSeqNo,
@@ -1268,8 +1273,7 @@
     public Future<Void> asyncClose() {
         return Utils.closeSequence(scheduler,
                 lock,
-                ledgerAllocator
-        );
+                logSegmentAllocator);
     }
 
     @Override
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
index c39ae4c..8d3c418 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
@@ -250,15 +250,22 @@
         return promise;
     }
 
-    public synchronized void close() {
-        if (closed) {
-            return;
+    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 != bkc) {
+        if (null != bkcToClose) {
             try {
-                bkc.close();
+                bkcToClose.close();
             } catch (InterruptedException e) {
                 LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
                 Thread.currentThread().interrupt();
@@ -266,12 +273,11 @@
                 LOG.warn("Error on closing bookkeeper client {} : ", name, e);
             }
         }
-        if (null != zkc) {
+        if (null != zkcToClose) {
             if (ownZK) {
-                zkc.close();
+                zkcToClose.close();
             }
         }
-        closed = true;
     }
 
     public synchronized void checkClosedOrInError() throws AlreadyClosedException {
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
index 6c6017e..6da4b8d 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
@@ -1301,6 +1301,7 @@
      * @return number of dedicated readahead worker threads.
      * @see #getNumWorkerThreads()
      */
+    @Deprecated
     public int getNumReadAheadWorkerThreads() {
         return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
     }
@@ -1313,6 +1314,7 @@
      * @return configuration
      * @see #getNumReadAheadWorkerThreads()
      */
+    @Deprecated
     public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
         setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
         return this;
@@ -3515,8 +3517,11 @@
         Preconditions.checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
             "Invalid timeout configuration: bkcReadTimeoutSeconds ("+getBKClientReadTimeout()+
                 ") should be longer than readLACLongPollTimeout ("+getReadLACLongPollTimeout()+")");
-        Preconditions.checkArgument(getReaderIdleWarnThresholdMillis() > 2 * getReadLACLongPollTimeout(),
-            "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
+        long readerIdleWarnThresholdMs = getReaderIdleWarnThresholdMillis();
+        if (readerIdleWarnThresholdMs > 0) { // NOTE: some test cases set the idle warn threshold to 0
+            Preconditions.checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
+                    "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
+        }
     }
 
 
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
index ccb0778..34cfb65 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
@@ -18,9 +18,13 @@
 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;
 
@@ -31,7 +35,20 @@
  * 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 MetadataAccessor {
+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.
@@ -282,15 +299,6 @@
     public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
 
     /**
-     * Get the subscription state storage provided by the distributed log manager
-     *
-     * @param subscriberId - Application specific Id associated with the subscriber
-     * @return Subscription state store
-     */
-    @Deprecated
-    public SubscriptionStateStore getSubscriptionStateStore(String subscriberId);
-
-    /**
      * Get the subscriptions store provided by the distributedlog manager.
      *
      * @return subscriptions store manages subscriptions for current stream.
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java
deleted file mode 100644
index 4caeeba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.acl.AccessControlManager;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-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;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * This is the legacy way to access bookkeeper based distributedlog namespace.
- * Use {@link DistributedLogNamespace} to manage logs instead if you could.
- */
-@Deprecated
-public class DistributedLogManagerFactory {
-    static final Logger LOG = LoggerFactory.getLogger(DistributedLogManagerFactory.class);
-
-    public static enum ClientSharingOption {
-        PerStreamClients,
-        SharedZKClientPerStreamBKClient,
-        SharedClients
-    }
-
-    private final BKDistributedLogNamespace namespace;
-
-    public DistributedLogManagerFactory(DistributedLogConfiguration conf, URI uri)
-            throws IOException, IllegalArgumentException {
-        this(conf, uri, NullStatsLogger.INSTANCE);
-    }
-
-    public DistributedLogManagerFactory(DistributedLogConfiguration conf, URI uri,
-                                        StatsLogger statsLogger)
-            throws IOException, IllegalArgumentException {
-        this(conf,
-             uri,
-             statsLogger,
-             DistributedLogConstants.UNKNOWN_CLIENT_ID,
-             DistributedLogConstants.LOCAL_REGION_ID);
-    }
-
-    public DistributedLogManagerFactory(DistributedLogConfiguration conf,
-                                        URI uri,
-                                        StatsLogger statsLogger,
-                                        String clientId,
-                                        int regionId)
-            throws IOException, IllegalArgumentException {
-        this.namespace = BKDistributedLogNamespace.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .statsLogger(statsLogger)
-                .clientId(clientId)
-                .regionId(regionId)
-                .build();
-    }
-
-    public DistributedLogNamespace getNamespace() {
-        return namespace;
-    }
-
-    public void registerNamespaceListener(NamespaceListener listener) {
-        namespace.registerNamespaceListener(listener);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with default shared clients.
-     *
-     * @param nameOfLogStream
-     *          name of log stream
-     * @return distributedlog manager
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManagerWithSharedClients(String nameOfLogStream)
-        throws InvalidStreamNameException, IOException {
-        return createDistributedLogManager(nameOfLogStream, ClientSharingOption.SharedClients);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
-     *
-     * @param nameOfLogStream
-     *          name of log stream.
-     * @param clientSharingOption
-     *          specifies if the ZK/BK clients are shared
-     * @return distributedlog manager instance.
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManager(
-            String nameOfLogStream,
-            ClientSharingOption clientSharingOption)
-        throws InvalidStreamNameException, IOException {
-        Optional<DistributedLogConfiguration> streamConfiguration = Optional.absent();
-        Optional<DynamicDistributedLogConfiguration> dynamicStreamConfiguration = Optional.absent();
-        return createDistributedLogManager(nameOfLogStream,
-            clientSharingOption,
-            streamConfiguration,
-            dynamicStreamConfiguration);
-    }
-
-    /**
-     * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
-     * This method allows the caller to override global configuration options by supplying stream
-     * configuration overrides. Stream config overrides come in two flavors, static and dynamic. Static
-     * config never changes, and DynamicDistributedLogConfiguration is a) reloaded periodically and
-     * b) safe to access from any context.
-     *
-     * @param nameOfLogStream
-     *          name of log stream.
-     * @param clientSharingOption
-     *          specifies if the ZK/BK clients are shared
-     * @param streamConfiguration
-     *          stream configuration overrides.
-     * @param dynamicStreamConfiguration
-     *          dynamic stream configuration overrides.
-     * @return distributedlog manager instance.
-     * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
-     * @throws IOException
-     */
-    public DistributedLogManager createDistributedLogManager(
-            String nameOfLogStream,
-            ClientSharingOption clientSharingOption,
-            Optional<DistributedLogConfiguration> streamConfiguration,
-            Optional<DynamicDistributedLogConfiguration> dynamicStreamConfiguration)
-        throws InvalidStreamNameException, IOException {
-        return namespace.createDistributedLogManager(
-            nameOfLogStream,
-            clientSharingOption,
-            streamConfiguration,
-            dynamicStreamConfiguration);
-    }
-
-    public MetadataAccessor createMetadataAccessor(String nameOfMetadataNode)
-            throws InvalidStreamNameException, IOException {
-        return namespace.createMetadataAccessor(nameOfMetadataNode);
-    }
-
-    public synchronized AccessControlManager createAccessControlManager() throws IOException {
-        return namespace.createAccessControlManager();
-    }
-
-    public boolean checkIfLogExists(String nameOfLogStream)
-        throws IOException, IllegalArgumentException {
-        return namespace.logExists(nameOfLogStream);
-    }
-
-    public Collection<String> enumerateAllLogsInNamespace()
-        throws IOException, IllegalArgumentException {
-        return namespace.enumerateAllLogsInNamespace();
-    }
-
-    public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
-        throws IOException, IllegalArgumentException {
-        return namespace.enumerateLogsWithMetadataInNamespace();
-    }
-
-    /**
-     * This method is to initialize the metadata for a unpartitioned stream with name <i>streamName</i>.
-     *
-     * TODO: after 0.2 is upgraded to 0.3, remove this.
-     *
-     * @param streamName
-     *          stream name.
-     * @throws IOException
-     */
-    public void createUnpartitionedStream(final String streamName) throws IOException {
-        namespace.createLog(streamName);
-    }
-
-    /**
-     * Close the distributed log manager factory, freeing any resources it may hold.
-     */
-    public void close() {
-        namespace.close();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
index 85a370f..f4a1e41 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
@@ -18,7 +18,7 @@
 package com.twitter.distributedlog;
 
 import com.google.common.base.Optional;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+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;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
index 94e618a..19f4497 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
@@ -65,6 +65,7 @@
 public class ReadAheadEntryReader implements
         AsyncCloseable,
         LogSegmentListener,
+        LogSegmentEntryReader.StateChangeListener,
         FutureEventListener<List<Entry.Reader>> {
 
     private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
@@ -169,6 +170,9 @@
         @Override
         synchronized public void onSuccess(LogSegmentEntryReader reader) {
             this.reader = reader;
+            if (reader.getSegment().isInProgress()) {
+                reader.registerListener(ReadAheadEntryReader.this);
+            }
         }
 
         @Override
@@ -271,7 +275,7 @@
     // State of the reader
     //
 
-    private boolean isInitialized;
+    private boolean isInitialized = false;
     private boolean readAheadPaused = false;
     private Promise<Void> closePromise = null;
     // segment readers
@@ -549,10 +553,22 @@
         }
     }
 
+    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
     //
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
index 90807b0..15f1805 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
@@ -20,7 +20,7 @@
 import com.google.common.base.Preconditions;
 import com.twitter.distributedlog.ZooKeeperClient.Credentials;
 import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
@@ -139,7 +139,7 @@
      * @return builder.
      */
     public synchronized ZooKeeperClientBuilder uri(URI uri) {
-        this.zkServers = DLUtils.getZKServersFromDLUri(uri);
+        this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
         return this;
     }
 
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
index 0a3fdb0..0512907 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
@@ -18,28 +18,27 @@
 package com.twitter.distributedlog.admin;
 
 import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
+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.acl.ZKAccessControl;
+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.impl.logsegment.BKLogSegmentEntryStore;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
 import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+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.DLUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.SchedulerUtils;
@@ -61,6 +60,7 @@
 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;
@@ -76,7 +76,6 @@
 /**
  * Admin Tool for DistributedLog.
  */
-@SuppressWarnings("deprecation")
 public class DistributedLogAdmin extends DistributedLogTool {
 
     static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
@@ -84,8 +83,8 @@
     /**
      * Fix inprogress segment with lower ledger sequence number.
      *
-     * @param factory
-     *          dlm factory.
+     * @param namespace
+     *          dl namespace
      * @param metadataUpdater
      *          metadata updater.
      * @param streamName
@@ -96,12 +95,12 @@
      *          is confirmation needed before executing actual action.
      * @throws IOException
      */
-    public static void fixInprogressSegmentWithLowerSequenceNumber(final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+    public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
                                                                    final MetadataUpdater metadataUpdater,
                                                                    final String streamName,
                                                                    final boolean verbose,
                                                                    final boolean interactive) throws IOException {
-        DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
         try {
             List<LogSegmentMetadata> segments = dlm.getLogSegments();
             if (verbose) {
@@ -194,37 +193,37 @@
     }
 
     public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+                                                 final DistributedLogNamespace namespace,
                                                  final MetadataUpdater metadataUpdater,
                                                  final OrderedScheduler scheduler,
-                                                 final BookKeeperClient bkc,
-                                                 final String digestpw,
                                                  final boolean verbose,
                                                  final boolean interactive) throws IOException {
-        checkAndRepairDLNamespace(uri, factory, metadataUpdater, scheduler, bkc, digestpw, verbose, interactive, 1);
+        checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
     }
 
     public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+                                                 final DistributedLogNamespace namespace,
                                                  final MetadataUpdater metadataUpdater,
                                                  final OrderedScheduler scheduler,
-                                                 final BookKeeperClient bkc,
-                                                 final String digestpw,
                                                  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.
-        Collection<String> streams = factory.enumerateAllLogsInNamespace();
+        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.size() + " streams under " + uri);
+            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(factory, streams, scheduler, bkc, digestpw, concurrency);
+                checkStreams(namespace, streams, scheduler, concurrency);
         if (verbose) {
             System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
         }
@@ -248,11 +247,9 @@
     }
 
     private static Map<String, StreamCandidate> checkStreams(
-            final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+            final DistributedLogNamespace namespace,
             final Collection<String> streams,
             final OrderedScheduler scheduler,
-            final BookKeeperClient bkc,
-            final String digestpw,
             final int concurrency) throws IOException {
         final LinkedBlockingQueue<String> streamQueue =
                 new LinkedBlockingQueue<String>();
@@ -275,7 +272,7 @@
                     StreamCandidate candidate;
                     try {
                         LOG.info("Checking stream {}.", stream);
-                        candidate = checkStream(factory, stream, scheduler, bkc, digestpw);
+                        candidate = checkStream(namespace, stream, scheduler);
                         LOG.info("Checked stream {} - {}.", stream, candidate);
                     } catch (IOException e) {
                         LOG.error("Error on checking stream {} : ", stream, e);
@@ -316,12 +313,10 @@
     }
 
     private static StreamCandidate checkStream(
-            final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+            final DistributedLogNamespace namespace,
             final String streamName,
-            final OrderedScheduler scheduler,
-            final BookKeeperClient bkc,
-            String digestpw) throws IOException {
-        DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+            final OrderedScheduler scheduler) throws IOException {
+        DistributedLogManager dlm = namespace.openLog(streamName);
         try {
             List<LogSegmentMetadata> segments = dlm.getLogSegments();
             if (segments.isEmpty()) {
@@ -330,7 +325,7 @@
             List<Future<LogSegmentCandidate>> futures =
                     new ArrayList<Future<LogSegmentCandidate>>(segments.size());
             for (LogSegmentMetadata segment : segments) {
-                futures.add(checkLogSegment(streamName, segment, scheduler, bkc, digestpw));
+                futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
             }
             List<LogSegmentCandidate> segmentCandidates;
             try {
@@ -354,21 +349,16 @@
     }
 
     private static Future<LogSegmentCandidate> checkLogSegment(
+            final DistributedLogNamespace namespace,
             final String streamName,
             final LogSegmentMetadata metadata,
-            final OrderedScheduler scheduler,
-            final BookKeeperClient bkc,
-            final String digestpw) {
+            final OrderedScheduler scheduler) {
         if (metadata.isInProgress()) {
             return Future.value(null);
         }
 
-        final LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
-                new DistributedLogConfiguration().setBKDigestPW(digestpw),
-                bkc,
-                scheduler,
-                NullStatsLogger.INSTANCE,
-                AsyncFailureInjector.NULL);
+        final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
+                .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
         return ReadUtils.asyncReadLastRecord(
                 streamName,
                 metadata,
@@ -432,6 +422,8 @@
 
     /**
      * Unbind the bookkeeper environment for a given distributedlog uri.
+     *
+     * TODO: move unbind operation to namespace driver
      */
     class UnbindCommand extends OptsCommand {
 
@@ -491,6 +483,8 @@
 
     /**
      * Bind Command to bind bookkeeper environment for a given distributed uri.
+     *
+     * TODO: move bind to namespace driver
      */
     class BindCommand extends OptsCommand {
 
@@ -559,7 +553,7 @@
             if (cmdline.hasOption("dlzw")) {
                 dlZkServersForWriter = cmdline.getOptionValue("dlzw");
             } else {
-                dlZkServersForWriter = DLUtils.getZKServersFromDLUri(uri);
+                dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
             }
             if (cmdline.hasOption("dlzr")) {
                 dlZkServersForReader = cmdline.getOptionValue("dlzr");
@@ -689,7 +683,7 @@
                 return -1;
             }
             for (String stream : streams) {
-                fixInprogressSegmentWithLowerSequenceNumber(getFactory(), metadataUpdater, stream, verbose, !getForce());
+                fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
             }
             return 0;
         }
@@ -739,10 +733,9 @@
                     .corePoolSize(Runtime.getRuntime().availableProcessors())
                     .build();
             ExecutorService executorService = Executors.newCachedThreadPool();
-            BookKeeperClient bkc = getBookKeeperClient();
             try {
-                checkAndRepairDLNamespace(getUri(), getFactory(), metadataUpdater, scheduler,
-                                          bkc, getConf().getBKDigestPW(), verbose, !getForce(), concurrency);
+                checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
+                                          verbose, !getForce(), concurrency);
             } finally {
                 SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
             }
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
index e551c22..a081606 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
@@ -21,18 +21,20 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.SettableFuture;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
 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.metadata.BKDLConfig;
+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;
@@ -52,8 +54,8 @@
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -72,7 +74,6 @@
 /**
  * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
  */
-@SuppressWarnings("deprecation")
 public class DLAuditor {
 
     private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
@@ -83,23 +84,23 @@
         this.conf = conf;
     }
 
-    private ZooKeeperClient getZooKeeperClient(com.twitter.distributedlog.DistributedLogManagerFactory factory) {
-        DistributedLogNamespace namespace = factory.getNamespace();
-        assert(namespace instanceof BKDistributedLogNamespace);
-        return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+    private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assert(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getWriterZKC();
     }
 
-    private BookKeeperClient getBookKeeperClient(com.twitter.distributedlog.DistributedLogManagerFactory factory) {
-        DistributedLogNamespace namespace = factory.getNamespace();
-        assert(namespace instanceof BKDistributedLogNamespace);
-        return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+    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 = DLUtils.getZKServersFromDLUri(firstURI);
+        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
         for (URI uri : uris) {
-            if (!zkServers.equalsIgnoreCase(DLUtils.getZKServersFromDLUri(uri))) {
+            if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
                 throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
             }
         }
@@ -224,19 +225,23 @@
     private Set<Long> collectLedgersFromDL(List<URI> uris, List<List<String>> allocationPaths)
             throws IOException {
         final Set<Long> ledgers = new TreeSet<Long>();
-        List<com.twitter.distributedlog.DistributedLogManagerFactory> factories =
-                new ArrayList<com.twitter.distributedlog.DistributedLogManagerFactory>(uris.size());
+        List<DistributedLogNamespace> namespaces =
+                new ArrayList<DistributedLogNamespace>(uris.size());
         try {
             for (URI uri : uris) {
-                factories.add(new com.twitter.distributedlog.DistributedLogManagerFactory(conf, uri));
+                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 (com.twitter.distributedlog.DistributedLogManagerFactory factory : factories) {
-                    final com.twitter.distributedlog.DistributedLogManagerFactory dlFactory = factory;
+                for (final DistributedLogNamespace namespace : namespaces) {
+                    final DistributedLogNamespace dlNamespace = namespace;
                     final URI uri = uris.get(i);
                     final List<String> aps = allocationPaths.get(i);
                     i++;
@@ -245,12 +250,12 @@
                         public void run() {
                             try {
                                 logger.info("Collecting ledgers from {} : {}", uri, aps);
-                                collectLedgersFromAllocator(uri, dlFactory, aps, ledgers);
+                                collectLedgersFromAllocator(uri, namespace, aps, ledgers);
                                 synchronized (ledgers) {
                                     logger.info("Collected {} ledgers from allocators for {} : {} ",
                                             new Object[]{ledgers.size(), uri, ledgers});
                                 }
-                                collectLedgersFromDL(uri, dlFactory, ledgers);
+                                collectLedgersFromDL(uri, namespace, ledgers);
                             } catch (IOException e) {
                                 numFailures.incrementAndGet();
                                 logger.info("Error to collect ledgers from DL : ", e);
@@ -273,15 +278,15 @@
                 executor.shutdown();
             }
         } finally {
-            for (com.twitter.distributedlog.DistributedLogManagerFactory factory : factories) {
-                factory.close();
+            for (DistributedLogNamespace namespace : namespaces) {
+                namespace.close();
             }
         }
         return ledgers;
     }
 
     private void collectLedgersFromAllocator(final URI uri,
-                                             final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+                                             final DistributedLogNamespace namespace,
                                              final List<String> allocationPaths,
                                              final Set<Long> ledgers) throws IOException {
         final LinkedBlockingQueue<String> poolQueue =
@@ -289,7 +294,7 @@
         for (String allocationPath : allocationPaths) {
             String rootPath = uri.getPath() + "/" + allocationPath;
             try {
-                List<String> pools = getZooKeeperClient(factory).get().getChildren(rootPath, false);
+                List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
                 for (String pool : pools) {
                     poolQueue.add(rootPath + "/" + pool);
                 }
@@ -318,11 +323,11 @@
 
             private void collectLedgersFromPool(String poolPath)
                     throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
-                List<String> allocators = getZooKeeperClient(factory).get()
+                List<String> allocators = getZooKeeperClient(namespace).get()
                                         .getChildren(poolPath, false);
                 for (String allocator : allocators) {
                     String allocatorPath = poolPath + "/" + allocator;
-                    byte[] data = getZooKeeperClient(factory).get().getData(allocatorPath, false, new Stat());
+                    byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
                     if (null != data && data.length > 0) {
                         try {
                             long ledgerId = DLUtils.bytes2LogSegmentId(data);
@@ -341,30 +346,31 @@
     }
 
     private void collectLedgersFromDL(final URI uri,
-                                      final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+                                      final DistributedLogNamespace namespace,
                                       final Set<Long> ledgers) throws IOException {
         logger.info("Enumerating {} to collect streams.", uri);
-        Collection<String> streams = factory.enumerateAllLogsInNamespace();
+        Iterator<String> streams = namespace.getLogs();
         final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
-        streamQueue.addAll(streams);
+        while (streams.hasNext()) {
+            streamQueue.add(streams.next());
+        }
 
         logger.info("Collected {} streams from uri {} : {}",
-                    new Object[] { streams.size(), uri, streams });
+                    new Object[] { streamQueue.size(), uri, streams });
 
         executeAction(streamQueue, 10, new Action<String>() {
             @Override
             public void execute(String stream) throws IOException {
-                collectLedgersFromStream(factory, stream, ledgers);
+                collectLedgersFromStream(namespace, stream, ledgers);
             }
         });
     }
 
-    private List<Long> collectLedgersFromStream(com.twitter.distributedlog.DistributedLogManagerFactory factory,
+    private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
                                                 String stream,
                                                 Set<Long> ledgers)
             throws IOException {
-        DistributedLogManager dlm = factory.createDistributedLogManager(stream,
-                com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption.SharedClients);
+        DistributedLogManager dlm = namespace.openLog(stream);
         try {
             List<LogSegmentMetadata> segments = dlm.getLogSegments();
             List<Long> sLedgers = new ArrayList<Long>();
@@ -388,21 +394,25 @@
      */
     public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
         logger.info("Collecting stream space usage for {}.", uri);
-        com.twitter.distributedlog.DistributedLogManagerFactory factory =
-                new com.twitter.distributedlog.DistributedLogManagerFactory(conf, uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
         try {
-            return calculateStreamSpaceUsage(uri, factory);
+            return calculateStreamSpaceUsage(uri, namespace);
         } finally {
-            factory.close();
+            namespace.close();
         }
     }
 
     private Map<String, Long> calculateStreamSpaceUsage(
-            final URI uri, final com.twitter.distributedlog.DistributedLogManagerFactory factory)
+            final URI uri, final DistributedLogNamespace namespace)
         throws IOException {
-        Collection<String> streams = factory.enumerateAllLogsInNamespace();
+        Iterator<String> streams = namespace.getLogs();
         final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
-        streamQueue.addAll(streams);
+        while (streams.hasNext()) {
+            streamQueue.add(streams.next());
+        }
 
         final Map<String, Long> streamSpaceUsageMap =
                 new ConcurrentSkipListMap<String, Long>();
@@ -412,7 +422,7 @@
             @Override
             public void execute(String stream) throws IOException {
                 streamSpaceUsageMap.put(stream,
-                        calculateStreamSpaceUsage(factory, stream));
+                        calculateStreamSpaceUsage(namespace, stream));
                 if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
                     logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
                 }
@@ -422,16 +432,15 @@
         return streamSpaceUsageMap;
     }
 
-    private long calculateStreamSpaceUsage(final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+    private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
                                            final String stream) throws IOException {
-        DistributedLogManager dlm = factory.createDistributedLogManager(stream,
-                com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption.SharedClients);
+        DistributedLogManager dlm = namespace.openLog(stream);
         long totalBytes = 0;
         try {
             List<LogSegmentMetadata> segments = dlm.getLogSegments();
             for (LogSegmentMetadata segment : segments) {
                 try {
-                    LedgerHandle lh = getBookKeeperClient(factory).get().openLedgerNoRecovery(segment.getLogSegmentId(),
+                    LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
                             BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
                     totalBytes += lh.getLength();
                     lh.close();
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java
deleted file mode 100644
index dd78a4e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.twitter.distributedlog.impl;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-
-import java.net.URI;
-
-/**
- * Utils for bookkeeper based distributedlog implementation
- */
-public class BKDLUtils {
-
-    /**
-     * 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 configuration and uri.
-     *
-     * @param conf
-     *          distributedlog configuration
-     * @param uri
-     *          distributedlog uri
-     * @throws IllegalArgumentException
-     */
-    public static void validateConfAndURI(DistributedLogConfiguration conf, URI uri)
-        throws IllegalArgumentException {
-        if (null == conf) {
-            throw new IllegalArgumentException("Incorrect Configuration");
-        } else {
-            conf.validate();
-        }
-        if ((null == uri) || (null == uri.getAuthority()) || (null == uri.getPath())) {
-            throw new IllegalArgumentException("Incorrect ZK URI");
-        }
-    }
-
-    /**
-     * Validate the stream name.
-     *
-     * @param nameOfStream
-     *          name of stream
-     * @throws InvalidStreamNameException
-     */
-    public static void validateName(String nameOfStream)
-            throws InvalidStreamNameException {
-        String reason = null;
-        char chars[] = nameOfStream.toCharArray();
-        char c;
-        // validate the stream to see if meet zookeeper path's requirement
-        for (int i = 0; i < chars.length; i++) {
-            c = chars[i];
-
-            if (c == 0) {
-                reason = "null character not allowed @" + i;
-                break;
-            } else if (c == '/') {
-                reason = "'/' not allowed @" + i;
-                break;
-            } else if (c > '\u0000' && c < '\u001f'
-                    || c > '\u007f' && c < '\u009F'
-                    || c > '\ud800' && c < '\uf8ff'
-                    || c > '\ufff0' && c < '\uffff') {
-                reason = "invalid charater @" + i;
-                break;
-            }
-        }
-        if (null != reason) {
-            throw new InvalidStreamNameException(nameOfStream, reason);
-        }
-        if (isReservedStreamName(nameOfStream)) {
-            throw new InvalidStreamNameException(nameOfStream,
-                    "Stream Name is reserved");
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
new file mode 100644
index 0000000..5921233
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.twitter.distributedlog.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.distributedlog.BookKeeperClient;
+import com.twitter.distributedlog.BookKeeperClientBuilder;
+import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.DistributedLogConstants;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.ZooKeeperClientBuilder;
+import com.twitter.distributedlog.acl.AccessControlManager;
+import com.twitter.distributedlog.acl.DefaultAccessControlManager;
+import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
+import com.twitter.distributedlog.bk.LedgerAllocator;
+import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.exceptions.AlreadyClosedException;
+import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
+import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
+import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
+import com.twitter.distributedlog.impl.subscription.ZKSubscriptionsStore;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.metadata.LogMetadataForReader;
+import com.twitter.distributedlog.metadata.LogMetadataStore;
+import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
+import com.twitter.distributedlog.namespace.NamespaceDriverManager;
+import com.twitter.distributedlog.subscription.SubscriptionsStore;
+import com.twitter.distributedlog.util.OrderedScheduler;
+import com.twitter.distributedlog.util.Utils;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.twitter.distributedlog.util.DLUtils.isReservedStreamName;
+import static com.twitter.distributedlog.util.DLUtils.validateName;
+
+/**
+ * Manager for ZooKeeper/BookKeeper based namespace
+ */
+public class BKNamespaceDriver implements NamespaceDriver {
+
+    private static Logger LOG = LoggerFactory.getLogger(BKNamespaceDriver.class);
+
+    // register itself
+    static {
+        NamespaceDriverManager.registerDriver(DistributedLogConstants.BACKEND_BK, BKNamespaceDriver.class);
+    }
+
+    /**
+     * Extract zk servers fro dl <i>namespace</i>.
+     *
+     * @param uri dl namespace
+     * @return zk servers
+     */
+    public static String getZKServersFromDLUri(URI uri) {
+        return uri.getAuthority().replace(";", ",");
+    }
+
+    // resources (passed from initialization)
+    private DistributedLogConfiguration conf;
+    private DynamicDistributedLogConfiguration dynConf;
+    private URI namespace;
+    private OrderedScheduler scheduler;
+    private FeatureProvider featureProvider;
+    private AsyncFailureInjector failureInjector;
+    private StatsLogger statsLogger;
+    private StatsLogger perLogStatsLogger;
+    private String clientId;
+    private int regionId;
+
+    //
+    // resources (created internally and initialized at #initialize())
+    //
+
+    // namespace binding
+    private BKDLConfig bkdlConfig;
+
+    // zookeeper clients
+    // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
+    //       {@link com.twitter.distributedlog.ZooKeeperClient#get()}. So it is safe to
+    //       keep builders and their client wrappers here, as they will be used when
+    //       instantiating readers or writers.
+    private ZooKeeperClientBuilder sharedWriterZKCBuilder;
+    private ZooKeeperClient writerZKC;
+    private ZooKeeperClientBuilder sharedReaderZKCBuilder;
+    private ZooKeeperClient readerZKC;
+    // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
+    //       {@link com.twitter.distributedlog.BookKeeperClient#get()}. So it is safe to
+    //       keep builders and their client wrappers here, as they will be used when
+    //       instantiating readers or writers.
+    private ClientSocketChannelFactory channelFactory;
+    private HashedWheelTimer requestTimer;
+    private BookKeeperClientBuilder sharedWriterBKCBuilder;
+    private BookKeeperClient writerBKC;
+    private BookKeeperClientBuilder sharedReaderBKCBuilder;
+    private BookKeeperClient readerBKC;
+
+    // log stream metadata store
+    private LogMetadataStore metadataStore;
+    private LogStreamMetadataStore writerStreamMetadataStore;
+    private LogStreamMetadataStore readerStreamMetadataStore;
+
+    //
+    // resources (lazily initialized)
+    //
+
+    // ledger allocator
+    private LedgerAllocator allocator;
+
+    // log segment entry stores
+    private LogSegmentEntryStore writerEntryStore;
+    private LogSegmentEntryStore readerEntryStore;
+
+    // access control manager
+    private AccessControlManager accessControlManager;
+
+    //
+    // states
+    //
+    protected boolean initialized = false;
+    protected AtomicBoolean closed = new AtomicBoolean(false);
+
+    /**
+     * Public constructor for reflection.
+     */
+    public BKNamespaceDriver() {
+    }
+
+    @Override
+    public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf,
+                                                   DynamicDistributedLogConfiguration dynConf,
+                                                   URI namespace,
+                                                   OrderedScheduler scheduler,
+                                                   FeatureProvider featureProvider,
+                                                   AsyncFailureInjector failureInjector,
+                                                   StatsLogger statsLogger,
+                                                   StatsLogger perLogStatsLogger,
+                                                   String clientId,
+                                                   int regionId) throws IOException {
+        if (initialized) {
+            return this;
+        }
+        // validate the namespace
+        if ((null == namespace) || (null == namespace.getAuthority()) || (null == namespace.getPath())) {
+            throw new IOException("Incorrect distributedlog namespace : " + namespace);
+        }
+
+        // initialize the resources
+        this.conf = conf;
+        this.dynConf = dynConf;
+        this.namespace = namespace;
+        this.scheduler = scheduler;
+        this.featureProvider = featureProvider;
+        this.failureInjector = failureInjector;
+        this.statsLogger = statsLogger;
+        this.perLogStatsLogger = perLogStatsLogger;
+        this.clientId = clientId;
+        this.regionId = regionId;
+
+        // initialize the zookeeper clients
+        initializeZooKeeperClients();
+
+        // initialize the bookkeeper clients
+        initializeBookKeeperClients();
+
+        // propagate bkdlConfig to configuration
+        BKDLConfig.propagateConfiguration(bkdlConfig, conf);
+
+        // initialize the log metadata & stream metadata store
+        initializeLogStreamMetadataStores();
+
+        // initialize other resources
+        initializeOtherResources();
+
+        initialized = true;
+
+        LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.",
+                new Object[]{clientId, regionId, bkdlConfig.isFederatedNamespace()});
+        return this;
+    }
+
+    private void initializeZooKeeperClients() throws IOException {
+        // Build the namespace zookeeper client
+        this.sharedWriterZKCBuilder = createZKClientBuilder(
+                String.format("dlzk:%s:factory_writer_shared", namespace),
+                conf,
+                getZKServersFromDLUri(namespace),
+                statsLogger.scope("dlzk_factory_writer_shared"));
+        this.writerZKC = sharedWriterZKCBuilder.build();
+
+        // Resolve namespace binding
+        this.bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, namespace);
+
+        // Build zookeeper client for readers
+        if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
+            this.sharedReaderZKCBuilder = this.sharedWriterZKCBuilder;
+        } else {
+            this.sharedReaderZKCBuilder = createZKClientBuilder(
+                    String.format("dlzk:%s:factory_reader_shared", namespace),
+                    conf,
+                    bkdlConfig.getDlZkServersForReader(),
+                    statsLogger.scope("dlzk_factory_reader_shared"));
+        }
+        this.readerZKC = this.sharedReaderZKCBuilder.build();
+    }
+
+    private synchronized BKDLConfig getBkdlConfig() {
+        return bkdlConfig;
+    }
+
+    private void initializeBookKeeperClients() throws IOException {
+        this.channelFactory = new NioClientSocketChannelFactory(
+                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-boss-%d").build()),
+                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-worker-%d").build()),
+                conf.getBKClientNumberIOThreads());
+        this.requestTimer = new HashedWheelTimer(
+                new ThreadFactoryBuilder().setNameFormat("DLFactoryTimer-%d").build(),
+                conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
+                conf.getTimeoutTimerNumTicks());
+        // Build bookkeeper client for writers
+        this.sharedWriterBKCBuilder = createBKCBuilder(
+                String.format("bk:%s:factory_writer_shared", namespace),
+                conf,
+                bkdlConfig.getBkZkServersForWriter(),
+                bkdlConfig.getBkLedgersPath(),
+                channelFactory,
+                requestTimer,
+                Optional.of(featureProvider.scope("bkc")),
+                statsLogger);
+        this.writerBKC = this.sharedWriterBKCBuilder.build();
+
+        // Build bookkeeper client for readers
+        if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
+            this.sharedReaderBKCBuilder = this.sharedWriterBKCBuilder;
+        } else {
+            this.sharedReaderBKCBuilder = createBKCBuilder(
+                    String.format("bk:%s:factory_reader_shared", namespace),
+                    conf,
+                    bkdlConfig.getBkZkServersForReader(),
+                    bkdlConfig.getBkLedgersPath(),
+                    channelFactory,
+                    requestTimer,
+                    Optional.<FeatureProvider>absent(),
+                    statsLogger);
+        }
+        this.readerBKC = this.sharedReaderBKCBuilder.build();
+    }
+
+    private void initializeLogStreamMetadataStores() throws IOException {
+        // log metadata store
+        if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
+            this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+        } else {
+            this.metadataStore = new ZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+        }
+
+        // create log stream metadata store
+        this.writerStreamMetadataStore =
+                new ZKLogStreamMetadataStore(
+                        clientId,
+                        conf,
+                        writerZKC,
+                        scheduler,
+                        statsLogger);
+        this.readerStreamMetadataStore =
+                new ZKLogStreamMetadataStore(
+                        clientId,
+                        conf,
+                        readerZKC,
+                        scheduler,
+                        statsLogger);
+    }
+
+    @VisibleForTesting
+    public static String validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException {
+        String poolPath = conf.getLedgerAllocatorPoolPath();
+        LOG.info("PoolPath is {}", poolPath);
+        if (null == poolPath || !poolPath.startsWith(".") || poolPath.endsWith("/")) {
+            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
+        }
+        String poolName = conf.getLedgerAllocatorPoolName();
+        if (null == poolName) {
+            LOG.error("No ledger allocator pool name specified when enabling ledger allocator pool.");
+            throw new IOException("No ledger allocator name specified when enabling ledger allocator pool.");
+        }
+        String rootPath = uri.getPath() + "/" + poolPath + "/" + poolName;
+        try {
+            PathUtils.validatePath(rootPath);
+        } catch (IllegalArgumentException iae) {
+            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
+        }
+        return rootPath;
+    }
+
+    private void initializeOtherResources() throws IOException {
+        // Ledger allocator
+        if (conf.getEnableLedgerAllocatorPool()) {
+            String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, namespace);
+            allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(
+                    allocatorPoolPath,
+                    conf.getLedgerAllocatorPoolCoreSize(),
+                    conf,
+                    writerZKC,
+                    writerBKC,
+                    scheduler);
+            if (null != allocator) {
+                allocator.start();
+            }
+            LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
+        } else {
+            allocator = null;
+        }
+
+    }
+
+    private void checkState() throws IOException {
+        if (closed.get()) {
+            LOG.error("BK namespace driver {} is already closed", namespace);
+            throw new AlreadyClosedException("BK namespace driver " + namespace + " is already closed");
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!closed.compareAndSet(false, true)) {
+            return;
+        }
+        doClose();
+    }
+
+    private void doClose() {
+        if (null != accessControlManager) {
+            accessControlManager.close();
+            LOG.info("Access Control Manager Stopped.");
+        }
+
+        // Close the allocator
+        if (null != allocator) {
+            Utils.closeQuietly(allocator);
+            LOG.info("Ledger Allocator stopped.");
+        }
+
+        // Shutdown log segment metadata stores
+        Utils.close(writerStreamMetadataStore);
+        Utils.close(readerStreamMetadataStore);
+
+        writerBKC.close();
+        readerBKC.close();
+        writerZKC.close();
+        readerZKC.close();
+        // release bookkeeper resources
+        channelFactory.releaseExternalResources();
+        LOG.info("Release external resources used by channel factory.");
+        requestTimer.stop();
+        LOG.info("Stopped request timer");
+    }
+
+    @Override
+    public URI getUri() {
+        return namespace;
+    }
+
+    @Override
+    public String getScheme() {
+        return DistributedLogConstants.BACKEND_BK;
+    }
+
+    @Override
+    public LogMetadataStore getLogMetadataStore() {
+        return metadataStore;
+    }
+
+    @Override
+    public LogStreamMetadataStore getLogStreamMetadataStore(Role role) {
+        if (Role.WRITER == role) {
+            return writerStreamMetadataStore;
+        } else {
+            return readerStreamMetadataStore;
+        }
+    }
+
+    @Override
+    public LogSegmentEntryStore getLogSegmentEntryStore(Role role) {
+        if (Role.WRITER == role) {
+            return getWriterEntryStore();
+        } else {
+            return getReaderEntryStore();
+        }
+    }
+
+    private LogSegmentEntryStore getWriterEntryStore() {
+        if (null == writerEntryStore) {
+            writerEntryStore = new BKLogSegmentEntryStore(
+                    conf,
+                    dynConf,
+                    writerZKC,
+                    writerBKC,
+                    scheduler,
+                    allocator,
+                    statsLogger,
+                    failureInjector);
+        }
+        return writerEntryStore;
+    }
+
+    private LogSegmentEntryStore getReaderEntryStore() {
+        if (null == readerEntryStore) {
+            readerEntryStore = new BKLogSegmentEntryStore(
+                    conf,
+                    dynConf,
+                    writerZKC,
+                    readerBKC,
+                    scheduler,
+                    allocator,
+                    statsLogger,
+                    failureInjector);
+        }
+        return readerEntryStore;
+    }
+
+    @Override
+    public AccessControlManager getAccessControlManager() throws IOException {
+        if (null == accessControlManager) {
+            String aclRootPath = getBkdlConfig().getACLRootPath();
+            // Build the access control manager
+            if (aclRootPath == null) {
+                accessControlManager = DefaultAccessControlManager.INSTANCE;
+                LOG.info("Created default access control manager for {}", namespace);
+            } else {
+                if (!isReservedStreamName(aclRootPath)) {
+                    throw new IOException("Invalid Access Control List Root Path : " + aclRootPath);
+                }
+                String zkRootPath = namespace.getPath() + "/" + aclRootPath;
+                LOG.info("Creating zk based access control manager @ {} for {}",
+                        zkRootPath, namespace);
+                accessControlManager = new ZKAccessControlManager(conf, readerZKC,
+                        zkRootPath, scheduler);
+                LOG.info("Created zk based access control manager @ {} for {}",
+                        zkRootPath, namespace);
+            }
+        }
+        return accessControlManager;
+    }
+
+    @Override
+    public SubscriptionsStore getSubscriptionsStore(String streamName) {
+        return new ZKSubscriptionsStore(
+                writerZKC,
+                LogMetadataForReader.getSubscribersPath(namespace, streamName, conf.getUnpartitionedStreamName()));
+    }
+
+    //
+    // Legacy Intefaces
+    //
+
+    @Override
+    public MetadataAccessor getMetadataAccessor(String streamName)
+            throws InvalidStreamNameException, IOException {
+        if (getBkdlConfig().isFederatedNamespace()) {
+            throw new UnsupportedOperationException();
+        }
+        checkState();
+        validateName(streamName);
+        return new ZKMetadataAccessor(
+                streamName,
+                conf,
+                namespace,
+                sharedWriterZKCBuilder,
+                sharedReaderZKCBuilder,
+                statsLogger);
+    }
+
+    public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
+        throws IOException, IllegalArgumentException {
+        String namespaceRootPath = namespace.getPath();
+        HashMap<String, byte[]> result = new HashMap<String, byte[]>();
+        ZooKeeperClient zkc = writerZKC;
+        try {
+            ZooKeeper zk = Utils.sync(zkc, namespaceRootPath);
+            Stat currentStat = zk.exists(namespaceRootPath, false);
+            if (currentStat == null) {
+                return result;
+            }
+            List<String> children = zk.getChildren(namespaceRootPath, false);
+            for(String child: children) {
+                if (isReservedStreamName(child)) {
+                    continue;
+                }
+                String zkPath = String.format("%s/%s", namespaceRootPath, child);
+                currentStat = zk.exists(zkPath, false);
+                if (currentStat == null) {
+                    result.put(child, new byte[0]);
+                } else {
+                    result.put(child, zk.getData(zkPath, false, currentStat));
+                }
+            }
+        } catch (InterruptedException ie) {
+            LOG.error("Interrupted while deleting " + namespaceRootPath, ie);
+            throw new IOException("Interrupted while reading " + namespaceRootPath, ie);
+        } catch (KeeperException ke) {
+            LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+            throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+        }
+        return result;
+    }
+
+    //
+    // Zk & Bk Utils
+    //
+
+    public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName,
+                                                               DistributedLogConfiguration conf,
+                                                               String zkServers,
+                                                               StatsLogger statsLogger) {
+        RetryPolicy retryPolicy = null;
+        if (conf.getZKNumRetries() > 0) {
+            retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
+        }
+        ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
+            .name(zkcName)
+            .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+            .retryThreadCount(conf.getZKClientNumberRetryThreads())
+            .requestRateLimit(conf.getZKRequestRateLimit())
+            .zkServers(zkServers)
+            .retryPolicy(retryPolicy)
+            .statsLogger(statsLogger)
+            .zkAclId(conf.getZkAclId());
+        LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}, retryBackoff = {},"
+                + " maxRetryBackoff = {}, zkAclId = {}.", new Object[] { zkcName, zkServers, conf.getZKNumRetries(),
+                conf.getZKSessionTimeoutMilliseconds(), conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() });
+        return builder;
+    }
+
+    private BookKeeperClientBuilder createBKCBuilder(String bkcName,
+                                                     DistributedLogConfiguration conf,
+                                                     String zkServers,
+                                                     String ledgersPath,
+                                                     ClientSocketChannelFactory channelFactory,
+                                                     HashedWheelTimer requestTimer,
+                                                     Optional<FeatureProvider> featureProviderOptional,
+                                                     StatsLogger statsLogger) {
+        BookKeeperClientBuilder builder = BookKeeperClientBuilder.newBuilder()
+                .name(bkcName)
+                .dlConfig(conf)
+                .zkServers(zkServers)
+                .ledgersPath(ledgersPath)
+                .channelFactory(channelFactory)
+                .requestTimer(requestTimer)
+                .featureProvider(featureProviderOptional)
+                .statsLogger(statsLogger);
+        LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}",
+                new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() });
+        return builder;
+    }
+
+    //
+    // Test Methods
+    //
+
+    @VisibleForTesting
+    public ZooKeeperClient getWriterZKC() {
+        return writerZKC;
+    }
+
+    @VisibleForTesting
+    public BookKeeperClient getReaderBKC() {
+        return readerBKC;
+    }
+
+    @VisibleForTesting
+    public AsyncFailureInjector getFailureInjector() {
+        return this.failureInjector;
+    }
+
+    @VisibleForTesting
+    public LogStreamMetadataStore getWriterStreamMetadataStore() {
+        return writerStreamMetadataStore;
+    }
+
+    @VisibleForTesting
+    public LedgerAllocator getLedgerAllocator() {
+        return allocator;
+    }
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
index b84ab2e..50b1405 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
@@ -36,7 +36,7 @@
 import java.util.Iterator;
 import java.util.List;
 
-import static com.twitter.distributedlog.impl.BKDLUtils.*;
+import static com.twitter.distributedlog.util.DLUtils.*;
 
 /**
  * ZooKeeper based log metadata store
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
similarity index 95%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
index 4d7a0e1..eeda804 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
@@ -15,16 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog;
+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.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.Utils;
 import com.twitter.util.Future;
@@ -37,6 +40,8 @@
 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;
@@ -87,7 +92,7 @@
         this.writerZKC = this.writerZKCBuilder.build();
 
         if (null == readerZKCBuilder) {
-            String zkServersForWriter = DLUtils.getZKServersFromDLUri(uri);
+            String zkServersForWriter = getZKServersFromDLUri(uri);
             String zkServersForReader;
             try {
                 BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri);
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
index 61c1760..06bc8fb 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
@@ -37,7 +37,7 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import static com.twitter.distributedlog.impl.BKDLUtils.*;
+import static com.twitter.distributedlog.util.DLUtils.*;
 
 /**
  * Watcher on watching a given namespace
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
similarity index 98%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
index bf16256..8126723 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.acl;
+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;
@@ -89,10 +90,12 @@
         return sb.toString();
     }
 
+    @VisibleForTesting
     public String getZKPath() {
         return zkPath;
     }
 
+    @VisibleForTesting
     public AccessControlEntry getAccessControlEntry() {
         return accessControlEntry;
     }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
similarity index 99%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
index 9c89b4a..0c90a50 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.acl;
+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;
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
new file mode 100644
index 0000000..d7ff4fb
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.twitter.distributedlog.impl.logsegment;
+
+import com.twitter.distributedlog.bk.LedgerAllocator;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
+import com.twitter.distributedlog.util.Allocator;
+import com.twitter.distributedlog.util.Transaction;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.client.LedgerHandle;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+
+import java.io.IOException;
+
+/**
+ * Allocate log segments
+ */
+class BKLogSegmentAllocator implements Allocator<LogSegmentEntryWriter, Object> {
+
+    private static class NewLogSegmentEntryWriterFn extends AbstractFunction1<LedgerHandle, LogSegmentEntryWriter> {
+
+        static final Function1<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
+                new NewLogSegmentEntryWriterFn();
+
+        private NewLogSegmentEntryWriterFn() {}
+
+        @Override
+        public LogSegmentEntryWriter apply(LedgerHandle lh) {
+            return new BKLogSegmentEntryWriter(lh);
+        }
+    }
+
+    LedgerAllocator allocator;
+
+    BKLogSegmentAllocator(LedgerAllocator allocator) {
+        this.allocator = allocator;
+    }
+
+    @Override
+    public void allocate() throws IOException {
+        allocator.allocate();
+    }
+
+    @Override
+    public Future<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
+                                                   final Transaction.OpListener<LogSegmentEntryWriter> listener) {
+        return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
+            @Override
+            public void onCommit(LedgerHandle lh) {
+                listener.onCommit(new BKLogSegmentEntryWriter(lh));
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                listener.onAbort(t);
+            }
+        }).map(NewLogSegmentEntryWriterFn.INSTANCE);
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return allocator.asyncClose();
+    }
+
+    @Override
+    public Future<Void> delete() {
+        return allocator.delete();
+    }
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
index dc382d2..f85760d 100644
--- 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
@@ -48,6 +48,7 @@
 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;
@@ -209,6 +210,7 @@
                 int numErrors = Math.max(1, numReadErrors.incrementAndGet());
                 int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
                 scheduler.schedule(
+                        getSegment().getLogSegmentId(),
                         this,
                         nextReadBackoffTime,
                         TimeUnit.MILLISECONDS);
@@ -284,6 +286,8 @@
     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;
@@ -374,6 +378,24 @@
         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
     //
@@ -440,7 +462,7 @@
             return;
         }
         // the reader is still catching up, retry opening the log segment later
-        scheduler.schedule(new Runnable() {
+        scheduler.schedule(segment.getLogSegmentId(), new Runnable() {
             @Override
             public void run() {
                 onLogSegmentMetadataUpdated(segment);
@@ -583,6 +605,7 @@
 
         if (!hasCaughtupOnInprogress) {
             hasCaughtupOnInprogress = true;
+            notifyCaughtupOnInprogress();
         }
         getLh().asyncReadLastConfirmedAndEntry(
                 cacheEntry.entryId,
@@ -633,7 +656,7 @@
 
         long prevCount = scheduleCount.getAndIncrement();
         if (0 == prevCount) {
-            scheduler.submit(this);
+            scheduler.submit(getSegment().getLogSegmentId(), this);
         }
     }
 
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
index f7f4acf..91e6dec 100644
--- 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
@@ -20,12 +20,21 @@
 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;
@@ -80,21 +89,31 @@
     }
 
     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;
     }
@@ -129,12 +148,44 @@
         FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
     }
 
-    @Override
-    public Future<LogSegmentEntryWriter> openWriter(LogSegmentMetadata segment) {
-        throw new UnsupportedOperationException("Not supported yet");
+    //
+    // 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;
@@ -220,15 +271,15 @@
                     segment.getLogSegmentId(),
                     BookKeeper.DigestType.CRC32,
                     passwd,
-                    this,
-                    openCallback);
+                    openCallback,
+                    null);
         } else {
             bk.asyncOpenLedger(
                     segment.getLogSegmentId(),
                     BookKeeper.DigestType.CRC32,
                     passwd,
-                    this,
-                    openCallback);
+                    openCallback,
+                    null);
         }
         return openPromise;
     }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
similarity index 97%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
index ac36ef2..3e859fb 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.metadata;
+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 com.twitter.distributedlog.util.DLUtils;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.apache.thrift.transport.TMemoryBuffer;
@@ -96,9 +97,9 @@
     /**
      * Construct a empty config with given <i>uri</i>.
      */
-    BKDLConfig(URI uri) {
-        this(DLUtils.getZKServersFromDLUri(uri),
-             DLUtils.getZKServersFromDLUri(uri),
+    public BKDLConfig(URI uri) {
+        this(BKNamespaceDriver.getZKServersFromDLUri(uri),
+             BKNamespaceDriver.getZKServersFromDLUri(uri),
              null, null, null);
     }
 
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
similarity index 94%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
index 303fbe6..6b7a231 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
@@ -15,9 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.metadata;
+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;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
similarity index 97%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
index 9cd2da5..b067ee9 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.subscription;
+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;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
similarity index 96%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
index f1e6251..17ba943 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.subscription;
+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.Function;
 import com.twitter.util.Future;
 import com.twitter.util.Promise;
 
@@ -30,8 +31,6 @@
 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 scala.runtime.AbstractFunction1;
 import scala.runtime.BoxedUnit;
 
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
index 07387cb..81eb5ed 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
@@ -32,6 +32,15 @@
 @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)}
@@ -39,6 +48,22 @@
     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
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
index 850f9c8..bcf8129 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
@@ -19,8 +19,14 @@
 
 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
  */
@@ -36,12 +42,14 @@
     Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
 
     /**
-     * Open the writer for writing data to the log <i>segment</i>.
+     * Create a new log segment allocator for allocating log segment entry writers.
      *
-     * @param segment the log <i>segment</i> to write data to
-     * @return future represent the opened writer
+     * @param metadata the metadata for the log stream
+     * @return future represent the log segment allocator
      */
-    Future<LogSegmentEntryWriter> openWriter(LogSegmentMetadata segment);
+    Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
+            LogMetadataForWriter metadata,
+            DynamicDistributedLogConfiguration dynConf) throws IOException;
 
     /**
      * Open the reader for reading data to the log <i>segment</i>.
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
index e0331c6..c0b5fb7 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
@@ -18,6 +18,7 @@
 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;
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
index b5abe9f..5d1d888 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
@@ -70,6 +70,13 @@
 @Beta
 public interface DistributedLogNamespace {
 
+    /**
+     * Get the namespace driver used by this namespace.
+     *
+     * @return namespace driver
+     */
+    NamespaceDriver getNamespaceDriver();
+
     //
     // Method to operate logs
     //
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
index a01bb70..07b3848 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
@@ -17,22 +17,29 @@
  */
 package com.twitter.distributedlog.namespace;
 
-import com.google.common.base.Objects;
 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.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 
 /**
  * Builder to construct a <code>DistributedLogNamespace</code>.
@@ -50,6 +57,7 @@
     }
 
     private DistributedLogConfiguration _conf = null;
+    private DynamicDistributedLogConfiguration _dynConf = null;
     private URI _uri = null;
     private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
     private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
@@ -73,6 +81,17 @@
     }
 
     /**
+     * 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
@@ -146,6 +165,18 @@
         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.
      *
@@ -160,25 +191,17 @@
         Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
         Preconditions.checkNotNull(_uri, "No DistributedLog URI");
 
-        // Validate the uri and load the backend according to scheme
-        String scheme = _uri.getScheme();
-        Preconditions.checkNotNull(scheme, "Invalid DistributedLog URI : " + _uri);
-        String[] schemeParts = StringUtils.split(scheme, '-');
-        Preconditions.checkArgument(schemeParts.length > 0,
-                "Invalid distributedlog scheme found : " + _uri);
-        Preconditions.checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
-                "Unknown distributedlog scheme found : " + _uri);
-
-        // both distributedlog: & distributedlog-bk: use bookkeeper as the backend
-        // TODO: we could do reflection to load backend in future.
-        //       if we are going to support other backends : e.g. 'distributedlog-mem:'.
-        if (schemeParts.length > 1) {
-            String backendProvider = schemeParts[1];
-            Preconditions.checkArgument(Objects.equal(DistributedLogConstants.BACKEND_BK, backendProvider.toLowerCase()),
-                    "Backend '" + backendProvider + "' is not supported yet.");
+        // validate the configuration
+        _conf.validate();
+        if (null == _dynConf) {
+            _dynConf = ConfUtils.getConstDynConf(_conf);
         }
 
-        // Built the feature provider
+        // 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);
@@ -187,25 +210,69 @@
             featureProvider = _featureProvider;
         }
 
-        URI bkUri;
-        try {
-            bkUri = new URI(
-                    schemeParts[0],     // remove backend info from bookkeeper backend
-                    _uri.getAuthority(),
-                    _uri.getPath(),
-                    _uri.getQuery(),
-                    _uri.getFragment());
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException("Invalid distributedlog uri found : " + _uri, e);
+        // 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 BKDistributedLogNamespace.newBuilder()
-                .conf(_conf)
-                .uri(bkUri)
-                .statsLogger(_statsLogger)
-                .featureProvider(featureProvider)
-                .clientId(_clientId)
-                .regionId(_regionId)
-                .build();
+        return new BKDistributedLogNamespace(
+                _conf,
+                normalizedUri,
+                driver,
+                scheduler,
+                featureProvider,
+                writeLimiter,
+                failureInjector,
+                _statsLogger,
+                perLogStatsLogger,
+                DLUtils.normalizeClientId(_clientId),
+                _regionId);
     }
 }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
new file mode 100644
index 0000000..738f124
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.twitter.distributedlog.namespace;
+
+import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.acl.AccessControlManager;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
+import com.twitter.distributedlog.metadata.LogMetadataStore;
+import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.subscription.SubscriptionsStore;
+import com.twitter.distributedlog.util.OrderedScheduler;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Manager to manage all the stores required by a namespace.
+ */
+public interface NamespaceDriver extends Closeable {
+
+    enum Role {
+        WRITER,
+        READER
+    }
+
+    /**
+     * Initialize the namespace manager.
+     *
+     * @param conf distributedlog configuration
+     * @param dynConf dynamic distributedlog configuration
+     * @param namespace root uri of the namespace
+     * @param scheduler ordered scheduler
+     * @param featureProvider feature provider
+     * @param statsLogger stats logger
+     * @param perLogStatsLogger per log stream stats logger
+     * @param clientId client id
+     * @return namespace manager
+     * @throws IOException when failed to initialize the namespace manager
+     */
+    NamespaceDriver initialize(DistributedLogConfiguration conf,
+                               DynamicDistributedLogConfiguration dynConf,
+                               URI namespace,
+                               OrderedScheduler scheduler,
+                               FeatureProvider featureProvider,
+                               AsyncFailureInjector failureInjector,
+                               StatsLogger statsLogger,
+                               StatsLogger perLogStatsLogger,
+                               String clientId,
+                               int regionId) throws IOException;
+
+    /**
+     * Get the scheme of the namespace driver.
+     *
+     * @return the scheme of the namespace driver.
+     */
+    String getScheme();
+
+    /**
+     * Get the root uri of the namespace driver.
+     *
+     * @return the root uri of the namespace driver.
+     */
+    URI getUri();
+
+    /**
+     * Retrieve the log {@code metadata store} used by the namespace.
+     *
+     * @return the log metadata store
+     */
+    LogMetadataStore getLogMetadataStore();
+
+    /**
+     * Retrieve the log stream {@code metadata store} used by the namespace.
+     *
+     * @param role the role to retrieve the log stream metadata store.
+     * @return the log stream metadata store
+     */
+    LogStreamMetadataStore getLogStreamMetadataStore(Role role);
+
+    /**
+     * Retrieve the log segment {@code entry store} used by the namespace.
+     *
+     * @param role the role to retrieve the log segment entry store.
+     * @return the log segment entry store.
+     * @throws IOException when failed to open log segment entry store.
+     */
+    LogSegmentEntryStore getLogSegmentEntryStore(Role role);
+
+    /**
+     * Create an access control manager to manage/check acl for logs.
+     *
+     * @return access control manager for logs under the namespace.
+     * @throws IOException
+     */
+    AccessControlManager getAccessControlManager()
+            throws IOException;
+
+    /**
+     * Retrieve the metadata accessor for log stream {@code streamName}.
+     * (TODO: it is a legacy interface. should remove it if we have metadata of stream.)
+     *
+     * @param streamName name of log stream.
+     * @return metadata accessor for log stream {@code streamName}.
+     */
+    MetadataAccessor getMetadataAccessor(String streamName)
+            throws InvalidStreamNameException, IOException;
+
+    /**
+     * Retrieve the subscriptions store for log stream {@code streamName}.
+     *
+     * @return the subscriptions store for log stream {@code streamName}
+     */
+    SubscriptionsStore getSubscriptionsStore(String streamName);
+
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
new file mode 100644
index 0000000..79945ad
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.twitter.distributedlog.namespace;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Sets;
+import com.twitter.distributedlog.DistributedLogConstants;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static com.google.common.base.Preconditions.*;
+
+/**
+ * The basic service for managing a set of namespace drivers.
+ */
+public class NamespaceDriverManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(NamespaceDriverManager.class);
+
+    static class NamespaceDriverInfo {
+
+        final Class<? extends NamespaceDriver> driverClass;
+        final String driverClassName;
+
+        NamespaceDriverInfo(Class<? extends NamespaceDriver> driverClass) {
+            this.driverClass = driverClass;
+            this.driverClassName = this.driverClass.getName();
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append("driver[")
+                    .append(driverClassName)
+                    .append("]");
+            return sb.toString();
+        }
+    }
+
+    private static final ConcurrentMap<String, NamespaceDriverInfo> drivers;
+    private static boolean initialized = false;
+
+    static {
+        drivers = new ConcurrentHashMap<String, NamespaceDriverInfo>();
+        initialize();
+    }
+
+    static void initialize() {
+        if (initialized) {
+            return;
+        }
+        loadInitialDrivers();
+        initialized = true;
+        logger.info("DistributedLog NamespaceDriverManager initialized");
+    }
+
+    private static void loadInitialDrivers() {
+        Set<String> driverList = Sets.newHashSet();
+        // add default bookkeeper based driver
+        driverList.add(BKNamespaceDriver.class.getName());
+        // load drivers from system property
+        String driversStr = System.getProperty("distributedlog.namespace.drivers");
+        if (null != driversStr) {
+            String[] driversArray = StringUtils.split(driversStr, ':');
+            for (String driver : driversArray) {
+                driverList.add(driver);
+            }
+        }
+        // initialize the drivers
+        for (String driverClsName : driverList) {
+            try {
+                NamespaceDriver driver =
+                        ReflectionUtils.newInstance(driverClsName, NamespaceDriver.class);
+                NamespaceDriverInfo driverInfo = new NamespaceDriverInfo(driver.getClass());
+                drivers.put(driver.getScheme().toLowerCase(), driverInfo);
+            } catch (Exception ex) {
+                logger.warn("Failed to load namespace driver {} : ", driverClsName, ex);
+            }
+        }
+    }
+
+    /**
+     * Prevent the NamespaceDriverManager class from being instantiated.
+     */
+    private NamespaceDriverManager() {}
+
+    /**
+     * Register the namespace {@code driver}.
+     *
+     * @param driver the namespace driver
+     * @return the namespace driver manager
+     */
+    public static void registerDriver(String backend, Class<? extends NamespaceDriver> driver) {
+        if (!initialized) {
+            initialize();
+        }
+
+        String scheme = backend.toLowerCase();
+        NamespaceDriverInfo oldDriverInfo = drivers.get(scheme);
+        if (null != oldDriverInfo) {
+            return;
+        }
+        NamespaceDriverInfo newDriverInfo = new NamespaceDriverInfo(driver);
+        oldDriverInfo = drivers.putIfAbsent(scheme, newDriverInfo);
+        if (null != oldDriverInfo) {
+            logger.debug("Driver for {} is already there.", scheme);
+        }
+    }
+
+    /**
+     * Retrieve the namespace driver for {@code scheme}.
+     *
+     * @param scheme the scheme for the namespace driver
+     * @return the namespace driver
+     * @throws NullPointerException when scheme is null
+     */
+    public static NamespaceDriver getDriver(String scheme) {
+        checkNotNull(scheme, "Driver Scheme is null");
+        if (!initialized) {
+            initialize();
+        }
+        NamespaceDriverInfo driverInfo = drivers.get(scheme.toLowerCase());
+        if (null == driverInfo) {
+            throw new IllegalArgumentException("Unknown backend " + scheme);
+        }
+        return ReflectionUtils.newInstance(driverInfo.driverClass);
+    }
+
+    /**
+     * Retrieve the namespace driver for {@code uri}.
+     *
+     * @param uri the distributedlog uri
+     * @return the namespace driver for {@code uri}
+     * @throws NullPointerException if the distributedlog {@code uri} is null or doesn't have scheme
+     *          or there is no namespace driver registered for the scheme
+     * @throws IllegalArgumentException if the distributedlog {@code uri} scheme is illegal
+     */
+    public static NamespaceDriver getDriver(URI uri) {
+        // Validate the uri and load the backend according to scheme
+        checkNotNull(uri, "DistributedLog uri is null");
+        String scheme = uri.getScheme();
+        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+        scheme = scheme.toLowerCase();
+        String[] schemeParts = StringUtils.split(scheme, '-');
+        checkArgument(schemeParts.length > 0,
+                "Invalid distributedlog scheme found : " + uri);
+        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+                "Unknown distributedlog scheme found : " + uri);
+        // bookkeeper is the default backend
+        String backend = DistributedLogConstants.BACKEND_BK;
+        if (schemeParts.length > 1) {
+            backend = schemeParts[1];
+        }
+        return getDriver(backend);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
index 30d6908..4565921 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
@@ -30,7 +30,6 @@
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Enumeration;
@@ -53,11 +52,16 @@
 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;
@@ -100,17 +104,15 @@
 import com.twitter.distributedlog.auditor.DLAuditor;
 import com.twitter.distributedlog.bk.LedgerAllocator;
 import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+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.Future;
 import com.twitter.util.FutureEventListener;
 
 import static com.google.common.base.Charsets.UTF_8;
 
-@SuppressWarnings("deprecation")
 public class DistributedLogTool extends Tool {
 
     static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
@@ -161,7 +163,7 @@
         protected URI uri;
         protected String zkAclId = null;
         protected boolean force = false;
-        protected com.twitter.distributedlog.DistributedLogManagerFactory factory = null;
+        protected DistributedLogNamespace namespace = null;
 
         protected PerDLCommand(String name, String description) {
             super(name, description);
@@ -187,8 +189,8 @@
                 return runCmd();
             } finally {
                 synchronized (this) {
-                    if (null != factory) {
-                        factory.close();
+                    if (null != namespace) {
+                        namespace.close();
                     }
                 }
             }
@@ -252,35 +254,33 @@
             this.force = force;
         }
 
-        protected synchronized com.twitter.distributedlog.DistributedLogManagerFactory getFactory() throws IOException {
-            if (null == this.factory) {
-                this.factory = new com.twitter.distributedlog.DistributedLogManagerFactory(getConf(), getUri());
-                logger.info("Construct DLM : uri = {}", getUri());
-            }
-            return this.factory;
-        }
-
         protected DistributedLogNamespace getNamespace() throws IOException {
-            return getFactory().getNamespace();
+            if (null == this.namespace) {
+                this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+                        .uri(getUri())
+                        .conf(getConf())
+                        .build();
+            }
+            return this.namespace;
         }
 
         protected LogSegmentMetadataStore getLogSegmentMetadataStore() throws IOException {
-            DistributedLogNamespace namespace = getFactory().getNamespace();
-            assert(namespace instanceof BKDistributedLogNamespace);
-            return ((BKDistributedLogNamespace) namespace).getWriterStreamMetadataStore()
+            return getNamespace()
+                    .getNamespaceDriver()
+                    .getLogStreamMetadataStore(NamespaceDriver.Role.READER)
                     .getLogSegmentMetadataStore();
         }
 
         protected ZooKeeperClient getZooKeeperClient() throws IOException {
-            DistributedLogNamespace namespace = getFactory().getNamespace();
-            assert(namespace instanceof BKDistributedLogNamespace);
-            return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+            NamespaceDriver driver = getNamespace().getNamespaceDriver();
+            assert(driver instanceof BKNamespaceDriver);
+            return ((BKNamespaceDriver) driver).getWriterZKC();
         }
 
         protected BookKeeperClient getBookKeeperClient() throws IOException {
-            DistributedLogNamespace namespace = getFactory().getNamespace();
-            assert(namespace instanceof BKDistributedLogNamespace);
-            return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+            NamespaceDriver driver = getNamespace().getNamespaceDriver();
+            assert(driver instanceof BKNamespaceDriver);
+            return ((BKNamespaceDriver) driver).getReaderBKC();
         }
     }
 
@@ -347,6 +347,10 @@
         }
     }
 
+    /**
+     * 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;
@@ -380,8 +384,12 @@
             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 = getZooKeeperClient().get().getChildren(rootPath, false);
+                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) {
@@ -401,7 +409,7 @@
                                     try {
                                         LedgerAllocator allocator =
                                                 LedgerAllocatorUtils.createLedgerAllocatorPool(poolPath, 0, getConf(),
-                                                        getZooKeeperClient(), getBookKeeperClient(),
+                                                        zkc, bkc,
                                                         allocationExecutor);
                                         allocator.delete();
                                         System.out.println("Deleted allocator pool : " + poolPath + " .");
@@ -454,43 +462,35 @@
 
         @Override
         protected int runCmd() throws Exception {
-            if (printMetadata) {
-                printStreamsWithMetadata(getFactory());
-            } else {
-                printStreams(getFactory());
-            }
+            printStreams(getNamespace());
             return 0;
         }
 
-        protected void printStreamsWithMetadata(com.twitter.distributedlog.DistributedLogManagerFactory factory)
-                throws Exception {
-            Map<String, byte[]> streams = factory.enumerateLogsWithMetadataInNamespace();
+        protected void printStreams(DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streams = namespace.getLogs();
             System.out.println("Streams under " + getUri() + " : ");
             System.out.println("--------------------------------");
-            for (Map.Entry<String, byte[]> entry : streams.entrySet()) {
-                println(entry.getKey());
-                if (null == entry.getValue() || entry.getValue().length == 0) {
+            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(entry.getValue()));
+                    System.out.println(Hex.encodeHexString(metadata));
                 } else {
-                    System.out.println(new String(entry.getValue(), UTF_8));
+                    System.out.println(new String(metadata, UTF_8));
                 }
                 System.out.println("");
             }
             System.out.println("--------------------------------");
         }
-
-        protected void printStreams(com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
-            Collection<String> streams = factory.enumerateAllLogsInNamespace();
-            System.out.println("Streams under " + getUri() + " : ");
-            System.out.println("--------------------------------");
-            for (String stream : streams) {
-                System.out.println(stream);
-            }
-            System.out.println("--------------------------------");
-        }
     }
 
     public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
@@ -609,16 +609,17 @@
 
         private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
                 throws Exception {
-            Collection<String> streamCollection = getFactory().enumerateAllLogsInNamespace();
+            Iterator<String> streamCollection = getNamespace().getLogs();
             final List<String> streams = new ArrayList<String>();
-            if (null != streamPrefix) {
-                for (String s : streamCollection) {
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
                     if (s.startsWith(streamPrefix)) {
                         streams.add(s);
                     }
+                } else {
+                    streams.add(s);
                 }
-            } else {
-                streams.addAll(streamCollection);
             }
             if (0 == streams.size()) {
                 return;
@@ -660,8 +661,7 @@
             for (int i = startIdx; i < endIdx; i++) {
                 String s = streams.get(i);
                 BookKeeperClient bkc = getBookKeeperClient();
-                DistributedLogManager dlm =
-                        getFactory().createDistributedLogManagerWithSharedClients(s);
+                DistributedLogManager dlm = getNamespace().openLog(s);
                 try {
                     List<LogSegmentMetadata> segments = dlm.getLogSegments();
                     if (segments.size() <= 1) {
@@ -782,20 +782,21 @@
         @Override
         protected int runCmd() throws Exception {
             getConf().setZkAclId(getZkAclId());
-            return truncateStreams(getFactory());
+            return truncateStreams(getNamespace());
         }
 
-        private int truncateStreams(final com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
-            Collection<String> streamCollection = factory.enumerateAllLogsInNamespace();
+        private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streamCollection = namespace.getLogs();
             final List<String> streams = new ArrayList<String>();
-            if (null != streamPrefix) {
-                for (String s : streamCollection) {
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
                     if (s.startsWith(streamPrefix)) {
                         streams.add(s);
                     }
+                } else {
+                    streams.add(s);
                 }
-            } else {
-                streams.addAll(streamCollection);
             }
             if (0 == streams.size()) {
                 return 0;
@@ -813,7 +814,7 @@
                     @Override
                     public void run() {
                         try {
-                            truncateStreams(factory, streams, tid, numStreamsPerThreads);
+                            truncateStreams(namespace, streams, tid, numStreamsPerThreads);
                             System.out.println("Thread " + tid + " finished.");
                         } catch (IOException e) {
                             System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
@@ -828,14 +829,13 @@
             return 0;
         }
 
-        private void truncateStreams(com.twitter.distributedlog.DistributedLogManagerFactory factory, List<String> streams,
+        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 =
-                        factory.createDistributedLogManagerWithSharedClients(s);
+                DistributedLogManager dlm = namespace.openLog(s);
                 try {
                     if (deleteStream) {
                         dlm.delete();
@@ -930,7 +930,7 @@
 
         @Override
         protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
             try {
                 if (listEppStats) {
                     bkc = new SimpleBookKeeperClient(getConf(), getUri());
@@ -1078,7 +1078,7 @@
 
         @Override
         protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
             try {
                 long count = 0;
                 if (null == endDLSN) {
@@ -1141,7 +1141,7 @@
         @Override
         protected int runCmd() throws Exception {
             getConf().setZkAclId(getZkAclId());
-            DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
             try {
                 dlm.delete();
             } finally {
@@ -1347,7 +1347,7 @@
             }
             getConf().setZkAclId(getZkAclId());
             for (String stream : streams) {
-                getFactory().getNamespace().createLog(stream);
+                getNamespace().createLog(stream);
             }
             return 0;
         }
@@ -1435,7 +1435,7 @@
 
         @Override
         protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
             long totalCount = dlm.getLogRecordCount();
             try {
                 AsyncLogReader reader;
@@ -1536,7 +1536,7 @@
 
         @Override
         protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
             try {
                 return inspectAndRepair(dlm.getLogSegments());
             } finally {
@@ -2640,11 +2640,11 @@
         @Override
         protected int runCmd() throws Exception {
             getConf().setZkAclId(getZkAclId());
-            return truncateStream(getFactory(), getStreamName(), dlsn);
+            return truncateStream(getNamespace(), getStreamName(), dlsn);
         }
 
-        private int truncateStream(final com.twitter.distributedlog.DistributedLogManagerFactory factory, String streamName, DLSN dlsn) throws Exception {
-            DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+        private int truncateStream(final DistributedLogNamespace namespace, String streamName, DLSN dlsn) throws Exception {
+            DistributedLogManager dlm = namespace.openLog(streamName);
             try {
                 long totalRecords = dlm.getLogRecordCount();
                 long recordsAfterTruncate = Await.result(dlm.getLogRecordCountAsync(dlsn));
@@ -2731,7 +2731,6 @@
         int numThreads = 1;
         String streamPrefix = null;
         String subscriberId = null;
-        AtomicInteger streamIndex = new AtomicInteger();
 
         DeleteSubscriberCommand() {
             super("delete_subscriber", "Delete the subscriber in subscription store. ");
@@ -2764,20 +2763,21 @@
         @Override
         protected int runCmd() throws Exception {
             getConf().setZkAclId(getZkAclId());
-            return deleteSubscriber(getFactory());
+            return deleteSubscriber(getNamespace());
         }
 
-        private int deleteSubscriber(final com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
-            Collection<String> streamCollection = factory.enumerateAllLogsInNamespace();
+        private int deleteSubscriber(final DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streamCollection = namespace.getLogs();
             final List<String> streams = new ArrayList<String>();
-            if (null != streamPrefix) {
-                for (String s : streamCollection) {
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
                     if (s.startsWith(streamPrefix)) {
                         streams.add(s);
                     }
+                } else {
+                    streams.add(s);
                 }
-            } else {
-                streams.addAll(streamCollection);
             }
             if (0 == streams.size()) {
                 return 0;
@@ -2796,7 +2796,7 @@
                     @Override
                     public void run() {
                         try {
-                            deleteSubscriber(factory, streams, tid, numStreamsPerThreads);
+                            deleteSubscriber(namespace, streams, tid, numStreamsPerThreads);
                             System.out.println("Thread " + tid + " finished.");
                         } catch (Exception e) {
                             System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
@@ -2811,14 +2811,13 @@
             return 0;
         }
 
-        private void deleteSubscriber(com.twitter.distributedlog.DistributedLogManagerFactory factory, List<String> streams,
+        private void deleteSubscriber(DistributedLogNamespace namespace, List<String> streams,
                                       int tid, int numStreamsPerThreads) throws Exception {
             int startIdx = tid * numStreamsPerThreads;
             int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
             for (int i = startIdx; i < endIdx; i++) {
                 final String s = streams.get(i);
-                DistributedLogManager dlm =
-                    factory.createDistributedLogManagerWithSharedClients(s);
+                DistributedLogManager dlm = namespace.openLog(s);
                 final CountDownLatch countDownLatch = new CountDownLatch(1);
                 dlm.getSubscriptionsStore().deleteSubscriber(subscriberId)
                     .addEventListener(new FutureEventListener<Boolean>() {
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
index 63db1fe..2f9e091 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
@@ -17,34 +17,27 @@
  */
 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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+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 {
 
-    static final Logger logger = LoggerFactory.getLogger(DLUtils.class);
-
-    /**
-     * Extract zk servers fro dl <i>uri</i>.
-     *
-     * @param uri
-     *          dl uri
-     * @return zk servers
-     */
-    public static String getZKServersFromDLUri(URI uri) {
-        return uri.getAuthority().replace(";", ",");
-    }
-
     /**
      * Find the log segment whose transaction ids are not less than provided <code>transactionId</code>.
      *
@@ -224,4 +217,105 @@
     public static long bytes2LogSegmentId(byte[] data) {
         return Long.parseLong(new String(data, UTF_8));
     }
+
+    /**
+     * Normalize the uri.
+     *
+     * @param uri the distributedlog uri.
+     * @return the normalized uri
+     */
+    public static URI normalizeURI(URI uri) {
+        checkNotNull(uri, "DistributedLog uri is null");
+        String scheme = uri.getScheme();
+        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+        scheme = scheme.toLowerCase();
+        String[] schemeParts = StringUtils.split(scheme, '-');
+        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+                "Unknown distributedlog scheme found : " + uri);
+        URI normalizedUri;
+        try {
+            normalizedUri = new URI(
+                    schemeParts[0],     // remove backend info
+                    uri.getAuthority(),
+                    uri.getPath(),
+                    uri.getQuery(),
+                    uri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Invalid distributedlog uri found : " + uri, e);
+        }
+        return normalizedUri;
+    }
+
+    private static String getHostIpLockClientId() {
+        try {
+            return InetAddress.getLocalHost().toString();
+        } catch(Exception ex) {
+            return DistributedLogConstants.UNKNOWN_CLIENT_ID;
+        }
+    }
+
+    /**
+     * Normalize the client id.
+     *
+     * @return the normalized client id.
+     */
+    public static String normalizeClientId(String clientId) {
+        String normalizedClientId;
+        if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
+            normalizedClientId = getHostIpLockClientId();
+        } else {
+            normalizedClientId = clientId;
+        }
+        return normalizedClientId;
+    }
+
+    /**
+     * Is it a reserved stream name in bkdl namespace?
+     *
+     * @param name
+     *          stream name
+     * @return true if it is reserved name, otherwise false.
+     */
+    public static boolean isReservedStreamName(String name) {
+        return name.startsWith(".");
+    }
+
+    /**
+     * Validate the stream name.
+     *
+     * @param nameOfStream
+     *          name of stream
+     * @throws InvalidStreamNameException
+     */
+    public static void validateName(String nameOfStream)
+            throws InvalidStreamNameException {
+        String reason = null;
+        char chars[] = nameOfStream.toCharArray();
+        char c;
+        // validate the stream to see if meet zookeeper path's requirement
+        for (int i = 0; i < chars.length; i++) {
+            c = chars[i];
+
+            if (c == 0) {
+                reason = "null character not allowed @" + i;
+                break;
+            } else if (c == '/') {
+                reason = "'/' not allowed @" + i;
+                break;
+            } else if (c > '\u0000' && c < '\u001f'
+                    || c > '\u007f' && c < '\u009F'
+                    || c > '\ud800' && c < '\uf8ff'
+                    || c > '\ufff0' && c < '\uffff') {
+                reason = "invalid charater @" + i;
+                break;
+            }
+        }
+        if (null != reason) {
+            throw new InvalidStreamNameException(nameOfStream, reason);
+        }
+        if (isReservedStreamName(nameOfStream)) {
+            throw new InvalidStreamNameException(nameOfStream,
+                    "Stream Name is reserved");
+        }
+    }
 }
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
index 266409e..f206a25 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
@@ -271,7 +271,7 @@
      *
      * @param throwable the cause of the exception
      * @return the bk exception return code. if the exception isn't bk exceptions,
-     *         it would return bk exception code.
+     *         it would return {@link BKException.Code#UnexpectedConditionException}.
      */
     public static int bkResultCode(Throwable throwable) {
         if (throwable instanceof BKException) {
@@ -455,13 +455,13 @@
      * @param key submit key of the ordered scheduler
      */
     public static <T> void setException(final Promise<T> promise,
-                                        final Throwable throwable,
+                                        final Throwable cause,
                                         OrderedScheduler scheduler,
                                         Object key) {
         scheduler.submit(key, new Runnable() {
             @Override
             public void run() {
-                setException(promise, throwable);
+                setException(promise, cause);
             }
         });
     }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
index b0a38cf..c403e26 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
@@ -17,12 +17,12 @@
  */
 package com.twitter.distributedlog;
 
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
 import com.twitter.distributedlog.util.ConfUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.PermitLimiter;
@@ -35,11 +35,7 @@
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.versioning.Version;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -89,11 +85,6 @@
         return segments;
     }
 
-    static void updateBKDLConfig(URI uri, String zkServers, String ledgersPath, boolean sanityCheckTxnID) throws Exception {
-        BKDLConfig bkdlConfig = new BKDLConfig(zkServers, ledgersPath).setSanityCheckTxnID(sanityCheckTxnID);
-        DLMetadata.create(bkdlConfig).update(uri);
-    }
-
     public static URI createDLMURI(int port, String path) throws Exception {
         return LocalDLMEmulator.createDLMURI("127.0.0.1:" + port, path);
     }
@@ -111,93 +102,18 @@
                                                       URI uri) throws Exception {
         // TODO: Metadata Accessor seems to be a legacy object which only used by kestrel
         //       (we might consider deprecating this)
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
-        return namespace.createMetadataAccessor(name);
-    }
-
-    public static class BKLogPartitionWriteHandlerAndClients {
-        private BKLogWriteHandler writeHandler;
-        private ZooKeeperClient zooKeeperClient;
-        private BookKeeperClient bookKeeperClient;
-
-        public BKLogPartitionWriteHandlerAndClients(BKLogWriteHandler writeHandler, ZooKeeperClient zooKeeperClient, BookKeeperClient bookKeeperClient) {
-            this.writeHandler = writeHandler;
-            this.zooKeeperClient = zooKeeperClient;
-            this.bookKeeperClient = bookKeeperClient;
-        }
-
-        public void close() {
-            bookKeeperClient.close();
-            zooKeeperClient.close();
-            Utils.closeQuietly(writeHandler);
-        }
-
-        public BKLogWriteHandler getWriteHandler() {
-            return writeHandler;
-        }
-    }
-
-    static BKLogPartitionWriteHandlerAndClients createNewBKDLM(DistributedLogConfiguration conf,
-                                                               String logName,
-                                                               int zkPort) throws Exception {
-        URI uri = createDLMURI(zkPort, "/" + logName);
-
-        ZooKeeperClientBuilder zkcBuilder = TestZooKeeperClientBuilder.newBuilder(conf)
-            .name(String.format("dlzk:%s:handler_dedicated", logName))
-            .uri(uri);
-
-        ZooKeeperClient zkClient = zkcBuilder.build();
-
-        try {
-            zkClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            // ignore
-        }
-
-        // resolve uri
-        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkClient, uri);
-        BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-        BookKeeperClientBuilder bkcBuilder = BookKeeperClientBuilder.newBuilder()
-            .dlConfig(conf)
-            .name(String.format("bk:%s:handler_dedicated", logName))
-            .zkServers(bkdlConfig.getBkZkServersForWriter())
-            .ledgersPath(bkdlConfig.getBkLedgersPath())
-            .statsLogger(NullStatsLogger.INSTANCE);
-
-        BKDistributedLogManager bkdlm = new BKDistributedLogManager(
-                logName,
-                conf,
-                uri,
-                zkcBuilder,
-                zkcBuilder,
-                zkClient,
-                zkClient,
-                bkcBuilder,
-                bkcBuilder,
-                new SettableFeatureProvider("", 0),
-                PermitLimiter.NULL_PERMIT_LIMITER,
-                NullStatsLogger.INSTANCE);
-
-        BKLogWriteHandler writeHandler = bkdlm.createWriteHandler(true);
-        return new BKLogPartitionWriteHandlerAndClients(writeHandler, zkClient, bkcBuilder.build());
+        return namespace.getNamespaceDriver().getMetadataAccessor(name);
     }
 
     public static void fenceStream(DistributedLogConfiguration conf, URI uri, String name) throws Exception {
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(name, conf, uri);
+        DistributedLogManager dlm = createNewDLM(name, conf, uri);
         try {
-            BKLogReadHandler readHandler = dlm.createReadHandler();
-            List<LogSegmentMetadata> ledgerList = FutureUtils.result(
-                    readHandler.readLogSegmentsFromStore(
-                            LogSegmentMetadata.COMPARATOR,
-                            LogSegmentFilter.DEFAULT_FILTER,
-                            null)
-            ).getValue();
-            LogSegmentMetadata lastSegment = ledgerList.get(ledgerList.size() - 1);
-            BookKeeperClient bkc = dlm.getWriterBKC();
-            LedgerHandle lh = bkc.get().openLedger(lastSegment.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-            lh.close();
+            List<LogSegmentMetadata> logSegmentList = dlm.getLogSegments();
+            LogSegmentMetadata lastSegment = logSegmentList.get(logSegmentList.size() - 1);
+            LogSegmentEntryStore entryStore = dlm.getNamespaceDriver().getLogSegmentEntryStore(NamespaceDriver.Role.READER);
+            Utils.close(FutureUtils.result(entryStore.openRandomAccessReader(lastSegment, true)));
         } finally {
             dlm.close();
         }
@@ -409,6 +325,14 @@
         return txid - startTxid;
     }
 
+    public static ZooKeeperClient getZooKeeperClient(BKDistributedLogManager dlm) {
+        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getWriterZKC();
+    }
+
+    public static BookKeeperClient getBookKeeperClient(BKDistributedLogManager dlm) {
+        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getReaderBKC();
+    }
+
     public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager, DistributedLogConfiguration conf,
                                                                 long logSegmentSeqNo, long startTxID, boolean writeEntries, long segmentSize,
                                                                 boolean completeLogSegment)
@@ -417,7 +341,7 @@
         BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
         FutureUtils.result(writeHandler.lockHandler());
         // Start a log segment with a given ledger seq number.
-        BookKeeperClient bkc = dlm.getWriterBKC();
+        BookKeeperClient bkc = getBookKeeperClient(dlm);
         LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
                 conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
         String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
@@ -429,7 +353,7 @@
                 .setLogSegmentSequenceNo(logSegmentSeqNo)
                 .setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion))
                 .build();
-        l.write(dlm.writerZKC);
+        l.write(getZooKeeperClient(dlm));
         writeHandler.maxTxId.update(Version.ANY, startTxID);
         writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
         BKLogSegmentWriter writer = new BKLogSegmentWriter(
@@ -468,7 +392,7 @@
         BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
         FutureUtils.result(writeHandler.lockHandler());
         // Start a log segment with a given ledger seq number.
-        BookKeeperClient bkc = dlm.getReaderBKC();
+        BookKeeperClient bkc = getBookKeeperClient(dlm);
         LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
                 conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
         String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
@@ -479,7 +403,7 @@
             .setLogSegmentSequenceNo(logSegmentSeqNo)
             .setInprogress(false)
             .build();
-        l.write(dlm.writerZKC);
+        l.write(getZooKeeperClient(dlm));
         writeHandler.maxTxId.update(Version.ANY, startTxID);
         writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
         BKLogSegmentWriter writer = new BKLogSegmentWriter(
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
index a6cffbb..124ea77 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
@@ -29,9 +29,11 @@
 import com.twitter.distributedlog.exceptions.LockCancelledException;
 import com.twitter.distributedlog.exceptions.LockingException;
 import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.lock.LockClosedException;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
 import com.twitter.distributedlog.subscription.SubscriptionsStore;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.Utils;
@@ -85,7 +87,8 @@
         Utils.close(reader1);
 
         // simulate a old stream created without readlock path
-        writer.bkDistributedLogManager.getWriterZKC().get().delete(readLockPath, -1);
+        NamespaceDriver driver = dlm.getNamespaceDriver();
+        ((BKNamespaceDriver) driver).getWriterZKC().get().delete(readLockPath, -1);
         Future<AsyncLogReader> futureReader2 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
         AsyncLogReader reader2 = Await.result(futureReader2);
         record = Await.result(reader2.readNext());
@@ -230,7 +233,7 @@
 
         DistributedLogManager dlm1 = createNewDLM(conf, name);
         Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        Await.result(futureReader1);
+        AsyncLogReader reader1 = Await.result(futureReader1);
 
         BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(conf, name);
         Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
@@ -243,6 +246,7 @@
         } catch (LockCancelledException ex) {
         }
 
+        Utils.close(reader1);
         dlm0.close();
         dlm1.close();
     }
@@ -250,16 +254,26 @@
     @Test(timeout = 60000)
     public void testReaderLockSessionExpires() throws Exception {
         String name = runtime.getMethodName();
-        DistributedLogManager dlm0 = createNewDLM(conf, name);
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+        DistributedLogNamespace ns0 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+        DistributedLogManager dlm0 = ns0.openLog(name);
         BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
         writer.write(DLMTestUtil.getLogRecordInstance(2L));
         writer.closeAndComplete();
 
-        DistributedLogManager dlm1 = createNewDLM(conf, name);
+        DistributedLogNamespace ns1 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+        DistributedLogManager dlm1 = ns1.openLog(name);
         Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
         AsyncLogReader reader1 = Await.result(futureReader1);
-        ZooKeeperClientUtils.expireSession(((BKDistributedLogManager)dlm1).getWriterZKC(), zkServers, 1000);
+        ZooKeeperClientUtils.expireSession(((BKNamespaceDriver) ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
 
         // The result of expireSession is somewhat non-deterministic with this lock.
         // It may fail with LockingException or it may succesfully reacquire, so for
@@ -276,7 +290,9 @@
 
         Utils.close(reader1);
         dlm0.close();
+        ns0.close();
         dlm1.close();
+        ns1.close();
     }
 
     @Test(timeout = 60000)
@@ -511,8 +527,11 @@
         Utils.close(Await.result(futureReader3));
 
         dlm1.close();
+        namespace1.close();
         dlm2.close();
+        namespace2.close();
         dlm3.close();
+        namespace3.close();
 
         executorService.shutdown();
     }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
index 41adbb9..46c8523 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
@@ -36,6 +36,7 @@
 import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
 import com.twitter.distributedlog.exceptions.BKTransmitException;
 import com.twitter.distributedlog.exceptions.LockingException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.io.CompressionCodec;
 import com.twitter.distributedlog.util.Utils;
 import com.twitter.util.Promise;
@@ -1270,8 +1271,9 @@
 
         BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
 
+        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
         // fence the ledger
-        dlm.getWriterBKC().get().openLedger(logWriter.getLogSegmentId(),
+        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
                 BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
 
         try {
@@ -1313,8 +1315,9 @@
 
         BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
 
+        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
         // fence the ledger
-        dlm.getWriterBKC().get().openLedger(logWriter.getLogSegmentId(),
+        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
                 BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
 
         try {
@@ -1500,6 +1503,7 @@
         confLocal.setImmediateFlushEnabled(true);
         confLocal.setReadAheadBatchSize(1);
         confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReadLACLongPollTimeout(49);
         confLocal.setReaderIdleWarnThresholdMillis(100);
         confLocal.setReaderIdleErrorThresholdMillis(20000);
         final DistributedLogManager dlm = createNewDLM(confLocal, name);
@@ -1976,7 +1980,7 @@
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
         assertEquals(1, segments.size());
         long ledgerId = segments.get(0).getLogSegmentId();
-        LedgerHandle lh = ((BKDistributedLogNamespace) namespace).getReaderBKC()
+        LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
                 .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
         LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh);
         assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize());
@@ -1995,7 +1999,7 @@
         segments = dlm.getLogSegments();
         assertEquals(1, segments.size());
         ledgerId = segments.get(0).getLogSegmentId();
-        lh = ((BKDistributedLogNamespace) namespace).getReaderBKC()
+        lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
                 .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
         metadata = BookKeeperAccessor.getLedgerMetadata(lh);
         assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize());
@@ -2147,6 +2151,7 @@
         confLocal.setImmediateFlushEnabled(false);
         confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
         confLocal.setPeriodicKeepAliveMilliSeconds(0);
+        confLocal.setReadLACLongPollTimeout(9);
         confLocal.setReaderIdleWarnThresholdMillis(20);
         confLocal.setReaderIdleErrorThresholdMillis(40);
 
@@ -2178,6 +2183,7 @@
         confLocal.setImmediateFlushEnabled(false);
         confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
         confLocal.setPeriodicKeepAliveMilliSeconds(1000);
+        confLocal.setReadLACLongPollTimeout(999);
         confLocal.setReaderIdleWarnThresholdMillis(2000);
         confLocal.setReaderIdleErrorThresholdMillis(4000);
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
index 96c33e2..f7d587d 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
@@ -22,6 +22,7 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -32,6 +33,7 @@
 import com.twitter.distributedlog.exceptions.LogNotFoundException;
 import com.twitter.distributedlog.exceptions.LogReadException;
 import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
+import com.twitter.distributedlog.io.Abortables;
 import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.OrderedScheduler;
@@ -48,14 +50,12 @@
 import com.twitter.distributedlog.exceptions.EndOfStreamException;
 import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
 import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
 import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
 import com.twitter.distributedlog.metadata.LogMetadata;
 import com.twitter.distributedlog.metadata.MetadataUpdater;
 import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
 import com.twitter.distributedlog.subscription.SubscriptionsStore;
 import com.twitter.util.Await;
 import com.twitter.util.Duration;
@@ -67,6 +67,8 @@
 public class TestBKDistributedLogManager extends TestDistributedLogBase {
     static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class);
 
+    private static final Random RAND = new Random(System.currentTimeMillis());
+
     @Rule
     public TestName testNames = new TestName();
 
@@ -254,20 +256,6 @@
     }
 
     @Test(timeout = 60000)
-    public void testTwoWriters() throws Exception {
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlm1 =
-                createNewBKDLM(conf, "distrlog-dualWriter");
-        try {
-             createNewBKDLM(conf, "distrlog-dualWriter");
-            fail("Shouldn't have been able to open the second writer");
-        } catch (OwnershipAcquireFailedException ioe) {
-            assertEquals(ioe.getCurrentOwner(), DistributedLogConstants.UNKNOWN_CLIENT_ID);
-        }
-
-        bkdlm1.close();
-    }
-
-    @Test(timeout = 60000)
     public void testTwoWritersOnLockDisabled() throws Exception {
         DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
         confLocal.addConfiguration(conf);
@@ -468,11 +456,10 @@
         writer.setReadyToFlush();
         writer.flushAndSync();
         writer.close();
-        dlm.createOrUpdateMetadata(name.getBytes());
-        assertEquals(name, new String(dlm.getMetadata()));
+        dlm.close();
 
         URI uri = createDLMURI("/" + name);
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         assertTrue(namespace.logExists(name));
         assertFalse(namespace.logExists("non-existent-log"));
@@ -490,9 +477,7 @@
         }
         assertEquals(1, logCount);
 
-        for(Map.Entry<String, byte[]> logEntry: namespace.enumerateLogsWithMetadataInNamespace().entrySet()) {
-            assertEquals(name, new String(logEntry.getValue()));
-        }
+        namespace.close();
     }
 
     @Test(timeout = 60000)
@@ -507,28 +492,6 @@
     }
 
     @Test(timeout = 60000)
-    @Deprecated
-    public void testSubscriptionStateStore() throws Exception {
-        String name = "distrlog-subscription-state";
-        String subscriberId = "defaultSubscriber";
-        DLSN commitPosition0 = new DLSN(4, 33, 5);
-        DLSN commitPosition1 = new DLSN(4, 34, 5);
-        DLSN commitPosition2 = new DLSN(5, 34, 5);
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        SubscriptionStateStore store = dlm.getSubscriptionStateStore(subscriberId);
-        assertEquals(Await.result(store.getLastCommitPosition()), DLSN.NonInclusiveLowerBound);
-        Await.result(store.advanceCommitPosition(commitPosition1));
-        assertEquals(Await.result(store.getLastCommitPosition()), commitPosition1);
-        Await.result(store.advanceCommitPosition(commitPosition0));
-        assertEquals(Await.result(store.getLastCommitPosition()), commitPosition1);
-        Await.result(store.advanceCommitPosition(commitPosition2));
-        assertEquals(Await.result(store.getLastCommitPosition()), commitPosition2);
-        SubscriptionStateStore store1 = dlm.getSubscriptionStateStore(subscriberId);
-        assertEquals(Await.result(store1.getLastCommitPosition()), commitPosition2);
-    }
-
-    @Test(timeout = 60000)
     public void testSubscriptionsStore() throws Exception {
         String name = "distrlog-subscriptions-store";
         String subscriber0 = "subscriber-0";
@@ -732,24 +695,12 @@
         reader.close();
     }
 
-    @Test(timeout = 60000)
+    @Test(timeout = 60000, expected = LogRecordTooLongException.class)
     public void testMaxLogRecSize() throws Exception {
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients =
-                createNewBKDLM(conf, "distrlog-maxlogRecSize");
-        long txid = 1;
-        BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
-        boolean exceptionEncountered = false;
-        try {
-            LogRecord op = new LogRecord(txid, DLMTestUtil.repeatString(
-                                DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes());
-            out.write(op);
-        } catch (LogRecordTooLongException exc) {
-            exceptionEncountered = true;
-        } finally {
-            FutureUtils.result(out.asyncClose());
-        }
-        bkdlmAndClients.close();
-        assertTrue(exceptionEncountered);
+        DistributedLogManager dlm = createNewDLM(conf, "distrlog-maxlogRecSize");
+        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        FutureUtils.result(writer.write(new LogRecord(1L, DLMTestUtil.repeatString(
+                                DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes())));
     }
 
     @Test(timeout = 60000)
@@ -757,25 +708,27 @@
         DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
         confLocal.loadConf(conf);
         confLocal.setOutputBufferSize(1024 * 1024);
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients =
-                createNewBKDLM(confLocal, "distrlog-transmissionSize");
-        long txid = 1;
-        BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
+        BKDistributedLogManager dlm =
+                createNewDLM(confLocal, "distrlog-transmissionSize");
+        AsyncLogWriter out = FutureUtils.result(dlm.openAsyncLogWriter());
         boolean exceptionEncountered = false;
-        byte[] largePayload = DLMTestUtil.repeatString(DLMTestUtil.repeatString("abcdefgh", 256), 256).getBytes();
+        byte[] largePayload = new byte[(LogRecord.MAX_LOGRECORDSET_SIZE / 2) + 2];
+        RAND.nextBytes(largePayload);
         try {
-            while (txid < 3) {
-                LogRecord op = new LogRecord(txid, largePayload);
-                out.write(op);
-                txid++;
-            }
+            LogRecord op = new LogRecord(1L, largePayload);
+            Future<DLSN> firstWriteFuture = out.write(op);
+            op = new LogRecord(2L, largePayload);
+            // the second write will flush the first one, since we reached the maximum transmission size.
+            out.write(op);
+            FutureUtils.result(firstWriteFuture);
         } catch (LogRecordTooLongException exc) {
             exceptionEncountered = true;
         } finally {
             FutureUtils.result(out.asyncClose());
         }
-        bkdlmAndClients.close();
-        assertTrue(!exceptionEncountered);
+        assertFalse(exceptionEncountered);
+        Abortables.abortQuietly(out);
+        dlm.close();
     }
 
     @Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
index ecc20e0..a8a82fa 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
@@ -34,9 +34,10 @@
 import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
 import com.twitter.distributedlog.exceptions.LockingException;
 import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.BKDLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.util.DLUtils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -112,63 +113,13 @@
     }
 
     @Test(timeout = 60000)
-    @SuppressWarnings("deprecation")
-    public void testClientSharingOptions() throws Exception {
-        URI uri = createDLMURI("/clientSharingOptions");
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
-                .conf(conf).uri(uri).build();
-
-        {
-            BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("perstream1",
-                                        DistributedLogManagerFactory.ClientSharingOption.PerStreamClients);
-
-            BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("perstream2",
-                DistributedLogManagerFactory.ClientSharingOption.PerStreamClients);
-
-            assertThat(bkdlm1.getReaderBKC(), not(bkdlm2.getReaderBKC()));
-            assertThat(bkdlm1.getWriterBKC(), not(bkdlm2.getWriterBKC()));
-            assertThat(bkdlm1.getReaderZKC(), not(bkdlm2.getReaderZKC()));
-            assertThat(bkdlm1.getWriterZKC(), not(bkdlm2.getWriterZKC()));
-
-        }
-
-        {
-            BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedZK1",
-                DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-
-            BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedZK2",
-                DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-
-            assertThat(bkdlm1.getReaderBKC(), not(bkdlm2.getReaderBKC()));
-            assertThat(bkdlm1.getWriterBKC(), not(bkdlm2.getWriterBKC()));
-            assertEquals(bkdlm1.getReaderZKC(), bkdlm2.getReaderZKC());
-            assertEquals(bkdlm1.getWriterZKC(), bkdlm2.getWriterZKC());
-        }
-
-        {
-            BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedBoth1",
-                DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-
-            BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedBoth2",
-                DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-
-            assertEquals(bkdlm1.getReaderBKC(), bkdlm2.getReaderBKC());
-            assertEquals(bkdlm1.getWriterBKC(), bkdlm2.getWriterBKC());
-            assertEquals(bkdlm1.getReaderZKC(), bkdlm2.getReaderZKC());
-            assertEquals(bkdlm1.getWriterZKC(), bkdlm2.getWriterZKC());
-        }
-
-    }
-
-
-    @Test(timeout = 60000)
     public void testInvalidStreamName() throws Exception {
-        assertFalse(BKDLUtils.isReservedStreamName("test"));
-        assertTrue(BKDLUtils.isReservedStreamName(".test"));
+        assertFalse(DLUtils.isReservedStreamName("test"));
+        assertTrue(DLUtils.isReservedStreamName(".test"));
 
         URI uri = createDLMURI("/" + runtime.getMethodName());
 
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
 
         try {
@@ -238,11 +189,6 @@
         assertTrue(streamSet.contains("test1"));
         assertTrue(streamSet.contains("test_2-3"));
 
-        Map<String, byte[]> streamMetadatas = namespace.enumerateLogsWithMetadataInNamespace();
-        assertEquals(2, streamMetadatas.size());
-        assertTrue(streamMetadatas.containsKey("test1"));
-        assertTrue(streamMetadatas.containsKey("test_2-3"));
-
         namespace.close();
     }
 
@@ -385,7 +331,7 @@
 
     static void validateBadAllocatorConfiguration(DistributedLogConfiguration conf, URI uri) throws Exception {
         try {
-            BKDistributedLogNamespace.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
+            BKNamespaceDriver.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
             fail("Should throw exception when bad allocator configuration provided");
         } catch (IOException ioe) {
             // expected
@@ -425,7 +371,7 @@
     @Test(timeout = 60000)
     public void testUseNamespaceAfterCloseShouldFailFast() throws Exception {
         URI uri = createDLMURI("/" + runtime.getMethodName());
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
             .conf(conf)
             .uri(uri)
             .build();
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
index 4b17500..854cb74 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
@@ -50,27 +50,6 @@
     @Rule
     public TestName runtime = new TestName();
 
-    private void prepareLogSegments(String name, int numSegments, int numEntriesPerSegment) throws Exception {
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, name);
-        long txid = 1;
-        for (int sid = 0; sid < numSegments; ++sid) {
-            BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
-            for (int eid = 0; eid < numEntriesPerSegment; ++eid) {
-                LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-                out.write(record);
-                ++txid;
-            }
-            FutureUtils.result(out.asyncClose());
-            bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment(
-                    out.getLogSegmentSequenceNumber(),
-                    out.getLogSegmentId(),
-                    1 + sid * numEntriesPerSegment,
-                    (sid + 1) * numEntriesPerSegment,
-                    numEntriesPerSegment);
-        }
-        bkdlmAndClients.close();
-    }
-
     private void prepareLogSegmentsNonPartitioned(String name, int numSegments, int numEntriesPerSegment) throws Exception {
         DistributedLogManager dlm = createNewDLM(conf, name);
         long txid = 1;
@@ -134,8 +113,8 @@
     @Test(timeout = 60000)
     public void testGetFirstDLSNWithLogSegments() throws Exception {
         String dlName = runtime.getMethodName();
-        prepareLogSegments(dlName, 3, 3);
         BKDistributedLogManager dlm = createNewDLM(conf, dlName);
+        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 3);
         BKLogReadHandler readHandler = dlm.createReadHandler();
         Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
         try {
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
index b350255..8f86192 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
@@ -21,14 +21,14 @@
 import com.twitter.distributedlog.exceptions.EndOfStreamException;
 import com.twitter.distributedlog.exceptions.WriteCancelledException;
 import com.twitter.distributedlog.exceptions.WriteException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
 import com.twitter.distributedlog.io.Abortables;
 import com.twitter.distributedlog.lock.SessionLockFactory;
 import com.twitter.distributedlog.lock.ZKDistributedLock;
 import com.twitter.distributedlog.lock.ZKSessionLockFactory;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.DLUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.PermitLimiter;
@@ -96,7 +96,7 @@
                 .dlConfig(conf)
                 .name("test-bkc")
                 .ledgersPath(bkdlConfig.getBkLedgersPath())
-                .zkServers(DLUtils.getZKServersFromDLUri(uri))
+                .zkServers(BKNamespaceDriver.getZKServersFromDLUri(uri))
                 .build();
     }
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
index 754f945..a0485bd 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
@@ -19,6 +19,7 @@
 
 import com.twitter.distributedlog.bk.LedgerAllocator;
 import com.twitter.distributedlog.bk.LedgerAllocatorPool;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
 import com.twitter.distributedlog.util.FailpointUtils;
 import com.twitter.distributedlog.util.FutureUtils;
@@ -75,7 +76,8 @@
                     FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
         }
 
-        LedgerAllocator allocator = namespace.getLedgerAllocator();
+        LedgerAllocator allocator = ((BKNamespaceDriver) namespace.getNamespaceDriver())
+                .getLedgerAllocator();
         assertTrue(allocator instanceof LedgerAllocatorPool);
         LedgerAllocatorPool allocatorPool = (LedgerAllocatorPool) allocator;
         assertEquals(0, allocatorPool.obtainMapSize());
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
index a388b68..d850db4 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
@@ -19,11 +19,24 @@
 
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
+import com.twitter.distributedlog.io.AsyncCloseable;
 import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
+import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
 import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
+import com.twitter.distributedlog.util.ConfUtils;
+import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.PermitLimiter;
+import com.twitter.distributedlog.util.SchedulerUtils;
+import com.twitter.util.Future;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
@@ -43,9 +56,11 @@
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 public class TestDistributedLogBase {
     static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class);
@@ -87,6 +102,12 @@
                 .build();
         bkutil.start();
         zkServers = "127.0.0.1:" + zkPort;
+        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+            @Override
+            public void uncaughtException(Thread t, Throwable e) {
+                LOG.warn("Uncaught exception at Thread {} : ", t.getName(), e);
+            }
+        });
     }
 
     @AfterClass
@@ -141,22 +162,7 @@
 
     public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
                                                 String name) throws Exception {
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        return new BKDistributedLogManager(
-                name,
-                conf,
-                uri,
-                null,
-                null,
-                null,
-                null,
-                null,
-                null,
-                new SettableFeatureProvider("", 0),
-                PermitLimiter.NULL_PERMIT_LIMITER,
-                NullStatsLogger.INSTANCE
-        );
+        return createNewDLM(conf, name, PermitLimiter.NULL_PERMIT_LIMITER);
     }
 
     public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
@@ -165,48 +171,69 @@
             throws Exception {
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
+        final DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .uri(uri)
+                .conf(conf)
+                .build();
+        final OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+                .corePoolSize(1)
+                .name("test-scheduler")
+                .build();
+        AsyncCloseable resourcesCloseable = new AsyncCloseable() {
+            @Override
+            public Future<Void> asyncClose() {
+                LOG.info("Shutting down the scheduler");
+                SchedulerUtils.shutdownScheduler(scheduler, 1, TimeUnit.SECONDS);
+                LOG.info("Shut down the scheduler");
+                LOG.info("Closing the namespace");
+                namespace.close();
+                LOG.info("Closed the namespace");
+                return Future.Void();
+            }
+        };
+        AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
+                .injectDelays(conf.getEIInjectReadAheadDelay(),
+                        conf.getEIInjectReadAheadDelayPercent(),
+                        conf.getEIInjectMaxReadAheadDelayMs())
+                .injectErrors(false, 10)
+                .injectStops(conf.getEIInjectReadAheadStall(), 10)
+                .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
+                .build();
         return new BKDistributedLogManager(
                 name,
                 conf,
+                ConfUtils.getConstDynConf(conf),
                 uri,
-                null,
-                null,
-                null,
-                null,
-                null,
-                null,
-                new SettableFeatureProvider("", 0),
+                namespace.getNamespaceDriver(),
+                new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
+                scheduler,
+                DistributedLogConstants.UNKNOWN_CLIENT_ID,
+                DistributedLogConstants.LOCAL_REGION_ID,
                 writeLimiter,
-                NullStatsLogger.INSTANCE
-        );
+                new SettableFeatureProvider("", 0),
+                failureInjector,
+                NullStatsLogger.INSTANCE,
+                NullStatsLogger.INSTANCE,
+                Optional.of(resourcesCloseable));
     }
 
-    public DLMTestUtil.BKLogPartitionWriteHandlerAndClients createNewBKDLM(
-            DistributedLogConfiguration conf,
-            String path) throws Exception {
-        return DLMTestUtil.createNewBKDLM(conf, path, zkPort);
-    }
-
-    @SuppressWarnings("deprecation")
-    protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogManagerFactory factory) {
-        DistributedLogNamespace namespace = factory.getNamespace();
-        assertTrue(namespace instanceof BKDistributedLogNamespace);
-        return ((BKDistributedLogNamespace) namespace).getWriterStreamMetadataStore()
+    protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogNamespace namespace)
+            throws IOException {
+        return namespace.getNamespaceDriver().getLogStreamMetadataStore(NamespaceDriver.Role.READER)
                 .getLogSegmentMetadataStore();
     }
 
-    @SuppressWarnings("deprecation")
-    protected ZooKeeperClient getZooKeeperClient(DistributedLogManagerFactory factory) throws Exception {
-        DistributedLogNamespace namespace = factory.getNamespace();
-        assertTrue(namespace instanceof BKDistributedLogNamespace);
-        return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+    protected ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) throws Exception {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assertTrue(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getWriterZKC();
     }
 
     @SuppressWarnings("deprecation")
-    protected BookKeeperClient getBookKeeperClient(DistributedLogManagerFactory factory) throws Exception {
-        DistributedLogNamespace namespace = factory.getNamespace();
-        assertTrue(namespace instanceof BKDistributedLogNamespace);
-        return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+    protected BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) throws Exception {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assertTrue(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getReaderBKC();
     }
 
     protected LedgerHandle getLedgerHandle(BKLogSegmentWriter segmentWriter) {
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.java
deleted file mode 100644
index e86e45a..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.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;
-
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.util.FutureUtils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestFailureAndRecovery extends TestDistributedLogBase {
-    static final Log LOG = LogFactory.getLog(TestFailureAndRecovery.class);
-
-    @Test(timeout = 60000)
-    public void testSimpleRecovery() throws Exception {
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-simplerecovery");
-        BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
-        long txid = 1;
-        for (long i = 1; i <= 100; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-            if ((i % 10) == 0) {
-                FutureUtils.result(out.flushAndCommit());
-            }
-
-        }
-        FutureUtils.result(out.flushAndCommit());
-
-        Abortables.abort(out, false);
-        FutureUtils.result(out.asyncClose());
-
-        assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
-        assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(out.getLogSegmentId(), 1, out.getLogSegmentSequenceNumber()), false));
-
-        FutureUtils.result(bkdlmAndClients.getWriteHandler().recoverIncompleteLogSegments());
-
-        assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
-        assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(out.getLogSegmentId(), 1, out.getLogSegmentSequenceNumber()), false));
-    }
-
-    /**
-     * Test that if enough bookies fail to prevent an ensemble,
-     * writes the bookkeeper will fail. Test that when once again
-     * an ensemble is available, it can continue to write.
-     */
-    @Test(timeout = 60000)
-    public void testAllBookieFailure() throws Exception {
-        BookieServer bookieToFail = bkutil.newBookie();
-        BookieServer replacementBookie = null;
-
-        try {
-            int ensembleSize = numBookies + 1;
-            assertEquals("Begin: New bookie didn't start",
-                ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-            // ensure that the journal manager has to use all bookies,
-            // so that a failure will fail the journal manager
-            DistributedLogConfiguration conf = new DistributedLogConfiguration();
-            conf.setEnsembleSize(ensembleSize);
-            conf.setWriteQuorumSize(ensembleSize);
-            conf.setAckQuorumSize(ensembleSize);
-            long txid = 1;
-            DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-allbookiefailure");
-            BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
-
-            for (long i = 1; i <= 3; i++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            FutureUtils.result(out.flushAndCommit());
-            bookieToFail.shutdown();
-            assertEquals("New bookie didn't die",
-                numBookies, bkutil.checkBookiesUp(numBookies, 10));
-
-            try {
-                for (long i = 1; i <= 3; i++) {
-                    LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                    out.write(op);
-                    txid++;
-                }
-                FutureUtils.result(out.flushAndCommit());
-                fail("should not get to this stage");
-            } catch (BKTransmitException bkte) {
-                LOG.debug("Error writing to bookkeeper", bkte);
-                assertEquals("Invalid exception message",
-                        BKException.Code.NotEnoughBookiesException, bkte.getBKResultCode());
-            }
-            replacementBookie = bkutil.newBookie();
-
-            assertEquals("Replacement: New bookie didn't start",
-                numBookies + 1, bkutil.checkBookiesUp(numBookies + 1, 10));
-            out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
-            for (long i = 1; i <= 3; i++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-
-            FutureUtils.result(out.flushAndCommit());
-        } catch (Exception e) {
-            LOG.error("Exception in test", e);
-            throw e;
-        } finally {
-            if (replacementBookie != null) {
-                replacementBookie.shutdown();
-            }
-            bookieToFail.shutdown();
-
-            if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
-                LOG.warn("Not all bookies from this test shut down, expect errors");
-            }
-        }
-    }
-
-    /**
-     * Test that a BookKeeper JM can continue to work across the
-     * failure of a bookie. This should be handled transparently
-     * by bookkeeper.
-     */
-    @Test(timeout = 60000)
-    public void testOneBookieFailure() throws Exception {
-        BookieServer bookieToFail = bkutil.newBookie();
-        BookieServer replacementBookie = null;
-
-        try {
-            int ensembleSize = numBookies + 1;
-            assertEquals("New bookie didn't start",
-                ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-            // ensure that the journal manager has to use all bookies,
-            // so that a failure will fail the journal manager
-            DistributedLogConfiguration conf = new DistributedLogConfiguration();
-            conf.setEnsembleSize(ensembleSize);
-            conf.setWriteQuorumSize(ensembleSize);
-            conf.setAckQuorumSize(ensembleSize);
-            long txid = 1;
-            DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-onebookiefailure");
-            BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
-            for (long i = 1; i <= 3; i++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            FutureUtils.result(out.flushAndCommit());
-
-            replacementBookie = bkutil.newBookie();
-            assertEquals("replacement bookie didn't start",
-                ensembleSize + 1, bkutil.checkBookiesUp(ensembleSize + 1, 10));
-            bookieToFail.shutdown();
-            assertEquals("New bookie didn't die",
-                ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-            for (long i = 1; i <= 3; i++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            FutureUtils.result(out.flushAndCommit());
-        } catch (Exception e) {
-            LOG.error("Exception in test", e);
-            throw e;
-        } finally {
-            if (replacementBookie != null) {
-                replacementBookie.shutdown();
-            }
-            bookieToFail.shutdown();
-
-            if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
-                LOG.warn("Not all bookies from this test shut down, expect errors");
-            }
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRecoveryEmptyLedger() throws Exception {
-        DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-recovery-empty-ledger");
-        BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
-        long txid = 1;
-        for (long i = 1; i <= 100; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-            if ((i % 10) == 0) {
-                FutureUtils.result(out.flushAndCommit());
-            }
-
-        }
-        FutureUtils.result(out.flushAndCommit());
-        FutureUtils.result(out.asyncClose());
-        bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment(out.getLogSegmentSequenceNumber(), out.getLogSegmentId(), 1, 100, 100);
-        assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
-        BKLogSegmentWriter outEmpty = bkdlmAndClients.getWriteHandler().startLogSegment(101);
-        Abortables.abort(outEmpty, false);
-
-        assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false));
-        assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), 101, outEmpty.getLogSegmentSequenceNumber()), false));
-
-        FutureUtils.result(bkdlmAndClients.getWriteHandler().recoverIncompleteLogSegments());
-
-        assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), outEmpty.getLogSegmentSequenceNumber(), 101), false));
-        assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false));
-    }
-
-    @Test(timeout = 60000)
-    public void testRecoveryAPI() throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, "distrlog-recovery-api");
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        long txid = 1;
-        for (long i = 1; i <= 100; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-            if ((i % 10) == 0) {
-                out.setReadyToFlush();
-                out.flushAndSync();
-            }
-
-        }
-        BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
-        out.setReadyToFlush();
-        out.flushAndSync();
-
-        out.abort();
-
-        BKLogWriteHandler blplm1 = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-
-        assertNull(zkc.exists(blplm1.completedLedgerZNode(1, 100,
-                                                          perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-        assertNotNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1,
-                                                        perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-
-        dlm.recover();
-
-        assertNotNull(zkc.exists(blplm1.completedLedgerZNode(1, 100,
-                                                             perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-        assertNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1,
-                                                     perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-        FutureUtils.result(blplm1.asyncClose());
-        assertEquals(100, dlm.getLogRecordCount());
-        dlm.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
index 3cdd676..830e059 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
@@ -22,8 +22,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.URI;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -337,72 +335,4 @@
         dlmreader1.close();
     }
 
-    @Test(timeout = 60000)
-    public void testFactorySharedClients() throws Exception {
-        String name = "distrlog-factorysharedclients";
-        testFactory(name, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testFactorySharedZK() throws Exception {
-        String name = "distrlog-factorysharedZK";
-        testFactory(name, false);
-    }
-
-    @SuppressWarnings("deprecation")
-    private void testFactory(String name, boolean shareBK) throws Exception {
-        int count = 3;
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
-                .conf(conf).uri(uri).build();
-        DistributedLogManager[] dlms = new DistributedLogManager[count];
-        for (int s = 0; s < count; s++) {
-            if (shareBK) {
-                dlms[s] = namespace.createDistributedLogManager(name + String.format("%d", s),
-                        DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-            } else {
-                dlms[s] = namespace.createDistributedLogManager(name + String.format("%d", s),
-                        DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-            }
-        }
-
-        int txid = 1;
-        for (long i = 0; i < 3; i++) {
-            BKSyncLogWriter[] writers = new BKSyncLogWriter[count];
-            for (int s = 0; s < count; s++) {
-                writers[s] = (BKSyncLogWriter)(dlms[s].startLogSegmentNonPartitioned());
-            }
-
-            for (long j = 0; j < 1; j++) {
-                final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                for (int s = 0; s < count; s++) {
-                    writers[s].write(record);
-                }
-            }
-            for (int s = 0; s < count; s++) {
-                writers[s].closeAndComplete();
-            }
-
-            if (i < 2) {
-                // Restart the zeroth stream and make sure that the other streams can
-                // continue without restart
-                dlms[0].close();
-                if (shareBK) {
-                    dlms[0] = namespace.createDistributedLogManager(name + String.format("%d", 0),
-                            DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-                } else {
-                    dlms[0] = namespace.createDistributedLogManager(name + String.format("%d", 0),
-                            DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-                }
-            }
-
-        }
-
-        for (int s = 0; s < count; s++) {
-            dlms[s].close();
-        }
-
-        namespace.close();
-    }
 }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
index bb67214..06c7bba 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
@@ -80,10 +80,10 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
         DistributedLogManager dlm = namespace.openLog(streamName);
         final int numSegments = 3;
@@ -92,7 +92,7 @@
             out.write(DLMTestUtil.getLogRecordInstance(i));
             out.closeAndComplete();
         }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(3, max2.getSequenceNumber());
         dlm.close();
         namespace.close();
@@ -111,10 +111,10 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
         DistributedLogManager dlm = namespace.openLog(streamName);
         final int numSegments = 3;
@@ -123,11 +123,11 @@
             out.write(DLMTestUtil.getLogRecordInstance(i));
             out.closeAndComplete();
         }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(3, max2.getSequenceNumber());
 
         // nuke the max ledger sequence number
-        updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf, new byte[0]);
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, new byte[0]);
         DistributedLogManager dlm1 = namespace.openLog(streamName);
         try {
             dlm1.startLogSegmentNonPartitioned();
@@ -139,7 +139,7 @@
         }
 
         // invalid max ledger sequence number
-        updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf, "invalid-max".getBytes(UTF_8));
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, "invalid-max".getBytes(UTF_8));
         DistributedLogManager dlm2 = namespace.openLog(streamName);
         try {
             dlm2.startLogSegmentNonPartitioned();
@@ -167,10 +167,10 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
         DistributedLogManager dlm = namespace.openLog(streamName);
         final int numSegments = 3;
@@ -179,11 +179,11 @@
             out.write(DLMTestUtil.getLogRecordInstance(i));
             out.closeAndComplete();
         }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
         assertEquals(3, max2.getSequenceNumber());
 
         // update the max ledger sequence number
-        updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf,
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf,
                 DLUtils.serializeLogSegmentSequenceNumber(99));
 
         DistributedLogManager dlm1 = namespace.openLog(streamName);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
index e322234..9553637 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
@@ -54,6 +54,7 @@
         confLocal.setReadAheadBatchSize(1);
         confLocal.setReadAheadMaxRecords(1);
         confLocal.setReaderIdleWarnThresholdMillis(100);
+        confLocal.setReadLACLongPollTimeout(49);
         final DistributedLogManager dlm = createNewDLM(confLocal, name);
         ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
         ScheduledFuture writerClosedFuture = null;
@@ -129,6 +130,7 @@
         confLocal.loadConf(conf);
         confLocal.setReadAheadBatchSize(1);
         confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReadLACLongPollTimeout(24);
         confLocal.setReaderIdleWarnThresholdMillis(50);
         confLocal.setReaderIdleErrorThresholdMillis(100);
         final DistributedLogManager dlm = createNewDLM(confLocal, name);
@@ -174,6 +176,7 @@
         confLocal.loadConf(conf);
         confLocal.setReadAheadBatchSize(1);
         confLocal.setReadAheadMaxRecords(3);
+        confLocal.setReadLACLongPollTimeout(249);
         confLocal.setReaderIdleWarnThresholdMillis(500);
         confLocal.setReaderIdleErrorThresholdMillis(30000);
         final DistributedLogManager dlm = createNewDLM(confLocal, name);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
index 74a5231..cf4fc4f 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
@@ -25,6 +25,7 @@
 import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
 import com.twitter.distributedlog.injector.AsyncFailureInjector;
 import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.util.ConfUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.Utils;
@@ -38,6 +39,7 @@
 import org.junit.rules.TestName;
 
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.*;
 
@@ -54,6 +56,7 @@
     private DistributedLogConfiguration baseConf;
     private OrderedScheduler scheduler;
     private BookKeeperClient bkc;
+    private ZooKeeperClient zkc;
 
     @Before
     public void setup() throws Exception {
@@ -66,6 +69,12 @@
         baseConf.setReadAheadMaxRecords(MAX_CACHED_ENTRIES);
         baseConf.setNumPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
         baseConf.setMaxPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("test-zk")
+                .zkServers(bkutil.getZkServers())
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(conf.getZkAclId())
+                .build();
         bkc = BookKeeperClientBuilder.newBuilder()
                 .name("test-bk")
                 .dlConfig(conf)
@@ -86,6 +95,9 @@
         if (null != scheduler) {
             scheduler.shutdown();
         }
+        if (null != zkc) {
+            zkc.close();
+        }
         super.teardown();
     }
 
@@ -99,8 +111,11 @@
                 true);
         LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
                 conf,
+                ConfUtils.getConstDynConf(conf),
+                zkc,
                 bkc,
                 scheduler,
+                null,
                 NullStatsLogger.INSTANCE,
                 AsyncFailureInjector.NULL);
         return new ReadAheadEntryReader(
@@ -309,7 +324,7 @@
         BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
 
         // generate list of log segments
-        generateCompletedLogSegments(dlm, 3, 2);
+        generateCompletedLogSegments(dlm, 3, 3);
         AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
         FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
 
@@ -321,23 +336,39 @@
         readAheadEntryReader.start(segments);
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
-        expectAlreadyTruncatedTransactionException(readAheadEntryReader,
-                "should fail on positioning to a truncated log segment");
+        expectNoException(readAheadEntryReader);
+        Entry.Reader entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(1L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
 
         // positioning on a partially truncated log segment (segment 2) before min active dlsn
         readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, baseConf);
         readAheadEntryReader.start(segments);
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
-        expectAlreadyTruncatedTransactionException(readAheadEntryReader,
-                "should fail on positioning to a partially truncated log segment");
+        expectNoException(readAheadEntryReader);
+        entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(1L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
 
         // positioning on a partially truncated log segment (segment 2) after min active dlsn
-        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, baseConf);
+        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 2L, 0L), dlm, baseConf);
         readAheadEntryReader.start(segments);
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
         expectNoException(readAheadEntryReader);
+        entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(2L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
+
+        Utils.close(writer);
+        dlm.close();
     }
 
     @Test(timeout = 60000)
@@ -363,6 +394,7 @@
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
         expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
 
         // positioning on a partially truncated log segment (segment 2) before min active dlsn
         readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, confLocal);
@@ -370,6 +402,7 @@
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
         expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
 
         // positioning on a partially truncated log segment (segment 2) after min active dlsn
         readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, confLocal);
@@ -377,6 +410,10 @@
         // ensure initialization to complete
         ensureOrderSchedulerEmpty(streamName);
         expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
+
+        Utils.close(writer);
+        dlm.close();
     }
 
     //
@@ -418,6 +455,9 @@
         ensureOrderSchedulerEmpty(streamName);
         expectIllegalStateException(readAheadEntryReader,
                 "inconsistent log segment found");
+
+        Utils.close(readAheadEntryReader);
+        dlm.close();
     }
 
 }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
index 99ef041..b183b84 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
@@ -331,6 +331,7 @@
 
     @FlakyTest
     @Test(timeout = 60000)
+    @SuppressWarnings("deprecation")
     public void testCaughtUpReaderOnLogSegmentRolling() throws Exception {
         String name = "distrlog-caughtup-reader-on-logsegment-rolling";
 
@@ -344,6 +345,8 @@
         confLocal.setWriteQuorumSize(1);
         confLocal.setAckQuorumSize(1);
         confLocal.setReadLACLongPollTimeout(99999999);
+        confLocal.setReaderIdleWarnThresholdMillis(2 * 99999999 + 1);
+        confLocal.setBKClientReadTimeout(99999999 + 1);
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
         BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
@@ -368,7 +371,7 @@
         }
 
         BKLogSegmentWriter perStreamWriter = writer.segmentWriter;
-        BookKeeperClient bkc = readDLM.getReaderBKC();
+        BookKeeperClient bkc = DLMTestUtil.getBookKeeperClient(readDLM);
         LedgerHandle readLh = bkc.get().openLedgerNoRecovery(getLedgerHandle(perStreamWriter).getId(),
                 BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
index ee2968d..ff924f8 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
@@ -20,6 +20,7 @@
 import com.twitter.distributedlog.TestZooKeeperClientBuilder;
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
 import com.twitter.distributedlog.thrift.AccessControlEntry;
 import com.twitter.util.Await;
 import org.apache.zookeeper.CreateMode;
@@ -103,7 +104,7 @@
 
         ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
 
-        assertEquals(zkPath, readZKAC.zkPath);
+        assertEquals(zkPath, readZKAC.getZKPath());
         assertEquals(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY, readZKAC.getAccessControlEntry());
         assertTrue(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY == readZKAC.getAccessControlEntry());
     }
@@ -145,7 +146,7 @@
         } catch (KeeperException.BadVersionException bve) {
             // expected
         }
-        readZKAC2.accessControlEntry.setDenyTruncate(true);
+        readZKAC2.getAccessControlEntry().setDenyTruncate(true);
         Await.result(readZKAC2.update(zkc));
         ZKAccessControl readZKAC3 = Await.result(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(readZKAC2, readZKAC3);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
index 8ba82f5..5625306 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
@@ -22,6 +22,8 @@
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.ZooKeeperClientUtils;
 import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
+import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
 import com.twitter.distributedlog.thrift.AccessControlEntry;
 import com.twitter.util.Await;
 import org.junit.After;
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
index d1069c3..60bc420 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
@@ -28,6 +28,8 @@
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
 import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
 import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.SchedulerUtils;
 import org.apache.zookeeper.CreateMode;
@@ -103,8 +105,10 @@
         confLocal.setLogSegmentCacheEnabled(false);
         URI uri = createDLMURI("/check-and-repair-dl-namespace");
         zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        com.twitter.distributedlog.DistributedLogManagerFactory factory =
-                new com.twitter.distributedlog.DistributedLogManagerFactory(confLocal, uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(confLocal)
+                .uri(uri)
+                .build();
         OrderedScheduler scheduler = OrderedScheduler.newBuilder()
                 .name("dlck-tool")
                 .corePoolSize(1)
@@ -114,17 +118,20 @@
         String streamName = "check-and-repair-dl-namespace";
 
         // Create completed log segments
-        DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
         DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 1L, 1L, 10, false);
         DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 2L, 11L, 10, true);
         DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 3L, 21L, 10, false);
         DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 4L, 31L, 10, true);
 
         // dryrun
-        BookKeeperClient bkc = getBookKeeperClient(factory);
-        DistributedLogAdmin.checkAndRepairDLNamespace(uri, factory,
-                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(factory)),
-                scheduler, bkc, confLocal.getBKDigestPW(), false, false);
+        DistributedLogAdmin.checkAndRepairDLNamespace(
+                uri,
+                namespace,
+                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+                scheduler,
+                false,
+                false);
 
         Map<Long, LogSegmentMetadata> segments = getLogSegments(dlm);
         LOG.info("segments after drynrun {}", segments);
@@ -134,10 +141,13 @@
         verifyLogSegment(segments, new DLSN(4L, 16L, 0L), 4L, 9, 39L);
 
         // check and repair
-        bkc = getBookKeeperClient(factory);
-        DistributedLogAdmin.checkAndRepairDLNamespace(uri, factory,
-                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(factory)),
-                scheduler, bkc, confLocal.getBKDigestPW(), false, false);
+        DistributedLogAdmin.checkAndRepairDLNamespace(
+                uri,
+                namespace,
+                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+                scheduler,
+                false,
+                false);
 
         segments = getLogSegments(dlm);
         LOG.info("segments after repair {}", segments);
@@ -148,7 +158,7 @@
 
         dlm.close();
         SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
-        factory.close();
+        namespace.close();
     }
 
 }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
index 66d7228..1e39e49 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
@@ -24,6 +24,8 @@
 import com.twitter.distributedlog.TestZooKeeperClientBuilder;
 import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
 import com.twitter.distributedlog.exceptions.UnexpectedException;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
 import com.twitter.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
@@ -90,21 +92,25 @@
 
         URI uri = createDLMURI("/change-sequence-number");
         zooKeeperClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        com.twitter.distributedlog.DistributedLogManagerFactory factory =
-                new com.twitter.distributedlog.DistributedLogManagerFactory(confLocal, uri);
-        com.twitter.distributedlog.DistributedLogManagerFactory readFactory =
-                new com.twitter.distributedlog.DistributedLogManagerFactory(readConf, uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(confLocal)
+                .uri(uri)
+                .build();
+        DistributedLogNamespace readNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(readConf)
+                .uri(uri)
+                .build();
 
         String streamName = "change-sequence-number";
 
         // create completed log segments
-        DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
         DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 4, 10);
         DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 5, 41, false, 10, true);
         dlm.close();
 
         // create a reader
-        DistributedLogManager readDLM = readFactory.createDistributedLogManagerWithSharedClients(streamName);
+        DistributedLogManager readDLM = readNamespace.openLog(streamName);
         AsyncLogReader reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
 
         // read the records
@@ -121,7 +127,7 @@
 
         LOG.info("Injecting bad log segment '3'");
 
-        dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+        dlm = namespace.openLog(streamName);
         DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 3L, 5 * 10 + 1, true, 10, false);
 
         LOG.info("Injected bad log segment '3'");
@@ -140,8 +146,8 @@
         LOG.info("Dryrun fix inprogress segment that has lower sequence number");
 
         // Dryrun
-        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(factory,
-                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(factory)), streamName, false, false);
+        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
 
         try {
             reader = readDLM.getAsyncLogReader(lastDLSN);
@@ -154,8 +160,8 @@
         LOG.info("Actual run fix inprogress segment that has lower sequence number");
 
         // Actual run
-        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(factory,
-                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(factory)), streamName, false, false);
+        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
 
         // be able to read more after fix
         reader = readDLM.getAsyncLogReader(lastDLSN);
@@ -182,7 +188,7 @@
         readDLM.close();
 
         dlm.close();
-        factory.close();
-        readFactory.close();
+        namespace.close();
+        readNamespace.close();
     }
 }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
index 46e8af0..de7016a 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -531,7 +531,7 @@
                 children, firstSegmentList);
 
         ZooKeeperClientUtils.expireSession(zkc,
-                DLUtils.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
+                BKNamespaceDriver.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
 
         logger.info("Create another {} segments.", numSegments);
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
index da9f577..c9a2e5b 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -174,7 +174,7 @@
         createLogInNamespace(uri, "test2");
         latches[2].await();
         assertEquals(2, receivedLogs.get().size());
-        ZooKeeperClientUtils.expireSession(zkc, DLUtils.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
         latches[3].await();
         assertEquals(2, receivedLogs.get().size());
         createLogInNamespace(uri, "test3");
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
index 673d856..0ce9f46 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -30,6 +30,7 @@
 import com.twitter.distributedlog.callback.NamespaceListener;
 import com.twitter.distributedlog.exceptions.LogExistsException;
 import com.twitter.distributedlog.exceptions.UnexpectedException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.metadata.LogMetadataStore;
 import com.twitter.distributedlog.util.DLUtils;
 import com.twitter.distributedlog.util.FutureUtils;
@@ -422,7 +423,7 @@
         TestNamespaceListenerWithExpectedSize listener =
                 new TestNamespaceListenerWithExpectedSize(2 * maxLogsPerSubnamespace + 1);
         metadataStore.registerNamespaceListener(listener);
-        ZooKeeperClientUtils.expireSession(zkc, DLUtils.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
         String testLogName = "test-log-name";
         allLogs.add(testLogName);
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
index 4cf86fa..183a405 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -30,10 +30,13 @@
 import com.twitter.distributedlog.LogRecordWithDLSN;
 import com.twitter.distributedlog.LogSegmentMetadata;
 import com.twitter.distributedlog.TestDistributedLogBase;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.ZooKeeperClientBuilder;
 import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
 import com.twitter.distributedlog.exceptions.ReadCancelledException;
 import com.twitter.distributedlog.injector.AsyncFailureInjector;
 import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.util.ConfUtils;
 import com.twitter.distributedlog.util.FutureUtils;
 import com.twitter.distributedlog.util.OrderedScheduler;
 import com.twitter.distributedlog.util.Utils;
@@ -59,10 +62,17 @@
     public TestName runtime = new TestName();
     private OrderedScheduler scheduler;
     private BookKeeperClient bkc;
+    private ZooKeeperClient zkc;
 
     @Before
     public void setup() throws Exception {
         super.setup();
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("test-zk")
+                .zkServers(bkutil.getZkServers())
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(conf.getZkAclId())
+                .build();
         bkc = BookKeeperClientBuilder.newBuilder()
                 .name("test-bk")
                 .dlConfig(conf)
@@ -83,6 +93,9 @@
         if (null != scheduler) {
             scheduler.shutdown();
         }
+        if (null != zkc) {
+            zkc.close();
+        }
         super.teardown();
     }
 
@@ -91,7 +104,14 @@
                                               DistributedLogConfiguration conf)
             throws Exception {
         LogSegmentEntryStore store = new BKLogSegmentEntryStore(
-                conf, bkc, scheduler, NullStatsLogger.INSTANCE, AsyncFailureInjector.NULL);
+                conf,
+                ConfUtils.getConstDynConf(conf),
+                zkc,
+                bkc,
+                scheduler,
+                null,
+                NullStatsLogger.INSTANCE,
+                AsyncFailureInjector.NULL);
         return (BKLogSegmentEntryReader) FutureUtils.result(store.openReader(segment, startEntryId));
     }
 
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
index 41544d6..1b19b2e 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -17,16 +17,16 @@
  */
 package com.twitter.distributedlog.impl.metadata;
 
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
 import com.google.common.collect.Lists;
 import com.twitter.distributedlog.DLMTestUtil;
 import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.TestZooKeeperClientBuilder;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.metadata.DLMetadata;
 import com.twitter.distributedlog.metadata.LogMetadataForWriter;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.DistributedLogManager;
 import com.twitter.distributedlog.DistributedLogConstants;
 import com.twitter.distributedlog.exceptions.LogNotFoundException;
 import com.twitter.distributedlog.ZooKeeperClient;
@@ -317,9 +317,9 @@
             .uri(uri)
             .build();
 
-        DistributedLogManager dlm = namespace.openLog(logName);
-        dlm.createOrUpdateMetadata(logName.getBytes("UTF-8"));
-        dlm.close();
+        MetadataAccessor accessor = namespace.getNamespaceDriver().getMetadataAccessor(logName);
+        accessor.createOrUpdateMetadata(logName.getBytes("UTF-8"));
+        accessor.close();
 
         testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
     }
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
similarity index 97%
rename from distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java
rename to distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
index 79fb539..bbabbb2 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
@@ -15,11 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.twitter.distributedlog.metadata;
+package com.twitter.distributedlog.impl.metadata;
 
 import com.twitter.distributedlog.DistributedLogConfiguration;
 import com.twitter.distributedlog.DistributedLogConstants;
 import com.twitter.distributedlog.TestZooKeeperClientBuilder;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.ZkMetadataResolver;
+import com.twitter.distributedlog.metadata.DLMetadata;
 import com.twitter.distributedlog.util.Utils;
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.ZooKeeperClientBuilder;
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
index d4c2f31..e3cc239 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
@@ -19,6 +19,7 @@
 
 import com.twitter.distributedlog.LocalDLMEmulator;
 import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
index 3225ced..a2a0ca6 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
@@ -20,7 +20,7 @@
 import com.twitter.distributedlog.DistributedLogConfiguration;
 import com.twitter.distributedlog.LocalDLMEmulator;
 import com.twitter.distributedlog.client.routing.SingleHostRoutingService;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import com.twitter.distributedlog.metadata.DLMetadata;
 import com.twitter.distributedlog.service.placement.EqualLoadAppraiser;
 import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
index cfb5b8d..2e49d92 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
@@ -23,12 +23,12 @@
 import com.twitter.common.zookeeper.ServerSet;
 import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
 import com.twitter.distributedlog.client.serverset.DLZkServerSet;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.service.ClientUtils;
 import com.twitter.distributedlog.service.DLSocketAddress;
 import com.twitter.distributedlog.service.DistributedLogClient;
 import com.twitter.distributedlog.service.DistributedLogClientBuilder;
 import com.twitter.distributedlog.tools.Tool;
-import com.twitter.distributedlog.util.DLUtils;
 import com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.thrift.ClientId$;
 import com.twitter.util.Await;
@@ -269,8 +269,8 @@
                         ClientUtils.buildClient(builder2);
                 try {
                     SimpleBalancer balancer = new SimpleBalancer(
-                            DLUtils.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
-                            DLUtils.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
+                            BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
+                            BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
                     try {
                         return runBalancer(balancer);
                     } finally {
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
index 18b9d1f..4f01bdc 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
@@ -38,7 +38,7 @@
 import com.twitter.distributedlog.BKDistributedLogNamespace;
 import com.twitter.distributedlog.DistributedLogConfiguration;
 import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
 import com.twitter.distributedlog.util.Utils;
 
 /**
@@ -55,11 +55,12 @@
   private boolean watching = false;
 
   public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
-    zkClient = BKDistributedLogNamespace.createDLZKClientBuilder(
-        String.format("dlzk:%s:factory_writer_shared", uri),
+    String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+    zkClient = BKNamespaceDriver.createZKClientBuilder(
+        String.format("ZKPlacementStateManager-%s", zkServers),
         conf,
-        DLUtils.getZKServersFromDLUri(uri),
-        statsLogger.scope("dlzk_factory_writer_shared")).build();
+        zkServers,
+        statsLogger.scope("placement_state_manager")).build();
     serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
   }
 
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
index c45e42c..218ea06 100644
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
+++ b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
@@ -30,10 +30,10 @@
 import com.twitter.distributedlog.LogRecordWithDLSN;
 import com.twitter.distributedlog.ZooKeeperClient;
 import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.acl.ZKAccessControl;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
 import com.twitter.distributedlog.client.routing.LocalRoutingService;
 import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
 import com.twitter.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.distributedlog.service.stream.StreamManagerImpl;
 import com.twitter.distributedlog.thrift.AccessControlEntry;
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
index 20c81f3..87ddec0 100644
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
@@ -18,12 +18,14 @@
 package com.twitter.distributedlog.mapreduce;
 
 import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
-import com.twitter.distributedlog.DLSN;
 import com.twitter.distributedlog.DistributedLogConfiguration;
 import com.twitter.distributedlog.DistributedLogManager;
+import com.twitter.distributedlog.DLSN;
 import com.twitter.distributedlog.LogRecordWithDLSN;
 import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAccessor;
@@ -57,7 +59,7 @@
     protected Configuration conf;
     protected DistributedLogConfiguration dlConf;
     protected URI dlUri;
-    protected BKDistributedLogNamespace namespace;
+    protected DistributedLogNamespace namespace;
     protected String streamName;
     protected DistributedLogManager dlm;
 
@@ -69,7 +71,7 @@
         dlUri = URI.create(configuration.get(DL_URI, ""));
         streamName = configuration.get(DL_STREAM, "");
         try {
-            namespace = BKDistributedLogNamespace.newBuilder()
+            namespace = DistributedLogNamespaceBuilder.newBuilder()
                     .conf(dlConf)
                     .uri(dlUri)
                     .build();
@@ -89,7 +91,7 @@
             throws IOException, InterruptedException {
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
         List<InputSplit> inputSplits = Lists.newArrayListWithCapacity(segments.size());
-        BookKeeper bk = namespace.getReaderBKC().get();
+        BookKeeper bk = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get();
         LedgerManager lm = BookKeeperAccessor.getLedgerManager(bk);
         final AtomicInteger rcHolder = new AtomicInteger(0);
         final AtomicReference<LedgerMetadata> metadataHolder = new AtomicReference<LedgerMetadata>(null);
@@ -121,7 +123,7 @@
         return new LogSegmentReader(
                 streamName,
                 dlConf,
-                namespace.getReaderBKC().get(),
+                ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get(),
                 (LogSegmentSplit) inputSplit);
     }
 }