You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by eo...@apache.org on 2021/11/24 15:27:43 UTC

[bookkeeper] branch master updated: Construct and inject bookie components (#2901)

This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new abf1a96  Construct and inject bookie components (#2901)
abf1a96 is described below

commit abf1a96e38bb74f7f7a47e9b6f2b337bd07d622f
Author: Jack Vanlightly <va...@gmail.com>
AuthorDate: Wed Nov 24 16:27:34 2021 +0100

    Construct and inject bookie components (#2901)
    
    * Construct and inject bookie components
    
    Construct and inject metadata driver and ledger storage outside of
    the BookieImpl. Also extract the cookie validation into a separate
    injectable interface.
    
    The metadata driver is a factory that creates the correct metadata
    accessors based on the configuration of the bookie. This is used by
    multiple components, so should be constructed before these
    components and passed in as a parameter.
    
    Previously it was constructed by the bookie, and other components
    called into the bookie to get the instance, which broken encapsulation
    of the bookie and let to a bunch of ugly mocking.
    
    This change also removes a circular dependency in the construction of
    the RegistrationManager. Previously it was requiring a listener on
    construction, but the listener needed the bookie instance, so it all
    went through a supplier. Now the listener is not needed at
    construction, but rather it is registered then the bookie starts.
    
    This change also adds an injectable interface for cookie validation. The
    legacy implementation of this interface is then injected into the
    BookieImpl on creation. The legacy implementation is code moved out of
    the BookieImpl class itself.
    
    Injecting ledger storage is good for encapsulation, as it stops the
    ledgerstorage depending on bookie behaviour. To that end, this change
    highlights a couple of circular dependencies. Notably, between the
    ledger storage and each of the statemanager, checkpointer and
    checkpointState. I've removed each of these from
    LedgerStorage#initialize() so that is is clearer that these are
    circular, and so that they can be fixed one by one.
    
    One thing to note in this change is that the order of initialization
    has changed. LedgerStorage is initialized before the cookie
    check. This means that directory structure needs to be checked before
    the cookie check. Previously it was done as part of the cookie
    check. Ledger storage must be initialized before the cookie check, as
    the cookie check needs to use ledger storage to update limbo bits.
    
    BookieImpl is still responsible for shutting down the LedgerStorage. I
    would have changed this, but it would have meant a lot of changes, as
    there are good few ledger storage implementations.
    
    * Ensure RegistrationManagers are closed
    
    * Remove PortManager next port and args that set it
    
    * Fix checkstyle in tests
    
    * Add stacktrace to standalone test
    
    * Add --debug to standalone test
    
    * Add dynamic scaling to LocalBookKeeper
    
    Some tests in distributedlog require the ability
    to scale up and down bookies, using the
    LocalBookKeeper class but doing so using
    BookieServer directly. Now that components are
    constructed outside of BookieServer, this scaling
    had to be built into LocalBookKeeper itself so
    the components could be constructed and destroyed
    correctly.
    
    * Moved state store server creation to constructor
    
    * Remove stacktrace option from standalone test
    
    Co-authored-by: Ivan Kelly <ik...@splunk.com>
---
 .../common/allocator/ByteBufAllocatorBuilder.java  |   2 +-
 .../allocator/ByteBufAllocatorWithOomHandler.java  |  29 ++
 .../impl/ByteBufAllocatorBuilderImpl.java          |   3 +-
 .../allocator/impl/ByteBufAllocatorImpl.java       |  10 +-
 .../component/AutoCloseableLifecycleComponent.java | 106 +++++
 .../java/org/apache/bookkeeper/bookie/Bookie.java  |   4 -
 .../org/apache/bookkeeper/bookie/BookieImpl.java   | 476 +++------------------
 .../apache/bookkeeper/bookie/BookieResources.java  | 116 +++++
 .../bookkeeper/bookie/BookieStateManager.java      |  33 +-
 .../apache/bookkeeper/bookie/CheckpointSource.java |  10 +
 .../apache/bookkeeper/bookie/CookieValidation.java |  29 ++
 .../bookkeeper/bookie/EntryLoggerAllocator.java    |   2 +-
 .../bookkeeper/bookie/GarbageCollectorThread.java  |   7 +-
 .../bookie/InterleavedLedgerStorage.java           |  34 +-
 .../InterleavedStorageRegenerateIndexOp.java       |   6 +-
 .../bookkeeper/bookie/LedgerDirsManager.java       |   8 +-
 .../apache/bookkeeper/bookie/LedgerStorage.java    |   7 +-
 .../bookkeeper/bookie/LegacyCookieValidation.java  | 269 ++++++++++++
 .../apache/bookkeeper/bookie/ReadOnlyBookie.java   |  34 +-
 .../bookie/ScanAndCompareGarbageCollector.java     |   2 -
 .../bookkeeper/bookie/SortedLedgerStorage.java     |  34 +-
 .../bookie/storage/ldb/DbLedgerStorage.java        |  23 +-
 .../bookie/storage/ldb/LedgersIndexRebuildOp.java  |   2 +-
 .../ldb/SingleDirectoryDbLedgerStorage.java        |  20 +-
 .../apache/bookkeeper/client/BookKeeperAdmin.java  |  47 +-
 .../bookkeeper/discover/RegistrationManager.java   |   7 +
 .../bookkeeper/discover/ZKRegistrationManager.java |  17 +-
 .../bookkeeper/meta/MetadataBookieDriver.java      |   7 +-
 .../apache/bookkeeper/meta/MetadataDrivers.java    |   9 +-
 .../bookkeeper/meta/NullMetadataBookieDriver.java  | 398 +++++++++++++++++
 .../bookkeeper/meta/zk/ZKMetadataBookieDriver.java |  30 +-
 .../apache/bookkeeper/proto/BookieNettyServer.java |   4 +
 .../org/apache/bookkeeper/proto/BookieServer.java  |  64 +--
 .../java/org/apache/bookkeeper/server/Main.java    | 107 ++++-
 .../server/http/BKHttpServiceProvider.java         |  17 +-
 .../server/http/service/ExpandStorageService.java  |  16 +-
 .../server/http/service/GetLedgerMetaService.java  |  10 +-
 .../server/http/service/ListLedgerService.java     |  10 +-
 .../service/ListUnderReplicatedLedgerService.java  |  11 +-
 .../bookkeeper/server/service/BookieService.java   |  31 +-
 .../bookie/ConvertToInterleavedStorageCommand.java |  10 +-
 .../tools/cli/commands/bookie/LastMarkCommand.java |  29 +-
 .../cli/commands/bookie/ReadJournalCommand.java    |   2 +-
 .../tools/cli/commands/cookie/AdminCommand.java    |  11 +-
 .../apache/bookkeeper/util/LocalBookKeeper.java    | 431 ++++++++++---------
 .../org/apache/bookkeeper/util/PortManager.java    |  39 +-
 .../bookkeeper/bookie/AdvertisedAddressTest.java   |   5 +-
 .../bookie/BookieInitializationTest.java           | 416 ++++++++++--------
 .../bookkeeper/bookie/BookieJournalTest.java       |  10 +-
 .../apache/bookkeeper/bookie/BookieShellTest.java  |   2 +-
 .../bookie/BookieStorageThresholdTest.java         |   6 +-
 .../bookie/BookieWriteToJournalTest.java           |   2 +-
 .../apache/bookkeeper/bookie/CompactionTest.java   |  76 ++--
 .../org/apache/bookkeeper/bookie/CookieTest.java   | 224 +++++-----
 .../org/apache/bookkeeper/bookie/EntryLogTest.java |   3 +-
 .../bookie/GcOverreplicatedLedgerTest.java         |   2 +-
 .../bookie/InterleavedLedgerStorageTest.java       |   5 +-
 .../apache/bookkeeper/bookie/LedgerCacheTest.java  |  15 +-
 .../bookie/LedgerStorageCheckpointTest.java        |  27 +-
 .../bookie/SlowInterleavedLedgerStorage.java       |   5 +-
 .../bookie/SortedLedgerStorageCheckpointTest.java  |   5 +-
 .../bookkeeper/bookie/SortedLedgerStorageTest.java |   6 +-
 .../apache/bookkeeper/bookie/StateManagerTest.java |  79 ++--
 .../apache/bookkeeper/bookie/SyncThreadTest.java   |   9 +-
 .../apache/bookkeeper/bookie/TestBookieImpl.java   | 169 +++++++-
 .../bookkeeper/bookie/UpdateCookieCmdTest.java     |  39 +-
 .../org/apache/bookkeeper/bookie/UpgradeTest.java  |  69 ++-
 .../bookie/storage/ldb/ConversionRollbackTest.java |  10 +-
 .../bookie/storage/ldb/ConversionTest.java         |  12 +-
 .../storage/ldb/DbLedgerStorageWriteCacheTest.java |  15 +-
 .../storage/ldb/LedgersIndexRebuildTest.java       |  30 +-
 .../storage/ldb/LocationsIndexRebuildTest.java     |  12 +-
 .../bookkeeper/client/BookKeeperAdminTest.java     |  36 +-
 .../bookkeeper/client/BookieWriteLedgerTest.java   |   4 +-
 .../client/ParallelLedgerRecoveryTest.java         |   3 +-
 .../apache/bookkeeper/client/SlowBookieTest.java   |   2 +-
 .../client/TestReadLastConfirmedAndEntry.java      |   6 +-
 .../bookkeeper/client/UpdateLedgerCmdTest.java     |   2 +-
 .../bookkeeper/conf/TestBKConfiguration.java       |   5 +-
 .../discover/TestZkRegistrationManager.java        |   2 +-
 .../org/apache/bookkeeper/meta/GcLedgersTest.java  |  10 +-
 .../bookkeeper/meta/LedgerManagerTestCase.java     |  10 +-
 .../bookkeeper/meta/MetadataDriversTest.java       |   4 +-
 .../meta/zk/ZKMetadataBookieDriverTest.java        |  37 +-
 .../replication/AuditorPeriodicCheckTest.java      |   1 -
 .../AuditorPlacementPolicyCheckTest.java           |  81 ++--
 .../replication/AuditorReplicasCheckTest.java      |  25 +-
 .../replication/TestReplicationWorker.java         |   1 -
 .../apache/bookkeeper/server/TestBookieBoot.java   | 132 ++++++
 .../org/apache/bookkeeper/server/TestMain.java     |  55 ++-
 .../bookkeeper/server/http/TestHttpService.java    |   9 +-
 .../server/http/service/ListLedgerServiceTest.java |   9 +-
 .../bookkeeper/test/BookKeeperClusterTestCase.java | 110 +++--
 .../apache/bookkeeper/test/BookieClientTest.java   |   5 +-
 .../apache/bookkeeper/test/BookieZKExpireTest.java |  10 +-
 .../apache/bookkeeper/test/LedgerDeleteTest.java   |   4 +-
 .../apache/bookkeeper/test/ReadOnlyBookieTest.java |   2 +-
 .../TestBookieImpl.java => test/TmpDirs.java}      |  39 +-
 .../metadata/etcd/EtcdBookieRegister.java          |  11 +-
 .../metadata/etcd/EtcdMetadataBookieDriver.java    |   9 +-
 .../metadata/etcd/EtcdRegistrationManager.java     |  10 +-
 .../etcd/testing/EtcdBKClusterTestBase.java        |   9 +-
 .../resolver/BKRegistrationNameResolverTest.java   |  10 +-
 stream/distributedlog/core/build.gradle            |   1 +
 .../apache/distributedlog/LocalDLMEmulator.java    |  34 +-
 .../java/org/apache/distributedlog/TestTxnId.java  |  73 ++--
 .../bookkeeper/stream/cluster/StreamCluster.java   |  23 +-
 .../stream/server/service/BookieService.java       |  83 +++-
 .../server/service/RegistrationStateService.java   |   6 +-
 .../ConvertToInterleavedStorageCommandTest.java    |   8 +-
 .../cli/commands/cookie/AdminCommandTest.java      |   6 +-
 111 files changed, 3016 insertions(+), 1697 deletions(-)

diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java
index d749efd..0e939ba 100644
--- a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java
+++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java
@@ -39,7 +39,7 @@ public interface ByteBufAllocatorBuilder {
     /**
      * Finalize the configured {@link ByteBufAllocator}.
      */
-    ByteBufAllocator build();
+    ByteBufAllocatorWithOomHandler build();
 
     /**
      * Specify a custom allocator where the allocation requests should be
diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorWithOomHandler.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorWithOomHandler.java
new file mode 100644
index 0000000..f468360
--- /dev/null
+++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorWithOomHandler.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.common.allocator;
+
+import io.netty.buffer.ByteBufAllocator;
+
+import java.util.function.Consumer;
+
+/**
+ * A ByteBufAllocatr interface with a OOM handler.
+ */
+public interface ByteBufAllocatorWithOomHandler extends ByteBufAllocator {
+    void setOomHandler(Consumer<OutOfMemoryError> handler);
+}
diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java
index fc6bd9d..a52063a 100644
--- a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java
+++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java
@@ -22,6 +22,7 @@ import io.netty.buffer.ByteBufAllocator;
 import java.util.function.Consumer;
 
 import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy;
 import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
@@ -40,7 +41,7 @@ public class ByteBufAllocatorBuilderImpl implements ByteBufAllocatorBuilder {
     LeakDetectionPolicy leakDetectionPolicy = LeakDetectionPolicy.Disabled;
 
     @Override
-    public ByteBufAllocator build() {
+    public ByteBufAllocatorWithOomHandler build() {
         return new ByteBufAllocatorImpl(pooledAllocator, unpooledAllocator, poolingPolicy, poolingConcurrency,
                 outOfMemoryPolicy, outOfMemoryListener, leakDetectionPolicy);
     }
diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java
index d6c98a9..dd5a476 100644
--- a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java
+++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java
@@ -26,6 +26,7 @@ import io.netty.util.ResourceLeakDetector;
 import io.netty.util.ResourceLeakDetector.Level;
 import java.util.function.Consumer;
 
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy;
 import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
@@ -35,7 +36,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Implementation of {@link ByteBufAllocator}.
  */
-public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements ByteBufAllocator {
+public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements ByteBufAllocatorWithOomHandler {
 
     private static final Logger log = LoggerFactory.getLogger(ByteBufAllocatorImpl.class);
 
@@ -47,7 +48,7 @@ public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements By
     private final ByteBufAllocator unpooledAllocator;
     private final PoolingPolicy poolingPolicy;
     private final OutOfMemoryPolicy outOfMemoryPolicy;
-    private final Consumer<OutOfMemoryError> outOfMemoryListener;
+    private Consumer<OutOfMemoryError> outOfMemoryListener;
 
     ByteBufAllocatorImpl(ByteBufAllocator pooledAllocator, ByteBufAllocator unpooledAllocator,
             PoolingPolicy poolingPolicy, int poolingConcurrency, OutOfMemoryPolicy outOfMemoryPolicy,
@@ -191,4 +192,9 @@ public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements By
     public boolean isDirectBufferPooled() {
         return pooledAllocator != null && pooledAllocator.isDirectBufferPooled();
     }
+
+    @Override
+    public void setOomHandler(Consumer<OutOfMemoryError> handler) {
+        this.outOfMemoryListener = handler;
+    }
 }
diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AutoCloseableLifecycleComponent.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AutoCloseableLifecycleComponent.java
new file mode 100644
index 0000000..933b0aa
--- /dev/null
+++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AutoCloseableLifecycleComponent.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.common.component;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Allows for AutoClosable resources to be added to the component
+ * lifecycle without having to implement ServerLifecycleComponent directly.
+ */
+public class AutoCloseableLifecycleComponent implements LifecycleComponent {
+    private static final Logger LOG = LoggerFactory.getLogger(AutoCloseableLifecycleComponent.class);
+
+    protected final Lifecycle lifecycle = new Lifecycle();
+    private final Set<LifecycleListener> listeners = new CopyOnWriteArraySet<>();
+    protected volatile UncaughtExceptionHandler uncaughtExceptionHandler;
+    private final String componentName;
+    private final AutoCloseable closeable;
+
+    public AutoCloseableLifecycleComponent(String componentName, AutoCloseable closeable) {
+        this.componentName = componentName;
+        this.closeable = closeable;
+    }
+
+    @Override
+    public String getName() {
+        return this.componentName;
+    }
+
+    @Override
+    public void setExceptionHandler(UncaughtExceptionHandler handler) {
+        this.uncaughtExceptionHandler = handler;
+    }
+
+    @Override
+    public Lifecycle.State lifecycleState() {
+        return this.lifecycle.state();
+    }
+
+    @Override
+    public void addLifecycleListener(LifecycleListener listener) {
+        listeners.add(listener);
+    }
+
+    @Override
+    public void removeLifecycleListener(LifecycleListener listener) {
+        listeners.remove(listener);
+    }
+
+    @Override
+    public void start() {
+        if (!lifecycle.canMoveToStarted()) {
+            return;
+        }
+        listeners.forEach(LifecycleListener::beforeStart);
+        lifecycle.moveToStarted();
+        listeners.forEach(LifecycleListener::afterStart);
+    }
+
+    @Override
+    public void stop() {
+        if (!lifecycle.canMoveToStopped()) {
+            return;
+        }
+        listeners.forEach(LifecycleListener::beforeStop);
+        lifecycle.moveToStopped();
+        listeners.forEach(LifecycleListener::afterStop);
+    }
+
+    @Override
+    public void close() {
+        if (lifecycle.started()) {
+            stop();
+        }
+        if (!lifecycle.canMoveToClosed()) {
+            return;
+        }
+        listeners.forEach(LifecycleListener::beforeClose);
+        lifecycle.moveToClosed();
+        try {
+            closeable.close();
+        } catch (Exception e) {
+            LOG.warn("failed to close {}", getClass().getName(), e);
+        }
+        listeners.forEach(LifecycleListener::afterClose);
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
index 39794c2..e5b42cc 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.PrimitiveIterator;
 import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.common.util.Watcher;
-import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
 
 /**
@@ -87,9 +86,6 @@ public interface Bookie {
     // TODO: Should be constructed and passed in as a parameter
     LedgerStorage getLedgerStorage();
 
-    // TODO: LedgerManagerFactory, should be constructed elsewhere, passed in as parameter
-    LedgerManagerFactory getLedgerManagerFactory();
-
     // TODO: Move this exceptions somewhere else
     /**
      * Exception is thrown when no such a ledger is found in this bookie.
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
index a94b8bd..3a1c3aa 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
@@ -31,36 +31,27 @@ import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.URI;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.file.FileStore;
 import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.PrimitiveIterator.OfLong;
-import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
-import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException;
-import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException;
 import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException;
-import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
-import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException;
-import org.apache.bookkeeper.bookie.BookieException.UnknownBookieIdException;
 import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
 import org.apache.bookkeeper.bookie.Journal.JournalScanner;
 import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener;
@@ -71,11 +62,6 @@ import org.apache.bookkeeper.common.util.Watcher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.meta.LedgerManagerFactory;
-import org.apache.bookkeeper.meta.MetadataBookieDriver;
-import org.apache.bookkeeper.meta.MetadataDrivers;
-import org.apache.bookkeeper.meta.exceptions.MetadataException;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.net.DNS;
@@ -87,12 +73,9 @@ import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.mutable.MutableBoolean;
-import org.apache.commons.lang3.tuple.Pair;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -107,9 +90,8 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
     final ServerConfiguration conf;
 
     final SyncThread syncThread;
-    final LedgerManagerFactory ledgerManagerFactory;
-    final LedgerManager ledgerManager;
     final LedgerStorage ledgerStorage;
+    final RegistrationManager registrationManager;
     final List<Journal> journals;
 
     final HandleFactory handles;
@@ -125,9 +107,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
     private final LedgerDirsManager indexDirsManager;
     LedgerDirsMonitor dirsMonitor;
 
-    // Registration Manager for managing registration
-    protected final MetadataBookieDriver metadataDriver;
-
     private int exitCode = ExitCode.OK;
 
     private final ConcurrentLongHashMap<byte[]> masterKeyCache = new ConcurrentLongHashMap<>();
@@ -188,7 +167,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
      * This means that the configuration has stayed the same as the
      * first run and the filesystem structure is up to date.
      */
-    private void checkEnvironment(MetadataBookieDriver metadataDriver)
+    private void checkEnvironment(RegistrationManager registrationManager)
             throws BookieException, IOException {
         List<File> allLedgerDirs = new ArrayList<File>(ledgerDirsManager.getAllLedgerDirs().size()
                 + indexDirsManager.getAllLedgerDirs().size());
@@ -196,19 +175,14 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         if (indexDirsManager != ledgerDirsManager) {
             allLedgerDirs.addAll(indexDirsManager.getAllLedgerDirs());
         }
-        if (metadataDriver == null) { // exists only for testing, just make sure directories are correct
 
-            for (File journalDirectory : journalDirectories) {
-                checkDirectoryStructure(journalDirectory);
-            }
-
-            for (File dir : allLedgerDirs) {
-                checkDirectoryStructure(dir);
-            }
-            return;
+        for (File journalDirectory : journalDirectories) {
+            checkDirectoryStructure(journalDirectory);
         }
 
-        checkEnvironmentWithStorageExpansion(conf, metadataDriver, journalDirectories, allLedgerDirs);
+        for (File dir : allLedgerDirs) {
+            checkDirectoryStructure(dir);
+        }
 
         checkIfDirsOnSameDiskPartition(allLedgerDirs);
         checkIfDirsOnSameDiskPartition(journalDirectories);
@@ -260,240 +234,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         }
     }
 
-    static List<BookieId> possibleBookieIds(ServerConfiguration conf)
-            throws BookieException {
-        // we need to loop through all possible bookie identifiers to ensure it is treated as a new environment
-        // just because of bad configuration
-        List<BookieId> addresses = Lists.newArrayListWithExpectedSize(3);
-        // we are checking all possibilities here, so we don't need to fail if we can only get
-        // loopback address. it will fail anyway when the bookie attempts to listen on loopback address.
-        try {
-            if (null != conf.getBookieId()) {
-                addresses.add(BookieId.parse(conf.getBookieId()));
-            } else {
-                // ip address
-                addresses.add(getBookieAddress(
-                    new ServerConfiguration(conf)
-                        .setUseHostNameAsBookieID(false)
-                        .setAdvertisedAddress(null)
-                        .setAllowLoopback(true)
-                ).toBookieId());
-                // host name
-                addresses.add(getBookieAddress(
-                    new ServerConfiguration(conf)
-                        .setUseHostNameAsBookieID(true)
-                        .setAdvertisedAddress(null)
-                        .setAllowLoopback(true)
-                ).toBookieId());
-                // advertised address
-                if (null != conf.getAdvertisedAddress()) {
-                    addresses.add(getBookieAddress(conf).toBookieId());
-                }
-            }
-        } catch (UnknownHostException e) {
-            throw new UnknownBookieIdException(e);
-        }
-        return addresses;
-    }
-
-    static Versioned<Cookie> readAndVerifyCookieFromRegistrationManager(
-            Cookie masterCookie, RegistrationManager rm,
-            List<BookieId> addresses, boolean allowExpansion)
-            throws BookieException {
-        Versioned<Cookie> rmCookie = null;
-        for (BookieId address : addresses) {
-            try {
-                rmCookie = Cookie.readFromRegistrationManager(rm, address);
-                // If allowStorageExpansion option is set, we should
-                // make sure that the new set of ledger/index dirs
-                // is a super set of the old; else, we fail the cookie check
-                if (allowExpansion) {
-                    masterCookie.verifyIsSuperSet(rmCookie.getValue());
-                } else {
-                    masterCookie.verify(rmCookie.getValue());
-                }
-            } catch (CookieNotFoundException e) {
-                continue;
-            }
-        }
-        return rmCookie;
-    }
-
-    private static Pair<List<File>, List<Cookie>> verifyAndGetMissingDirs(
-            Cookie masterCookie, boolean allowExpansion, List<File> dirs)
-            throws InvalidCookieException, IOException {
-        List<File> missingDirs = Lists.newArrayList();
-        List<Cookie> existedCookies = Lists.newArrayList();
-        for (File dir : dirs) {
-            checkDirectoryStructure(dir);
-            try {
-                Cookie c = Cookie.readFromDirectory(dir);
-                if (allowExpansion) {
-                    masterCookie.verifyIsSuperSet(c);
-                } else {
-                    masterCookie.verify(c);
-                }
-                existedCookies.add(c);
-            } catch (FileNotFoundException fnf) {
-                missingDirs.add(dir);
-            }
-        }
-        return Pair.of(missingDirs, existedCookies);
-    }
-
-    private static void stampNewCookie(ServerConfiguration conf,
-                                       Cookie masterCookie,
-                                       RegistrationManager rm,
-                                       Version version,
-                                       List<File> journalDirectories,
-                                       List<File> allLedgerDirs)
-            throws BookieException, IOException {
-        // backfill all the directories that miss cookies (for storage expansion)
-        LOG.info("Stamping new cookies on all dirs {} {}",
-                 journalDirectories, allLedgerDirs);
-        for (File journalDirectory : journalDirectories) {
-            masterCookie.writeToDirectory(journalDirectory);
-        }
-        for (File dir : allLedgerDirs) {
-            masterCookie.writeToDirectory(dir);
-        }
-        masterCookie.writeToRegistrationManager(rm, conf, version);
-    }
-
-    public static void checkEnvironmentWithStorageExpansion(
-            ServerConfiguration conf,
-            MetadataBookieDriver metadataDriver,
-            List<File> journalDirectories,
-            List<File> allLedgerDirs) throws BookieException {
-        RegistrationManager rm = metadataDriver.getRegistrationManager();
-        try {
-            // 1. retrieve the instance id
-            String instanceId = rm.getClusterInstanceId();
-
-            // 2. build the master cookie from the configuration
-            Cookie.Builder builder = Cookie.generateCookie(conf);
-            if (null != instanceId) {
-                builder.setInstanceId(instanceId);
-            }
-            Cookie masterCookie = builder.build();
-            boolean allowExpansion = conf.getAllowStorageExpansion();
-
-            // 3. read the cookie from registration manager. it is the `source-of-truth` of a given bookie.
-            //    if it doesn't exist in registration manager, this bookie is a new bookie, otherwise it is
-            //    an old bookie.
-            List<BookieId> possibleBookieIds = possibleBookieIds(conf);
-            final Versioned<Cookie> rmCookie = readAndVerifyCookieFromRegistrationManager(
-                        masterCookie, rm, possibleBookieIds, allowExpansion);
-
-            // 4. check if the cookie appear in all the directories.
-            List<File> missedCookieDirs = new ArrayList<>();
-            List<Cookie> existingCookies = Lists.newArrayList();
-            if (null != rmCookie) {
-                existingCookies.add(rmCookie.getValue());
-            }
-
-            // 4.1 verify the cookies in journal directories
-            Pair<List<File>, List<Cookie>> journalResult =
-                verifyAndGetMissingDirs(masterCookie,
-                                        allowExpansion, journalDirectories);
-            missedCookieDirs.addAll(journalResult.getLeft());
-            existingCookies.addAll(journalResult.getRight());
-            // 4.2. verify the cookies in ledger directories
-            Pair<List<File>, List<Cookie>> ledgerResult =
-                verifyAndGetMissingDirs(masterCookie,
-                                        allowExpansion, allLedgerDirs);
-            missedCookieDirs.addAll(ledgerResult.getLeft());
-            existingCookies.addAll(ledgerResult.getRight());
-
-            // 5. if there are directories missing cookies,
-            //    this is either a:
-            //    - new environment
-            //    - a directory is being added
-            //    - a directory has been corrupted/wiped, which is an error
-            if (!missedCookieDirs.isEmpty()) {
-                if (rmCookie == null) {
-                    // 5.1 new environment: all directories should be empty
-                    verifyDirsForNewEnvironment(missedCookieDirs);
-                    stampNewCookie(conf, masterCookie, rm, Version.NEW,
-                                   journalDirectories, allLedgerDirs);
-                } else if (allowExpansion) {
-                    // 5.2 storage is expanding
-                    Set<File> knownDirs = getKnownDirs(existingCookies);
-                    verifyDirsForStorageExpansion(missedCookieDirs, knownDirs);
-                    stampNewCookie(conf, masterCookie,
-                                   rm, rmCookie.getVersion(),
-                                   journalDirectories, allLedgerDirs);
-                } else {
-                    // 5.3 Cookie-less directories and
-                    //     we can't do anything with them
-                    LOG.error("There are directories without a cookie,"
-                              + " and this is neither a new environment,"
-                              + " nor is storage expansion enabled. "
-                              + "Empty directories are {}", missedCookieDirs);
-                    throw new InvalidCookieException();
-                }
-            } else {
-                if (rmCookie == null) {
-                    // No corresponding cookie found in registration manager. The bookie should fail to come up.
-                    LOG.error("Cookie for this bookie is not stored in metadata store. Bookie failing to come up");
-                    throw new InvalidCookieException();
-                }
-            }
-        } catch (IOException ioe) {
-            LOG.error("Error accessing cookie on disks", ioe);
-            throw new BookieException.InvalidCookieException(ioe);
-        }
-    }
-
-    private static void verifyDirsForNewEnvironment(List<File> missedCookieDirs)
-            throws InvalidCookieException {
-        List<File> nonEmptyDirs = new ArrayList<>();
-        for (File dir : missedCookieDirs) {
-            String[] content = dir.list();
-            if (content != null && content.length != 0) {
-                nonEmptyDirs.add(dir);
-            }
-        }
-        if (!nonEmptyDirs.isEmpty()) {
-            LOG.error("Not all the new directories are empty. New directories that are not empty are: " + nonEmptyDirs);
-            throw new InvalidCookieException();
-        }
-    }
-
-    private static Set<File> getKnownDirs(List<Cookie> cookies) {
-        return cookies.stream()
-            .flatMap((c) -> Arrays.stream(c.getLedgerDirPathsFromCookie()))
-            .map((s) -> new File(s))
-            .collect(Collectors.toSet());
-    }
-
-    private static void verifyDirsForStorageExpansion(
-            List<File> missedCookieDirs,
-            Set<File> existingLedgerDirs) throws InvalidCookieException {
-
-        List<File> dirsMissingData = new ArrayList<File>();
-        List<File> nonEmptyDirs = new ArrayList<File>();
-        for (File dir : missedCookieDirs) {
-            if (existingLedgerDirs.contains(dir.getParentFile())) {
-                // if one of the existing ledger dirs doesn't have cookie,
-                // let us not proceed further
-                dirsMissingData.add(dir);
-                continue;
-            }
-            String[] content = dir.list();
-            if (content != null && content.length != 0) {
-                nonEmptyDirs.add(dir);
-            }
-        }
-        if (dirsMissingData.size() > 0 || nonEmptyDirs.size() > 0) {
-            LOG.error("Either not all local directories have cookies or directories being added "
-                    + " newly are not empty. "
-                    + "Directories missing cookie file are: " + dirsMissingData
-                    + " New directories that are not empty are: " + nonEmptyDirs);
-            throw new InvalidCookieException();
-        }
-    }
-
     public static BookieId getBookieId(ServerConfiguration conf) throws UnknownHostException {
         String customBookieId = conf.getBookieId();
         if (customBookieId != null) {
@@ -582,13 +322,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         return currentDirs;
     }
 
-    private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws IOException {
-        // Instantiate the ledger storage implementation
-        String ledgerStorageClass = conf.getLedgerStorageClass();
-        LOG.info("Using ledger storage: {}", ledgerStorageClass);
-        return LedgerStorageFactory.createLedgerStorage(ledgerStorageClass);
-    }
-
     /**
      * Initialize LedgerStorage instance without checkpointing for use within the shell
      * and other RO users.  ledgerStorage must not have already been initialized.
@@ -605,45 +338,51 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         StatsLogger statsLogger = NullStatsLogger.INSTANCE;
         DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
 
-        LedgerDirsManager ledgerDirsManager = createLedgerDirsManager(
+        LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
                 conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE));
-        LedgerDirsManager indexDirsManager = createIndexDirsManager(
+        LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
                 conf, diskChecker, statsLogger.scope(LD_INDEX_SCOPE), ledgerDirsManager);
 
         if (null == ledgerStorage) {
-            ledgerStorage = buildLedgerStorage(conf);
-        }
-
-        CheckpointSource checkpointSource = new CheckpointSource() {
-            @Override
-            public Checkpoint newCheckpoint() {
-                return Checkpoint.MAX;
-            }
-
-            @Override
-            public void checkpointComplete(Checkpoint checkpoint, boolean compact)
-                    throws IOException {
-            }
-        };
-
-        Checkpointer checkpointer = Checkpointer.NULL;
-
-        ledgerStorage.initialize(
+            ledgerStorage = BookieResources.createLedgerStorage(conf, null,
+                                                                ledgerDirsManager,
+                                                                indexDirsManager,
+                                                                statsLogger,
+                                                                UnpooledByteBufAllocator.DEFAULT);
+        } else {
+            ledgerStorage.initialize(
                 conf,
                 null,
                 ledgerDirsManager,
                 indexDirsManager,
-                null,
-                checkpointSource,
-                checkpointer,
                 statsLogger,
                 UnpooledByteBufAllocator.DEFAULT);
+        }
 
+        ledgerStorage.setCheckpointSource(new CheckpointSource() {
+                @Override
+                public Checkpoint newCheckpoint() {
+                    return Checkpoint.MAX;
+                }
+
+                @Override
+                public void checkpointComplete(Checkpoint checkpoint, boolean compact)
+                        throws IOException {
+                }
+            });
+        ledgerStorage.setCheckpointer(Checkpointer.NULL);
         return ledgerStorage;
     }
 
-    public BookieImpl(ServerConfiguration conf, StatsLogger statsLogger,
-                      ByteBufAllocator allocator, Supplier<BookieServiceInfo> bookieServiceInfoProvider)
+    public BookieImpl(ServerConfiguration conf,
+                      RegistrationManager registrationManager,
+                      LedgerStorage storage,
+                      DiskChecker diskChecker,
+                      LedgerDirsManager ledgerDirsManager,
+                      LedgerDirsManager indexDirsManager,
+                      StatsLogger statsLogger,
+                      ByteBufAllocator allocator,
+                      Supplier<BookieServiceInfo> bookieServiceInfoProvider)
             throws IOException, InterruptedException, BookieException {
         super("Bookie-" + conf.getBookiePort());
         this.bookieServiceInfoProvider = bookieServiceInfoProvider;
@@ -653,30 +392,14 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         for (File journalDirectory : conf.getJournalDirs()) {
             this.journalDirectories.add(getCurrentDirectory(journalDirectory));
         }
-        DiskChecker diskChecker = createDiskChecker(conf);
-        this.ledgerDirsManager = createLedgerDirsManager(conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE));
-        this.indexDirsManager = createIndexDirsManager(conf, diskChecker, statsLogger.scope(LD_INDEX_SCOPE),
-                                                       this.ledgerDirsManager);
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
         this.writeDataToJournal = conf.getJournalWriteData();
         this.allocator = allocator;
-
-        // instantiate zookeeper client to initialize ledger manager
-        this.metadataDriver = instantiateMetadataDriver(conf);
-        checkEnvironment(this.metadataDriver);
-        try {
-            if (this.metadataDriver != null) {
-                // current the registration manager is zookeeper only
-                ledgerManagerFactory = metadataDriver.getLedgerManagerFactory();
-                LOG.info("instantiate ledger manager {}", ledgerManagerFactory.getClass().getName());
-                ledgerManager = ledgerManagerFactory.newLedgerManager();
-            } else {
-                ledgerManagerFactory = null;
-                ledgerManager = null;
-            }
-        } catch (MetadataException e) {
-            throw new MetadataStoreException("Failed to initialize ledger manager", e);
-        }
+        this.registrationManager = registrationManager;
         stateManager = initializeStateManager();
+        checkEnvironment(registrationManager);
+
         // register shutdown handler using trigger mode
         stateManager.setShutdownHandler(exitCode -> triggerBookieShutdown(exitCode));
         // Initialise dirsMonitor. This would look through all the
@@ -709,8 +432,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         this.entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled();
         CheckpointSource checkpointSource = new CheckpointSourceList(journals);
 
-        ledgerStorage = buildLedgerStorage(conf);
-
+        this.ledgerStorage = storage;
         boolean isDbLedgerStorage = ledgerStorage instanceof DbLedgerStorage;
 
         /*
@@ -727,7 +449,8 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
          * 2) DbLedgerStorage
          */
         if (entryLogPerLedgerEnabled || isDbLedgerStorage) {
-            syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource) {
+            syncThread =
+                new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource) {
                 @Override
                 public void startCheckpoint(Checkpoint checkpoint) {
                     /*
@@ -749,17 +472,9 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
             syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource);
         }
 
-        ledgerStorage.initialize(
-            conf,
-            ledgerManager,
-            ledgerDirsManager,
-            indexDirsManager,
-            stateManager,
-            checkpointSource,
-            syncThread,
-            statsLogger,
-            allocator);
-
+        ledgerStorage.setStateManager(stateManager);
+        ledgerStorage.setCheckpointSource(checkpointSource);
+        ledgerStorage.setCheckpointer(syncThread);
 
         handles = new HandleFactoryImpl(ledgerStorage);
 
@@ -768,7 +483,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
     }
 
     StateManager initializeStateManager() throws IOException {
-        return new BookieStateManager(conf, statsLogger, metadataDriver,
+        return new BookieStateManager(conf, statsLogger, registrationManager,
                 ledgerDirsManager, bookieServiceInfoProvider);
     }
 
@@ -1031,34 +746,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         };
     }
 
-    /**
-     * Instantiate the metadata driver for the Bookie.
-     */
-    private MetadataBookieDriver instantiateMetadataDriver(ServerConfiguration conf) throws BookieException {
-        try {
-            String metadataServiceUriStr = conf.getMetadataServiceUri();
-            if (null == metadataServiceUriStr) {
-                return null;
-            }
-
-            MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(
-                URI.create(metadataServiceUriStr));
-            driver.initialize(
-                conf,
-                () -> {
-                    stateManager.forceToUnregistered();
-                    // schedule a re-register operation
-                    stateManager.registerBookie(false);
-                },
-                statsLogger);
-            return driver;
-        } catch (MetadataException me) {
-            throw new MetadataStoreException("Failed to initialize metadata bookie driver", me);
-        } catch (ConfigurationException e) {
-            throw new BookieIllegalOpException(e);
-        }
-    }
-
     /*
      * Check whether Bookie is writable.
      */
@@ -1164,24 +851,10 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
                 ledgerStorage.shutdown();
 
                 // close Ledger Manager
-                try {
-                    if (null != ledgerManager) {
-                        ledgerManager.close();
-                    }
-                    if (null != ledgerManagerFactory) {
-                        ledgerManagerFactory.close();
-                    }
-                } catch (IOException ie) {
-                    LOG.error("Failed to close active ledger manager : ", ie);
-                }
 
                 //Shutdown disk checker
                 dirsMonitor.shutdown();
             }
-            // Shutdown the ZK client
-            if (metadataDriver != null) {
-                metadataDriver.close();
-            }
         } catch (InterruptedException ie) {
             Thread.currentThread().interrupt();
             LOG.error("Interrupted during shutting down bookie : ", ie);
@@ -1447,34 +1120,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         return (BookieStateManager) this.stateManager;
     }
 
-    @VisibleForTesting
-    public LedgerManagerFactory getLedgerManagerFactory() {
-        return ledgerManagerFactory;
-    }
-
-    // The rest of the code is test stuff
-    static class CounterCallback implements WriteCallback {
-        int count;
-
-        @Override
-        public synchronized void writeComplete(int rc, long l, long e, BookieId addr, Object ctx) {
-            count--;
-            if (count == 0) {
-                notifyAll();
-            }
-        }
-
-        public synchronized void incCount() {
-            count++;
-        }
-
-        public synchronized void waitZero() throws InterruptedException {
-            while (count > 0) {
-                wait();
-            }
-        }
-    }
-
     public ByteBufAllocator getAllocator() {
         return allocator;
     }
@@ -1570,25 +1215,6 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         return exitCode;
     }
 
-    static DiskChecker createDiskChecker(ServerConfiguration conf) {
-        return new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
-    }
-
-    static LedgerDirsManager createLedgerDirsManager(ServerConfiguration conf, DiskChecker diskChecker,
-                                                     StatsLogger statsLogger) {
-        return new LedgerDirsManager(conf, conf.getLedgerDirs(), diskChecker, statsLogger);
-    }
-
-    static LedgerDirsManager createIndexDirsManager(ServerConfiguration conf, DiskChecker diskChecker,
-                                                    StatsLogger statsLogger, LedgerDirsManager fallback) {
-        File[] idxDirs = conf.getIndexDirs();
-        if (null == idxDirs) {
-            return fallback;
-        } else {
-            return new LedgerDirsManager(conf, idxDirs, diskChecker, statsLogger);
-        }
-    }
-
     public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException, NoLedgerException {
         long requestNanos = MathUtils.nowInNano();
         boolean success = false;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieResources.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieResources.java
new file mode 100644
index 0000000..b36b9ad
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieResources.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.bookie;
+
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+
+import io.netty.buffer.ByteBufAllocator;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.meta.exceptions.MetadataException;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.configuration.ConfigurationException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Centralizes the creation of injected resources.
+ */
+public class BookieResources {
+    private static final Logger log = LoggerFactory.getLogger(BookieResources.class);
+
+    /**
+     * Instantiate the metadata driver for the Bookie.
+     */
+    public static MetadataBookieDriver createMetadataDriver(ServerConfiguration conf,
+                                                            StatsLogger statsLogger) throws BookieException {
+        try {
+            String metadataServiceUriStr = conf.getMetadataServiceUri();
+            if (null == metadataServiceUriStr) {
+                throw new BookieException.MetadataStoreException("Metadata URI must not be null");
+            }
+
+            MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(
+                URI.create(metadataServiceUriStr));
+            driver.initialize(conf, statsLogger.scope(BOOKIE_SCOPE));
+            return driver;
+        } catch (MetadataException me) {
+            throw new BookieException.MetadataStoreException("Failed to initialize metadata bookie driver", me);
+        } catch (ConfigurationException e) {
+            throw new BookieException.BookieIllegalOpException(e);
+        }
+    }
+
+    public static ByteBufAllocatorWithOomHandler createAllocator(ServerConfiguration conf) {
+        return ByteBufAllocatorBuilder.create()
+            .poolingPolicy(conf.getAllocatorPoolingPolicy())
+            .poolingConcurrency(conf.getAllocatorPoolingConcurrency())
+            .outOfMemoryPolicy(conf.getAllocatorOutOfMemoryPolicy())
+            .leakDetectionPolicy(conf.getAllocatorLeakDetectionPolicy())
+            .build();
+    }
+
+    public static DiskChecker createDiskChecker(ServerConfiguration conf) {
+        return new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+    }
+
+    public static LedgerDirsManager createLedgerDirsManager(ServerConfiguration conf, DiskChecker diskChecker,
+                                                            StatsLogger statsLogger) throws IOException {
+        return new LedgerDirsManager(conf, conf.getLedgerDirs(), diskChecker, statsLogger);
+    }
+
+    public static LedgerDirsManager createIndexDirsManager(ServerConfiguration conf, DiskChecker diskChecker,
+                                                           StatsLogger statsLogger, LedgerDirsManager fallback)
+            throws IOException {
+        File[] idxDirs = conf.getIndexDirs();
+        if (null == idxDirs) {
+            return fallback;
+        } else {
+            return new LedgerDirsManager(conf, idxDirs, diskChecker, statsLogger);
+        }
+    }
+
+    public static LedgerStorage createLedgerStorage(ServerConfiguration conf,
+                                                    LedgerManager ledgerManager,
+                                                    LedgerDirsManager ledgerDirsManager,
+                                                    LedgerDirsManager indexDirsManager,
+                                                    StatsLogger statsLogger,
+                                                    ByteBufAllocator allocator) throws IOException {
+        // Instantiate the ledger storage implementation
+        String ledgerStorageClass = conf.getLedgerStorageClass();
+        log.info("Using ledger storage: {}", ledgerStorageClass);
+        LedgerStorage storage = LedgerStorageFactory.createLedgerStorage(ledgerStorageClass);
+
+        storage.initialize(conf, ledgerManager, ledgerDirsManager, indexDirsManager, statsLogger, allocator);
+        storage.setCheckpointSource(CheckpointSource.DEFAULT);
+        return storage;
+    }
+
+
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java
index d27209c..3d4c0f3 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java
@@ -41,7 +41,6 @@ import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -80,9 +79,9 @@ public class BookieStateManager implements StateManager {
     private final AtomicBoolean forceReadOnly = new AtomicBoolean(false);
     private volatile boolean availableForHighPriorityWrites = true;
 
-    private final BookieId bookieId;
+    private final Supplier<BookieId> bookieIdSupplier;
     private ShutdownHandler shutdownHandler;
-    private final Supplier<RegistrationManager> rm;
+    private final RegistrationManager rm;
     // Expose Stats
     @StatsDoc(
         name = SERVER_STATUS,
@@ -92,13 +91,13 @@ public class BookieStateManager implements StateManager {
 
     public BookieStateManager(ServerConfiguration conf,
                               StatsLogger statsLogger,
-                              MetadataBookieDriver metadataDriver,
+                              RegistrationManager rm,
                               LedgerDirsManager ledgerDirsManager,
                               Supplier<BookieServiceInfo> bookieServiceInfoProvider) throws IOException {
         this(
             conf,
             statsLogger,
-            () -> null == metadataDriver ? null : metadataDriver.getRegistrationManager(),
+            rm,
             ledgerDirsManager.getAllLedgerDirs(),
             () -> {
                 try {
@@ -111,15 +110,23 @@ public class BookieStateManager implements StateManager {
     }
     public BookieStateManager(ServerConfiguration conf,
                               StatsLogger statsLogger,
-                              Supplier<RegistrationManager> rm,
+                              RegistrationManager rm,
                               List<File> statusDirs,
                               Supplier<BookieId> bookieIdSupplier,
                               Supplier<BookieServiceInfo> bookieServiceInfoProvider) throws IOException {
         this.conf = conf;
         this.rm = rm;
+        if (this.rm != null) {
+            rm.addRegistrationListener(() -> {
+                    forceToUnregistered();
+                    // schedule a re-register operation
+                    registerBookie(false);
+                });
+        }
+
         this.statusDirs = statusDirs;
         // ZK ephemeral node for this Bookie.
-        this.bookieId = bookieIdSupplier.get();
+        this.bookieIdSupplier = bookieIdSupplier;
         this.bookieServiceInfoProvider = bookieServiceInfoProvider;
         // 1 : up, 0 : readonly, -1 : unregistered
         this.serverStatusGauge = new Gauge<Number>() {
@@ -143,12 +150,12 @@ public class BookieStateManager implements StateManager {
     }
 
     private boolean isRegistrationManagerDisabled() {
-        return null == rm || null == rm.get();
+        return null == rm;
     }
 
     @VisibleForTesting
-    BookieStateManager(ServerConfiguration conf, MetadataBookieDriver metadataDriver) throws IOException {
-        this(conf, NullStatsLogger.INSTANCE, metadataDriver, new LedgerDirsManager(conf, conf.getLedgerDirs(),
+    BookieStateManager(ServerConfiguration conf, RegistrationManager registrationManager) throws IOException {
+        this(conf, NullStatsLogger.INSTANCE, registrationManager, new LedgerDirsManager(conf, conf.getLedgerDirs(),
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()),
                 NullStatsLogger.INSTANCE), BookieServiceInfo.NO_INFO);
     }
@@ -270,7 +277,7 @@ public class BookieStateManager implements StateManager {
 
         rmRegistered.set(false);
         try {
-            rm.get().registerBookie(bookieId, isReadOnly, bookieServiceInfoProvider.get());
+            rm.registerBookie(bookieIdSupplier.get(), isReadOnly, bookieServiceInfoProvider.get());
             rmRegistered.set(true);
         } catch (BookieException e) {
             throw new IOException(e);
@@ -304,7 +311,7 @@ public class BookieStateManager implements StateManager {
         }
         // clear the readonly state
         try {
-            rm.get().unregisterBookie(bookieId, true);
+            rm.unregisterBookie(bookieIdSupplier.get(), true);
         } catch (BookieException e) {
             // if we failed when deleting the readonly flag in zookeeper, it is OK since client would
             // already see the bookie in writable list. so just log the exception
@@ -340,7 +347,7 @@ public class BookieStateManager implements StateManager {
             return;
         }
         try {
-            rm.get().registerBookie(bookieId, true, bookieServiceInfoProvider.get());
+            rm.registerBookie(bookieIdSupplier.get(), true, bookieServiceInfoProvider.get());
         } catch (BookieException e) {
             LOG.error("Error in transition to ReadOnly Mode."
                     + " Shutting down", e);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
index a6b41f4..07ef316 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
@@ -90,4 +90,14 @@ public interface CheckpointSource {
      *          Flag to compact old checkpoints.
      */
     void checkpointComplete(Checkpoint checkpoint, boolean compact) throws IOException;
+
+    CheckpointSource DEFAULT = new CheckpointSource() {
+            @Override
+            public Checkpoint newCheckpoint() {
+                return Checkpoint.MAX;
+            }
+
+            @Override
+            public void checkpointComplete(Checkpoint checkpoint, boolean compact) {}
+        };
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java
new file mode 100644
index 0000000..9b88ff8
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.bookie;
+
+import java.io.File;
+import java.net.UnknownHostException;
+import java.util.List;
+
+/**
+ * Interface for cookie validation.
+ */
+public interface CookieValidation {
+    void checkCookies(List<File> directories) throws BookieException, UnknownHostException;
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java
index 5b05e64..2066e6d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java
@@ -185,7 +185,7 @@ class EntryLoggerAllocator {
 
 
     private synchronized void closePreAllocateLog() {
-        if (preallocatedLogId != -1) {
+        if (preallocation != null) {
             // if preallocate new log success, release the file channel
             try {
                 BufferedLogChannel bufferedLogChannel = getPreallocationFuture().get(3, TimeUnit.SECONDS);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
index 05649b4..32a45fd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
@@ -520,8 +520,10 @@ public class GarbageCollectorThread extends SafeRunnable {
      *
      * @throws InterruptedException if there is an exception stopping gc thread.
      */
-    public void shutdown() throws InterruptedException {
-        this.running = false;
+    public synchronized void shutdown() throws InterruptedException {
+        if (!this.running) {
+            return;
+        }
         LOG.info("Shutting down GarbageCollectorThread");
 
         while (!compacting.compareAndSet(false, true)) {
@@ -529,6 +531,7 @@ public class GarbageCollectorThread extends SafeRunnable {
             Thread.sleep(100);
         }
 
+        this.running = false;
         // Interrupt GC executor thread
         gcExecutor.shutdownNow();
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
index 6da8d21..4cc34c7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
@@ -91,8 +91,8 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
     EntryLogger entryLogger;
     @Getter
     LedgerCache ledgerCache;
-    protected CheckpointSource checkpointSource;
-    protected Checkpointer checkpointer;
+    protected CheckpointSource checkpointSource = CheckpointSource.DEFAULT;
+    protected Checkpointer checkpointer = Checkpointer.NULL;
     private final CopyOnWriteArrayList<LedgerDeletionListener> ledgerDeletionListeners =
             Lists.newCopyOnWriteArrayList();
 
@@ -136,9 +136,6 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
                            LedgerManager ledgerManager,
                            LedgerDirsManager ledgerDirsManager,
                            LedgerDirsManager indexDirsManager,
-                           StateManager stateManager,
-                           CheckpointSource checkpointSource,
-                           Checkpointer checkpointer,
                            StatsLogger statsLogger,
                            ByteBufAllocator allocator)
             throws IOException {
@@ -147,9 +144,6 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
             ledgerManager,
             ledgerDirsManager,
             indexDirsManager,
-            stateManager,
-            checkpointSource,
-            checkpointer,
             this,
             statsLogger,
             allocator);
@@ -159,9 +153,6 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
                                         LedgerManager ledgerManager,
                                         LedgerDirsManager ledgerDirsManager,
                                         LedgerDirsManager indexDirsManager,
-                                        StateManager stateManager,
-                                        CheckpointSource checkpointSource,
-                                        Checkpointer checkpointer,
                                         EntryLogListener entryLogListener,
                                         StatsLogger statsLogger,
                                         ByteBufAllocator allocator) throws IOException {
@@ -170,30 +161,35 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
                 ledgerManager,
                 ledgerDirsManager,
                 indexDirsManager,
-                stateManager,
-                checkpointSource,
-                checkpointer,
                 new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE),
                         allocator),
                 statsLogger);
     }
 
+    @Override
+    public void setStateManager(StateManager stateManager) {}
+
+    @Override
+    public void setCheckpointSource(CheckpointSource checkpointSource) {
+        this.checkpointSource = checkpointSource;
+    }
+
+    @Override
+    public void setCheckpointer(Checkpointer checkpointer) {
+        this.checkpointer = checkpointer;
+    }
+
     @VisibleForTesting
     public void initializeWithEntryLogger(ServerConfiguration conf,
                 LedgerManager ledgerManager,
                 LedgerDirsManager ledgerDirsManager,
                 LedgerDirsManager indexDirsManager,
-                StateManager stateManager,
-                CheckpointSource checkpointSource,
-                Checkpointer checkpointer,
                 EntryLogger entryLogger,
                 StatsLogger statsLogger) throws IOException {
         checkNotNull(checkpointSource, "invalid null checkpoint source");
         checkNotNull(checkpointer, "invalid null checkpointer");
         this.entryLogger = entryLogger;
         this.entryLogger.addListener(this);
-        this.checkpointSource = checkpointSource;
-        this.checkpointer = checkpointer;
         ledgerCache = new LedgerCacheImpl(conf, activeLedgers,
                 null == indexDirsManager ? ledgerDirsManager : indexDirsManager, statsLogger);
         gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger.scope("gc"));
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
index e0c86e8..ad80d4a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
@@ -89,10 +89,10 @@ public class InterleavedStorageRegenerateIndexOp {
     public void initiate(boolean dryRun) throws IOException {
         LOG.info("Starting index rebuilding");
 
-        DiskChecker diskChecker = BookieImpl.createDiskChecker(conf);
-        LedgerDirsManager ledgerDirsManager = BookieImpl.createLedgerDirsManager(
+        DiskChecker diskChecker = BookieResources.createDiskChecker(conf);
+        LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
                 conf, diskChecker, NullStatsLogger.INSTANCE);
-        LedgerDirsManager indexDirsManager = BookieImpl.createIndexDirsManager(
+        LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
                 conf, diskChecker,  NullStatsLogger.INSTANCE, ledgerDirsManager);
         EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager);
         final LedgerCache ledgerCache;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
index 2768aff..5ca39c5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
@@ -58,12 +58,16 @@ public class LedgerDirsManager {
 
     private final DiskChecker diskChecker;
 
-    public LedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker) {
+    public LedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker) throws IOException {
         this(conf, dirs, diskChecker, NullStatsLogger.INSTANCE);
     }
 
-    public LedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker, StatsLogger statsLogger) {
+    public LedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker, StatsLogger statsLogger)
+            throws IOException {
         this.ledgerDirectories = Arrays.asList(BookieImpl.getCurrentDirectories(dirs));
+        for (File f : this.ledgerDirectories) {
+            BookieImpl.checkDirectoryStructure(f);
+        }
         this.writableLedgerDirectories = new ArrayList<File>(ledgerDirectories);
         this.filledDirs = new ArrayList<File>();
         this.listeners = new ArrayList<LedgerDirsListener>();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
index 7342f11..a671832 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
@@ -54,13 +54,14 @@ public interface LedgerStorage {
                     LedgerManager ledgerManager,
                     LedgerDirsManager ledgerDirsManager,
                     LedgerDirsManager indexDirsManager,
-                    StateManager stateManager,
-                    CheckpointSource checkpointSource,
-                    Checkpointer checkpointer,
                     StatsLogger statsLogger,
                     ByteBufAllocator allocator)
             throws IOException;
 
+    void setStateManager(StateManager stateManager);
+    void setCheckpointSource(CheckpointSource checkpointSource);
+    void setCheckpointer(Checkpointer checkpointer);
+
     /**
      * Start any background threads belonging to the storage system. For example, garbage collection.
      */
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LegacyCookieValidation.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LegacyCookieValidation.java
new file mode 100644
index 0000000..481e342
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LegacyCookieValidation.java
@@ -0,0 +1,269 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.bookie;
+
+import com.google.common.collect.Lists;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.commons.lang3.tuple.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Legacy implementation of CookieValidation.
+ */
+public class LegacyCookieValidation implements CookieValidation {
+    private static final Logger log = LoggerFactory.getLogger(LegacyCookieValidation.class);
+
+    private final ServerConfiguration conf;
+    private final RegistrationManager registrationManager;
+
+    public LegacyCookieValidation(ServerConfiguration conf,
+                                  RegistrationManager registrationManager) {
+        this.conf = conf;
+        this.registrationManager = registrationManager;
+    }
+
+    @Override
+    public void checkCookies(List<File> directories) throws BookieException {
+        try {
+            // 1. retrieve the instance id
+            String instanceId = registrationManager.getClusterInstanceId();
+
+            // 2. build the master cookie from the configuration
+            Cookie.Builder builder = Cookie.generateCookie(conf);
+            if (null != instanceId) {
+                builder.setInstanceId(instanceId);
+            }
+            Cookie masterCookie = builder.build();
+            boolean allowExpansion = conf.getAllowStorageExpansion();
+
+            // 3. read the cookie from registration manager. it is the `source-of-truth` of a given bookie.
+            //    if it doesn't exist in registration manager, this bookie is a new bookie, otherwise it is
+            //    an old bookie.
+            List<BookieId> possibleBookieIds = possibleBookieIds(conf);
+            final Versioned<Cookie> rmCookie = readAndVerifyCookieFromRegistrationManager(
+                    masterCookie, registrationManager, possibleBookieIds, allowExpansion);
+
+            // 4. check if the cookie appear in all the directories.
+            List<File> missedCookieDirs = new ArrayList<>();
+            List<Cookie> existingCookies = Lists.newArrayList();
+            if (null != rmCookie) {
+                existingCookies.add(rmCookie.getValue());
+            }
+
+            // 4.1 verify the cookies in journal directories
+            Pair<List<File>, List<Cookie>> result =
+                    verifyAndGetMissingDirs(masterCookie,
+                            allowExpansion, directories);
+            missedCookieDirs.addAll(result.getLeft());
+            existingCookies.addAll(result.getRight());
+
+            // 5. if there are directories missing cookies,
+            //    this is either a:
+            //    - new environment
+            //    - a directory is being added
+            //    - a directory has been corrupted/wiped, which is an error
+            if (!missedCookieDirs.isEmpty()) {
+                if (rmCookie == null) {
+                    // 5.1 new environment: all directories should be empty
+                    verifyDirsForNewEnvironment(missedCookieDirs);
+                    stampNewCookie(conf, masterCookie, registrationManager,
+                            Version.NEW, directories);
+                } else if (allowExpansion) {
+                    // 5.2 storage is expanding
+                    Set<File> knownDirs = getKnownDirs(existingCookies);
+                    verifyDirsForStorageExpansion(missedCookieDirs, knownDirs);
+                    stampNewCookie(conf, masterCookie, registrationManager,
+                            rmCookie.getVersion(), directories);
+                } else {
+                    // 5.3 Cookie-less directories and
+                    //     we can't do anything with them
+                    log.error("There are directories without a cookie,"
+                            + " and this is neither a new environment,"
+                            + " nor is storage expansion enabled. "
+                            + "Empty directories are {}", missedCookieDirs);
+                    throw new BookieException.InvalidCookieException();
+                }
+            } else {
+                if (rmCookie == null) {
+                    // No corresponding cookie found in registration manager. The bookie should fail to come up.
+                    log.error("Cookie for this bookie is not stored in metadata store. Bookie failing to come up");
+                    throw new BookieException.InvalidCookieException();
+                }
+            }
+        } catch (IOException ioe) {
+            log.error("Error accessing cookie on disks", ioe);
+            throw new BookieException.InvalidCookieException(ioe);
+        }
+    }
+
+    private static List<BookieId> possibleBookieIds(ServerConfiguration conf)
+            throws BookieException {
+        // we need to loop through all possible bookie identifiers to ensure it is treated as a new environment
+        // just because of bad configuration
+        List<BookieId> addresses = Lists.newArrayListWithExpectedSize(3);
+        // we are checking all possibilities here, so we don't need to fail if we can only get
+        // loopback address. it will fail anyway when the bookie attempts to listen on loopback address.
+        try {
+            // ip address
+            addresses.add(BookieImpl.getBookieAddress(
+                    new ServerConfiguration(conf)
+                            .setUseHostNameAsBookieID(false)
+                            .setAdvertisedAddress(null)
+                            .setAllowLoopback(true)
+            ).toBookieId());
+            // host name
+            addresses.add(BookieImpl.getBookieAddress(
+                    new ServerConfiguration(conf)
+                            .setUseHostNameAsBookieID(true)
+                            .setAdvertisedAddress(null)
+                            .setAllowLoopback(true)
+            ).toBookieId());
+            // advertised address
+            if (null != conf.getAdvertisedAddress()) {
+                addresses.add(BookieImpl.getBookieId(conf));
+            }
+        } catch (UnknownHostException e) {
+            throw new BookieException.UnknownBookieIdException(e);
+        }
+        return addresses;
+    }
+
+    private static Versioned<Cookie> readAndVerifyCookieFromRegistrationManager(
+            Cookie masterCookie, RegistrationManager rm,
+            List<BookieId> addresses, boolean allowExpansion)
+            throws BookieException {
+        Versioned<Cookie> rmCookie = null;
+        for (BookieId address : addresses) {
+            try {
+                rmCookie = Cookie.readFromRegistrationManager(rm, address);
+                // If allowStorageExpansion option is set, we should
+                // make sure that the new set of ledger/index dirs
+                // is a super set of the old; else, we fail the cookie check
+                if (allowExpansion) {
+                    masterCookie.verifyIsSuperSet(rmCookie.getValue());
+                } else {
+                    masterCookie.verify(rmCookie.getValue());
+                }
+            } catch (BookieException.CookieNotFoundException e) {
+                continue;
+            }
+        }
+        return rmCookie;
+    }
+
+    private static Pair<List<File>, List<Cookie>> verifyAndGetMissingDirs(
+            Cookie masterCookie, boolean allowExpansion, List<File> dirs)
+            throws BookieException.InvalidCookieException, IOException {
+        List<File> missingDirs = Lists.newArrayList();
+        List<Cookie> existedCookies = Lists.newArrayList();
+        for (File dir : dirs) {
+            try {
+                Cookie c = Cookie.readFromDirectory(dir);
+                if (allowExpansion) {
+                    masterCookie.verifyIsSuperSet(c);
+                } else {
+                    masterCookie.verify(c);
+                }
+                existedCookies.add(c);
+            } catch (FileNotFoundException fnf) {
+                missingDirs.add(dir);
+            }
+        }
+        return Pair.of(missingDirs, existedCookies);
+    }
+
+    private static void verifyDirsForNewEnvironment(List<File> missedCookieDirs)
+            throws BookieException.InvalidCookieException {
+        List<File> nonEmptyDirs = new ArrayList<>();
+        for (File dir : missedCookieDirs) {
+            String[] content = dir.list();
+            if (content != null && content.length != 0) {
+                nonEmptyDirs.add(dir);
+            }
+        }
+        if (!nonEmptyDirs.isEmpty()) {
+            log.error("Not all the new directories are empty. New directories that are not empty are: " + nonEmptyDirs);
+            throw new BookieException.InvalidCookieException();
+        }
+    }
+
+    private static void stampNewCookie(ServerConfiguration conf,
+                                       Cookie masterCookie,
+                                       RegistrationManager rm,
+                                       Version version,
+                                       List<File> dirs)
+            throws BookieException, IOException {
+        // backfill all the directories that miss cookies (for storage expansion)
+        log.info("Stamping new cookies on all dirs {}", dirs);
+        for (File dir : dirs) {
+            masterCookie.writeToDirectory(dir);
+        }
+        masterCookie.writeToRegistrationManager(rm, conf, version);
+    }
+
+    private static Set<File> getKnownDirs(List<Cookie> cookies) {
+        return cookies.stream()
+                .flatMap((c) -> Arrays.stream(c.getLedgerDirPathsFromCookie()))
+                .map((s) -> new File(s))
+                .collect(Collectors.toSet());
+    }
+
+    private static void verifyDirsForStorageExpansion(
+            List<File> missedCookieDirs,
+            Set<File> existingLedgerDirs) throws BookieException.InvalidCookieException {
+
+        List<File> dirsMissingData = new ArrayList<File>();
+        List<File> nonEmptyDirs = new ArrayList<File>();
+        for (File dir : missedCookieDirs) {
+            if (existingLedgerDirs.contains(dir.getParentFile())) {
+                // if one of the existing ledger dirs doesn't have cookie,
+                // let us not proceed further
+                dirsMissingData.add(dir);
+                continue;
+            }
+            String[] content = dir.list();
+            if (content != null && content.length != 0) {
+                nonEmptyDirs.add(dir);
+            }
+        }
+        if (dirsMissingData.size() > 0 || nonEmptyDirs.size() > 0) {
+            log.error("Either not all local directories have cookies or directories being added "
+                    + " newly are not empty. "
+                    + "Directories missing cookie file are: " + dirsMissingData
+                    + " New directories that are not empty are: " + nonEmptyDirs);
+            throw new BookieException.InvalidCookieException();
+        }
+    }
+}
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java
index e6cdf64..4615d91 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java
@@ -27,7 +27,9 @@ import java.io.IOException;
 import java.util.function.Supplier;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,10 +44,17 @@ public class ReadOnlyBookie extends BookieImpl {
 
     private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyBookie.class);
 
-    public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger,
-            ByteBufAllocator allocator, Supplier<BookieServiceInfo> bookieServiceInfoProvider)
+    public ReadOnlyBookie(ServerConfiguration conf,
+                          RegistrationManager registrationManager,
+                          LedgerStorage storage,
+                          DiskChecker diskChecker,
+                          LedgerDirsManager ledgerDirsManager,
+                          LedgerDirsManager indexDirsManager,
+                          StatsLogger statsLogger,
+                          ByteBufAllocator allocator, Supplier<BookieServiceInfo> bookieServiceInfoProvider)
             throws IOException, KeeperException, InterruptedException, BookieException {
-        super(conf, statsLogger, allocator, bookieServiceInfoProvider);
+        super(conf, registrationManager, storage, diskChecker,
+              ledgerDirsManager, indexDirsManager, statsLogger, allocator, bookieServiceInfoProvider);
         if (conf.isReadOnlyModeEnabled()) {
             stateManager.forceToReadOnly();
         } else {
@@ -55,23 +64,4 @@ public class ReadOnlyBookie extends BookieImpl {
         }
         LOG.info("Running bookie in force readonly mode.");
     }
-
-    @Override
-    StateManager initializeStateManager() throws IOException {
-        return new BookieStateManager(conf, statsLogger, metadataDriver, getLedgerDirsManager(),
-                                      bookieServiceInfoProvider) {
-
-            @Override
-            public void doTransitionToWritableMode() {
-                // no-op
-                LOG.info("Skip transition to writable mode for readonly bookie");
-            }
-
-            @Override
-            public void doTransitionToReadOnlyMode() {
-                // no-op
-                LOG.info("Skip transition to readonly mode for readonly bookie");
-            }
-        };
-    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
index 1d28dd1..93bf47c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
@@ -297,8 +297,6 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector {
             MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(URI.create(metadataServiceUriStr));
             driver.initialize(
                     conf,
-                    () -> {
-                    },
                     statsLogger);
             return driver;
         } catch (MetadataException me) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
index 60822b8..b8a46c1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
@@ -59,6 +59,8 @@ public class SortedLedgerStorage
     EntryMemTable memTable;
     private ScheduledExecutorService scheduler;
     private StateManager stateManager;
+    private ServerConfiguration conf;
+    private StatsLogger statsLogger;
     private final InterleavedLedgerStorage interleavedLedgerStorage;
 
     public SortedLedgerStorage() {
@@ -75,38 +77,48 @@ public class SortedLedgerStorage
                            LedgerManager ledgerManager,
                            LedgerDirsManager ledgerDirsManager,
                            LedgerDirsManager indexDirsManager,
-                           StateManager stateManager,
-                           CheckpointSource checkpointSource,
-                           Checkpointer checkpointer,
                            StatsLogger statsLogger,
                            ByteBufAllocator allocator)
             throws IOException {
+        this.conf = conf;
+        this.statsLogger = statsLogger;
 
         interleavedLedgerStorage.initializeWithEntryLogListener(
             conf,
             ledgerManager,
             ledgerDirsManager,
             indexDirsManager,
-            stateManager,
-            checkpointSource,
-            checkpointer,
             // uses sorted ledger storage's own entry log listener
             // since it manages entry log rotations and checkpoints.
             this,
             statsLogger,
             allocator);
 
-        if (conf.isEntryLogPerLedgerEnabled()) {
-            this.memTable = new EntryMemTableWithParallelFlusher(conf, checkpointSource, statsLogger);
-        } else {
-            this.memTable = new EntryMemTable(conf, checkpointSource, statsLogger);
-        }
         this.scheduler = Executors.newSingleThreadScheduledExecutor(
                 new ThreadFactoryBuilder()
                 .setNameFormat("SortedLedgerStorage-%d")
                 .setPriority((Thread.NORM_PRIORITY + Thread.MAX_PRIORITY) / 2).build());
+    }
+
+    @Override
+    public void setStateManager(StateManager stateManager) {
+        interleavedLedgerStorage.setStateManager(stateManager);
         this.stateManager = stateManager;
     }
+    @Override
+    public void setCheckpointSource(CheckpointSource checkpointSource) {
+        interleavedLedgerStorage.setCheckpointSource(checkpointSource);
+
+        if (conf.isEntryLogPerLedgerEnabled()) {
+            this.memTable = new EntryMemTableWithParallelFlusher(conf, checkpointSource, statsLogger);
+        } else {
+            this.memTable = new EntryMemTable(conf, checkpointSource, statsLogger);
+        }
+    }
+    @Override
+    public void setCheckpointer(Checkpointer checkpointer) {
+        interleavedLedgerStorage.setCheckpointer(checkpointer);
+    }
 
     @VisibleForTesting
     ScheduledExecutorService getScheduler() {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
index 5c42781..28bcf3f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
@@ -95,8 +95,8 @@ public class DbLedgerStorage implements LedgerStorage {
 
     @Override
     public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager,
-            LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource,
-            Checkpointer checkpointer, StatsLogger statsLogger, ByteBufAllocator allocator) throws IOException {
+                           LedgerDirsManager indexDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator)
+            throws IOException {
         long writeCacheMaxSize = getLongVariableOrDefault(conf, WRITE_CACHE_MAX_SIZE_MB,
                 DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB;
         long readCacheMaxSize = getLongVariableOrDefault(conf, READ_AHEAD_CACHE_MAX_SIZE_MB,
@@ -127,7 +127,7 @@ public class DbLedgerStorage implements LedgerStorage {
             dirs[0] = ledgerDir.getParentFile();
             LedgerDirsManager ldm = new LedgerDirsManager(conf, dirs, ledgerDirsManager.getDiskChecker(), statsLogger);
             ledgerStorageList.add(newSingleDirectoryDbLedgerStorage(conf, ledgerManager, ldm, indexDirsManager,
-                    stateManager, checkpointSource, checkpointer, statsLogger, gcExecutor, perDirectoryWriteCacheSize,
+                    statsLogger, gcExecutor, perDirectoryWriteCacheSize,
                     perDirectoryReadCacheSize));
             ldm.getListeners().forEach(ledgerDirsManager::addLedgerDirsListener);
         }
@@ -144,12 +144,23 @@ public class DbLedgerStorage implements LedgerStorage {
     @VisibleForTesting
     protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf,
             LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager,
-            StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer,
             StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize)
             throws IOException {
         return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager,
-                stateManager, checkpointSource, checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize,
-                readCacheSize);
+                                                  statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize);
+    }
+
+    @Override
+    public void setStateManager(StateManager stateManager) {
+        ledgerStorageList.forEach(s -> s.setStateManager(stateManager));
+    }
+    @Override
+    public void setCheckpointSource(CheckpointSource checkpointSource) {
+        ledgerStorageList.forEach(s -> s.setCheckpointSource(checkpointSource));
+    }
+    @Override
+    public void setCheckpointer(Checkpointer checkpointer) {
+        ledgerStorageList.forEach(s -> s.setCheckpointer(checkpointer));
     }
 
     @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
index 5e44b76..15410ef 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
@@ -181,7 +181,7 @@ public class LedgersIndexRebuildOp {
         }
     }
 
-    private List<Journal> getJournals(ServerConfiguration conf) {
+    private List<Journal> getJournals(ServerConfiguration conf) throws IOException {
         List<Journal> journals = Lists.newArrayListWithCapacity(conf.getJournalDirs().length);
         int idx = 0;
         for (File journalDir : conf.getJournalDirs()) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
index 94904be..f0bbd28 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
@@ -118,7 +118,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     private final CopyOnWriteArrayList<LedgerDeletionListener> ledgerDeletionListeners = Lists
             .newCopyOnWriteArrayList();
 
-    private final CheckpointSource checkpointSource;
+    private CheckpointSource checkpointSource = CheckpointSource.DEFAULT;
     private Checkpoint lastCheckpoint = Checkpoint.MIN;
 
     private final long writeCacheMaxSize;
@@ -137,8 +137,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     private final long maxReadAheadBytesSize;
 
     public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager,
-            LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager,
-            CheckpointSource checkpointSource, Checkpointer checkpointer, StatsLogger statsLogger,
+            LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger,
             ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize)
             throws IOException {
 
@@ -152,8 +151,6 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
         this.writeCache = new WriteCache(allocator, writeCacheMaxSize / 2);
         this.writeCacheBeingFlushed = new WriteCache(allocator, writeCacheMaxSize / 2);
 
-        this.checkpointSource = checkpointSource;
-
         readCacheMaxSize = readCacheSize;
         readAheadCacheBatchSize = conf.getInt(READ_AHEAD_CACHE_BATCH_SIZE, DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE);
 
@@ -190,12 +187,21 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
 
     @Override
     public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager,
-            LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource,
-            Checkpointer checkpointer, StatsLogger statsLogger,
+            LedgerDirsManager indexDirsManager, StatsLogger statsLogger,
             ByteBufAllocator allocator) throws IOException {
         /// Initialized in constructor
     }
 
+    @Override
+    public void setStateManager(StateManager stateManager) { }
+
+    @Override
+    public void setCheckpointSource(CheckpointSource checkpointSource) {
+        this.checkpointSource = checkpointSource;
+    }
+    @Override
+    public void setCheckpointer(Checkpointer checkpointer) { }
+
     /**
      * Evict all the ledger info object that were not used recently.
      */
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index 4eadb12..8332218 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -64,6 +64,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.LedgerAuditorManager;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator;
@@ -1231,32 +1232,34 @@ public class BookKeeperAdmin implements AutoCloseable {
             boolean isInteractive, boolean force) throws Exception {
         return runFunctionWithMetadataBookieDriver(conf, driver -> {
             try {
-                boolean ledgerRootExists = driver.getRegistrationManager().prepareFormat();
-
-                // If old data was there then confirm with admin.
-                boolean doFormat = true;
-                if (ledgerRootExists) {
-                    if (!isInteractive) {
-                        // If non interactive and force is set, then delete old data.
-                        doFormat = force;
-                    } else {
-                        // Confirm with the admin.
-                        doFormat = IOUtils
-                            .confirmPrompt("Ledger root already exists. "
-                                + "Are you sure to format bookkeeper metadata? "
-                                + "This may cause data loss.");
+                try (RegistrationManager regManager = driver.createRegistrationManager()) {
+                    boolean ledgerRootExists = regManager.prepareFormat();
+
+                    // If old data was there then confirm with admin.
+                    boolean doFormat = true;
+                    if (ledgerRootExists) {
+                        if (!isInteractive) {
+                            // If non interactive and force is set, then delete old data.
+                            doFormat = force;
+                        } else {
+                            // Confirm with the admin.
+                            doFormat = IOUtils
+                                    .confirmPrompt("Ledger root already exists. "
+                                            + "Are you sure to format bookkeeper metadata? "
+                                            + "This may cause data loss.");
+                        }
                     }
-                }
 
-                if (!doFormat) {
-                    return false;
-                }
+                    if (!doFormat) {
+                        return false;
+                    }
 
-                driver.getLedgerManagerFactory().format(
-                    conf,
-                    driver.getLayoutManager());
+                    driver.getLedgerManagerFactory().format(
+                            conf,
+                            driver.getLayoutManager());
 
-                return driver.getRegistrationManager().format();
+                    return regManager.format();
+                }
             } catch (Exception e) {
                 throw new UncheckedExecutionException(e.getMessage(), e);
             }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java
index 5068851..ee8bd2f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java
@@ -143,4 +143,11 @@ public interface RegistrationManager extends AutoCloseable {
      * @throws Exception
      */
     boolean nukeExistingCluster() throws Exception;
+
+    /**
+     * Add a listener to be triggered when an registration event occurs.
+     *
+     * @param listener the listener to be added
+     */
+    void addRegistrationListener(RegistrationListener listener);
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
index df5499b..5826361 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Lists;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
@@ -109,17 +110,16 @@ public class ZKRegistrationManager implements RegistrationManager {
     protected final String bookieReadonlyRegistrationPath;
     // session timeout in milliseconds
     private final int zkTimeoutMs;
+    private final List<RegistrationListener> listeners = new ArrayList<>();
 
     public ZKRegistrationManager(ServerConfiguration conf,
-                                 ZooKeeper zk,
-                                 RegistrationListener listener) {
-        this(conf, zk, ZKMetadataDriverBase.resolveZkLedgersRootPath(conf), listener);
+                                 ZooKeeper zk) {
+        this(conf, zk, ZKMetadataDriverBase.resolveZkLedgersRootPath(conf));
     }
 
     public ZKRegistrationManager(ServerConfiguration conf,
                                  ZooKeeper zk,
-                                 String ledgersRootPath,
-                                 RegistrationListener listener) {
+                                 String ledgersRootPath) {
         this.conf = conf;
         this.zk = zk;
         this.zkAcls = ZkUtils.getACLs(conf);
@@ -142,7 +142,7 @@ public class ZKRegistrationManager implements RegistrationManager {
             // Check for expired connection.
             if (event.getType().equals(EventType.None)
                 && event.getState().equals(KeeperState.Expired)) {
-                listener.onRegistrationExpired();
+                listeners.forEach(RegistrationListener::onRegistrationExpired);
             }
         });
     }
@@ -610,4 +610,9 @@ public class ZKRegistrationManager implements RegistrationManager {
             throw new MetadataStoreException(e);
         }
     }
+
+    @Override
+    public void addRegistrationListener(RegistrationListener listener) {
+        listeners.add(listener);
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
index b4d0e0d..dbbab2e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
@@ -20,7 +20,6 @@ package org.apache.bookkeeper.meta;
 
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
 import org.apache.bookkeeper.stats.StatsLogger;
 
@@ -33,12 +32,10 @@ public interface MetadataBookieDriver extends AutoCloseable {
      * Initialize the metadata driver.
      *
      * @param conf configuration
-     * @param listener registration listener listening on registration state changes.
      * @param statsLogger stats logger
      * @return metadata driver
      */
     MetadataBookieDriver initialize(ServerConfiguration conf,
-                                    RegistrationListener listener,
                                     StatsLogger statsLogger)
         throws MetadataException;
 
@@ -50,11 +47,11 @@ public interface MetadataBookieDriver extends AutoCloseable {
     String getScheme();
 
     /**
-     * Return the registration manager used for registering/unregistering bookies.
+     * Create the registration manager used for registering/unregistering bookies.
      *
      * @return the registration manager used for registering/unregistering bookies.
      */
-    RegistrationManager getRegistrationManager();
+    RegistrationManager createRegistrationManager();
 
     /**
      * Return the ledger manager factory used for accessing ledger metadata.
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
index 90dad34..83fb439 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
@@ -342,7 +342,7 @@ public final class MetadataDrivers {
         try (MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(
             URI.create(conf.getMetadataServiceUri())
         )) {
-            driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
+            driver.initialize(conf, NullStatsLogger.INSTANCE);
             try {
                 return function.apply(driver);
             } catch (Exception uee) {
@@ -369,7 +369,12 @@ public final class MetadataDrivers {
     public static <T> T runFunctionWithRegistrationManager(ServerConfiguration conf,
                                                            Function<RegistrationManager, T> function)
             throws MetadataException, ExecutionException {
-        return runFunctionWithMetadataBookieDriver(conf, driver -> function.apply(driver.getRegistrationManager()));
+        return runFunctionWithMetadataBookieDriver(
+                conf, driver -> {
+                    try (RegistrationManager rm = driver.createRegistrationManager()) {
+                        return function.apply(rm);
+                    }
+                });
     }
 
     /**
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/NullMetadataBookieDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/NullMetadataBookieDriver.java
new file mode 100644
index 0000000..3a4247f
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/NullMetadataBookieDriver.java
@@ -0,0 +1,398 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.meta;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor;
+import org.apache.bookkeeper.replication.ReplicationException;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.AsyncCallback;
+
+/**
+ * A no-op implementation of MetadataBookieDriver.
+ */
+public class NullMetadataBookieDriver implements MetadataBookieDriver {
+    @Override
+    public MetadataBookieDriver initialize(ServerConfiguration conf,
+            StatsLogger statsLogger) {
+        return this;
+    }
+
+    @Override
+    public String getScheme() {
+        return "null";
+    }
+
+    @Override
+    public RegistrationManager createRegistrationManager() {
+        return new NullRegistrationManager();
+    }
+
+    @Override
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return new NullLedgerManagerFactory();
+    }
+
+    @Override
+    public LayoutManager getLayoutManager() {
+        return new NullLayoutManager();
+    }
+
+    @Override
+    public void close() {}
+
+    /**
+     * A no-op implementation of LedgerManagerFactory.
+     */
+    public static class NullLedgerManagerFactory implements LedgerManagerFactory {
+        @Override
+        public int getCurrentVersion() {
+            return 1;
+        }
+        @Override
+        public LedgerManagerFactory initialize(AbstractConfiguration conf,
+                                               LayoutManager layoutManager,
+                                               int factoryVersion) {
+            return this;
+        }
+        @Override
+        public void close() {}
+        @Override
+        public LedgerIdGenerator newLedgerIdGenerator() {
+            return new NullLedgerIdGenerator();
+        }
+
+        @Override
+        public LedgerManager newLedgerManager() {
+            return new NullLedgerManager();
+        }
+
+        @Override
+        public LedgerUnderreplicationManager newLedgerUnderreplicationManager() {
+            return new NullLedgerUnderreplicationManager();
+        }
+
+        @Override
+        public LedgerAuditorManager newLedgerAuditorManager() throws IOException, InterruptedException {
+            return new NullLedgerAuditorManager();
+        }
+
+        @Override
+        public void format(AbstractConfiguration<?> conf, LayoutManager lm) {}
+        @Override
+        public boolean validateAndNukeExistingCluster(AbstractConfiguration<?> conf,
+                                                      LayoutManager lm) {
+            return false;
+        }
+    }
+
+    /**
+     * A no-op implementation of LedgerAuditorManager.
+     */
+    public static class NullLedgerAuditorManager implements LedgerAuditorManager {
+
+        @Override
+        public void tryToBecomeAuditor(String bookieId, Consumer<AuditorEvent> listener)
+                throws IOException, InterruptedException {
+            // no-op
+        }
+
+        @Override
+        public BookieId getCurrentAuditor() throws IOException, InterruptedException {
+            return BookieId.parse("127.0.0.1:3181");
+        }
+
+        @Override
+        public void close() throws Exception {
+            // no-op
+        }
+    }
+
+    /**
+     * A no-op implementation of LayoutManager.
+     */
+    public static class NullLayoutManager implements LayoutManager {
+        @Override
+        public LedgerLayout readLedgerLayout() {
+            return new LedgerLayout("null", -1);
+        }
+
+        @Override
+        public void storeLedgerLayout(LedgerLayout layout) { }
+
+        @Override
+        public void deleteLedgerLayout() { }
+    }
+
+    /**
+     * A no-op implementation of RegistrationManager.
+     */
+    public static class NullRegistrationManager implements RegistrationManager {
+        @Override
+        public void close() {}
+
+        @Override
+        public String getClusterInstanceId() {
+            return "null";
+        }
+
+        @Override
+        public void registerBookie(BookieId bookieId, boolean readOnly, BookieServiceInfo bookieService) {}
+
+        @Override
+        public void unregisterBookie(BookieId bookieId, boolean readOnly) {}
+
+        @Override
+        public boolean isBookieRegistered(BookieId bookieId) {
+            return false;
+        }
+
+        @Override
+        public void writeCookie(BookieId bookieId, Versioned<byte[]> cookieData) throws BookieException {
+
+        }
+
+        @Override
+        public Versioned<byte[]> readCookie(BookieId bookieId) throws BookieException {
+            return null;
+        }
+
+        @Override
+        public void removeCookie(BookieId bookieId, Version version) {}
+
+        @Override
+        public boolean prepareFormat() {
+            return false;
+        }
+
+        @Override
+        public boolean initNewCluster() {
+            return false;
+        }
+
+        @Override
+        public boolean format() {
+            return false;
+        }
+
+        @Override
+        public boolean nukeExistingCluster() {
+            return false;
+        }
+
+        @Override
+        public void addRegistrationListener(RegistrationListener listener) {}
+    }
+
+    /**
+     * A no-op implementation of LedgerIdGenerator.
+     */
+    public static class NullLedgerIdGenerator implements LedgerIdGenerator {
+        @Override
+        public void close() {}
+        @Override
+        public void generateLedgerId(GenericCallback<Long> cb) {
+            cb.operationComplete(BKException.Code.IllegalOpException, null);
+        }
+    }
+
+    /**
+     * A no-op implementation of LedgerManager.
+     */
+    public static class NullLedgerManager implements LedgerManager {
+        private CompletableFuture<Versioned<LedgerMetadata>> illegalOp() {
+            CompletableFuture<Versioned<LedgerMetadata>> promise = new CompletableFuture<>();
+            promise.completeExceptionally(new BKException.BKIllegalOpException());
+            return promise;
+        }
+
+        @Override
+        public CompletableFuture<Versioned<LedgerMetadata>> createLedgerMetadata(long ledgerId,
+                                                                                 LedgerMetadata metadata) {
+            return illegalOp();
+        }
+        @Override
+        public CompletableFuture<Void> removeLedgerMetadata(long ledgerId, Version version) {
+            CompletableFuture<Void> promise = new CompletableFuture<>();
+            promise.completeExceptionally(new BKException.BKIllegalOpException());
+            return promise;
+        }
+
+        @Override
+        public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+            return illegalOp();
+        }
+
+        @Override
+        public CompletableFuture<Versioned<LedgerMetadata>> writeLedgerMetadata(
+                long ledgerId, LedgerMetadata metadata, Version currentVersion) {
+            return illegalOp();
+        }
+
+        @Override
+        public void registerLedgerMetadataListener(long ledgerId,
+                                                   LedgerMetadataListener listener) {}
+        @Override
+        public void unregisterLedgerMetadataListener(long ledgerId,
+                                                     LedgerMetadataListener listener) {}
+        @Override
+        public void asyncProcessLedgers(Processor<Long> processor,
+                                        AsyncCallback.VoidCallback finalCb,
+                                        Object context, int successRc, int failureRc) {}
+        @Override
+        public LedgerManager.LedgerRangeIterator getLedgerRanges(long zkOpTimeOutMs) {
+            return new LedgerManager.LedgerRangeIterator() {
+                @Override
+                public boolean hasNext() {
+                    return false;
+                }
+                @Override
+                public LedgerManager.LedgerRange next() {
+                    throw new NoSuchElementException();
+                }
+            };
+        }
+
+        @Override
+        public void close() {}
+    }
+
+    /**
+     * A no-op implementation of LedgerUnderreplicationManager.
+     */
+    public static class NullLedgerUnderreplicationManager implements LedgerUnderreplicationManager {
+        @Override
+        public boolean isLedgerBeingReplicated(long ledgerId) throws ReplicationException {
+            return false;
+        }
+
+        @Override
+        public CompletableFuture<Void> markLedgerUnderreplicatedAsync(long ledgerId,
+                                                                      Collection<String> missingReplicas) {
+            CompletableFuture<Void> promise = new CompletableFuture<>();
+            promise.completeExceptionally(new ReplicationException.UnavailableException("null"));
+            return promise;
+        }
+        @Override
+        public void markLedgerReplicated(long ledgerId)
+                throws ReplicationException.UnavailableException {
+            throw new ReplicationException.UnavailableException("null");
+        }
+        @Override
+        public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId)
+                throws ReplicationException.UnavailableException {
+            throw new ReplicationException.UnavailableException("null");
+        }
+        @Override
+        public Iterator<UnderreplicatedLedger> listLedgersToRereplicate(Predicate<List<String>> predicate) {
+            return new Iterator<UnderreplicatedLedger>() {
+                @Override
+                public boolean hasNext() {
+                    return false;
+                }
+                @Override
+                public UnderreplicatedLedger next() {
+                    throw new NoSuchElementException();
+                }
+            };
+        }
+        @Override
+        public long getLedgerToRereplicate() throws ReplicationException.UnavailableException {
+            throw new ReplicationException.UnavailableException("null");
+        }
+        @Override
+        public long pollLedgerToRereplicate() throws ReplicationException.UnavailableException {
+            throw new ReplicationException.UnavailableException("null");
+        }
+
+        @Override
+        public void acquireUnderreplicatedLedger(long ledgerId) throws ReplicationException {
+            // no-op
+        }
+
+        @Override
+        public void releaseUnderreplicatedLedger(long ledgerId) {}
+        @Override
+        public void close() {}
+        @Override
+        public void disableLedgerReplication() {}
+        @Override
+        public void enableLedgerReplication() {}
+        @Override
+        public boolean isLedgerReplicationEnabled() {
+            return false;
+        }
+        @Override
+        public void notifyLedgerReplicationEnabled(GenericCallback<Void> cb) {}
+        @Override
+        public boolean initializeLostBookieRecoveryDelay(int lostBookieRecoveryDelay) {
+            return false;
+        }
+        @Override
+        public void setLostBookieRecoveryDelay(int lostBookieRecoveryDelay) {}
+        @Override
+        public int getLostBookieRecoveryDelay() {
+            return Integer.MAX_VALUE;
+        }
+        @Override
+        public void setCheckAllLedgersCTime(long checkAllLedgersCTime) {}
+        @Override
+        public long getCheckAllLedgersCTime() {
+            return Integer.MAX_VALUE;
+        }
+        @Override
+        public void setPlacementPolicyCheckCTime(long placementPolicyCheckCTime) {}
+        @Override
+        public long getPlacementPolicyCheckCTime() {
+            return Long.MAX_VALUE;
+        }
+        @Override
+        public void setReplicasCheckCTime(long replicasCheckCTime) {}
+        @Override
+        public long getReplicasCheckCTime() {
+            return Long.MAX_VALUE;
+        }
+        @Override
+        public void notifyLostBookieRecoveryDelayChanged(GenericCallback<Void> cb) {}
+        @Override
+        public String getReplicationWorkerIdRereplicatingLedger(long ledgerId)
+                throws ReplicationException.UnavailableException {
+            throw new ReplicationException.UnavailableException("null");
+        }
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriver.java
index e7b1ad0..9eb853b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriver.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriver.java
@@ -20,12 +20,10 @@ package org.apache.bookkeeper.meta.zk;
 
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
 
-import com.google.common.annotations.VisibleForTesting;
 import java.util.Optional;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.discover.ZKRegistrationManager;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
@@ -49,12 +47,9 @@ public class ZKMetadataBookieDriver
     }
 
     ServerConfiguration serverConf;
-    RegistrationManager regManager;
-    RegistrationListener listener;
 
     @Override
     public synchronized MetadataBookieDriver initialize(ServerConfiguration conf,
-                                                        RegistrationListener listener,
                                                         StatsLogger statsLogger)
             throws MetadataException {
         super.initialize(
@@ -64,38 +59,17 @@ public class ZKMetadataBookieDriver
                         conf.getZkRetryBackoffMaxMs(), Integer.MAX_VALUE),
             Optional.empty());
         this.serverConf = conf;
-        this.listener = listener;
         this.statsLogger = statsLogger;
         return this;
     }
 
-    @VisibleForTesting
-    public synchronized void setRegManager(RegistrationManager regManager) {
-        this.regManager = regManager;
-    }
-
     @Override
-    public synchronized RegistrationManager getRegistrationManager() {
-        if (null == regManager) {
-            regManager = new ZKRegistrationManager(
-                serverConf,
-                zk,
-                listener
-            );
-        }
-        return regManager;
+    public synchronized RegistrationManager createRegistrationManager() {
+        return new ZKRegistrationManager(serverConf, zk);
     }
 
     @Override
     public void close() {
-        RegistrationManager rmToClose;
-        synchronized (this) {
-            rmToClose = regManager;
-            regManager = null;
-        }
-        if (null != rmToClose) {
-            rmToClose.close();
-        }
         super.close();
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java
index 17f89de..9c59f1f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java
@@ -352,11 +352,15 @@ class BookieNettyServer {
             // Bind and start to accept incoming connections
             LOG.info("Binding bookie-rpc endpoint to {}", address);
             Channel listen = bootstrap.bind(address.getAddress(), address.getPort()).sync().channel();
+
             if (listen.localAddress() instanceof InetSocketAddress) {
                 if (conf.getBookiePort() == 0) {
+                    // this is really really nasty. It's using the configuration object as a notification
+                    // bus. We should get rid of this at some point
                     conf.setBookiePort(((InetSocketAddress) listen.localAddress()).getPort());
                 }
             }
+
         }
 
         if (conf.isEnableLocalTransport()) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
index 4e6a782..ed83cc8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
@@ -20,7 +20,6 @@
  */
 package org.apache.bookkeeper.proto;
 
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SERVER_SCOPE;
 import static org.apache.bookkeeper.conf.AbstractConfiguration.PERMITTED_STARTUP_USERS;
 import com.google.common.annotations.VisibleForTesting;
@@ -31,25 +30,19 @@ import java.net.UnknownHostException;
 import java.security.AccessControlException;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieCriticalThread;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.ExitCode;
-import org.apache.bookkeeper.bookie.ReadOnlyBookie;
-import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder;
 import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.discover.BookieServiceInfo;
-import org.apache.bookkeeper.discover.BookieServiceInfoUtils;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.processor.RequestProcessor;
 import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.server.Main;
-import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.tls.SecurityException;
 import org.apache.bookkeeper.tls.SecurityHandlerFactory;
@@ -81,26 +74,12 @@ public class BookieServer {
     // Exception handler
     private volatile UncaughtExceptionHandler uncaughtExceptionHandler = null;
 
-    public BookieServer(ServerConfiguration conf) throws IOException,
-            KeeperException, InterruptedException, BookieException,
-            UnavailableException, CompatibilityException, SecurityException {
-        this(conf, NullStatsLogger.INSTANCE, null);
-    }
-
-    public BookieServer(ServerConfiguration conf, StatsLogger statsLogger,
-            Supplier<BookieServiceInfo> bookieServiceInfoProvider)
+    public BookieServer(ServerConfiguration conf,
+                        Bookie bookie,
+                        StatsLogger statsLogger,
+                        ByteBufAllocator allocator)
             throws IOException, KeeperException, InterruptedException,
             BookieException, UnavailableException, CompatibilityException, SecurityException {
-        if (bookieServiceInfoProvider == null) {
-            bookieServiceInfoProvider = () -> {
-                try {
-                    return BookieServiceInfoUtils
-                            .buildLegacyBookieServiceInfo(this.getLocalAddress().toBookieId().toString());
-                } catch (IOException err) {
-                    throw new RuntimeException(err);
-                }
-            };
-        }
         this.conf = conf;
         validateUser(conf);
         String configAsString;
@@ -111,16 +90,10 @@ public class BookieServer {
             LOG.error("Got ParseJsonException while converting Config to JSONString", pe);
         }
 
-        ByteBufAllocator allocator = getAllocator(conf);
         this.statsLogger = statsLogger;
+        this.bookie = bookie;
         this.nettyServer = new BookieNettyServer(this.conf, null, allocator);
-        try {
-            this.bookie = newBookie(conf, allocator, bookieServiceInfoProvider);
-        } catch (IOException | KeeperException | InterruptedException | BookieException e) {
-            // interrupted on constructing a bookie
-            this.nettyServer.shutdown();
-            throw e;
-        }
+
         final SecurityHandlerFactory shFactory;
 
         shFactory = SecurityProviderFactoryFactory
@@ -142,14 +115,6 @@ public class BookieServer {
         this.uncaughtExceptionHandler = exceptionHandler;
     }
 
-    protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator,
-            Supplier<BookieServiceInfo> bookieServiceInfoProvider)
-        throws IOException, KeeperException, InterruptedException, BookieException {
-        return conf.isForceReadOnlyBookie()
-            ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider)
-            : new BookieImpl(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider);
-    }
-
     public void start() throws InterruptedException {
         this.bookie.start();
         // fail fast, when bookie startup is not successful
@@ -307,23 +272,6 @@ public class BookieServer {
         }
     }
 
-    private ByteBufAllocator getAllocator(ServerConfiguration conf) {
-        return ByteBufAllocatorBuilder.create()
-                .poolingPolicy(conf.getAllocatorPoolingPolicy())
-                .poolingConcurrency(conf.getAllocatorPoolingConcurrency())
-                .outOfMemoryPolicy(conf.getAllocatorOutOfMemoryPolicy())
-                .outOfMemoryListener((ex) -> {
-                    try {
-                        LOG.error("Unable to allocate memory, exiting bookie", ex);
-                    } finally {
-                        if (uncaughtExceptionHandler != null) {
-                            uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), ex);
-                        }
-                    }
-                })
-                .leakDetectionPolicy(conf.getAllocatorLeakDetectionPolicy())
-                .build();
-    }
 
     /**
      * Legacy Method to run bookie server.
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
index 64da95c..15677d7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
@@ -18,19 +18,34 @@
 
 package org.apache.bookkeeper.server;
 
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
 import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
 import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
 
 import java.io.File;
+import java.io.IOException;
 import java.net.MalformedURLException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
 import org.apache.bookkeeper.bookie.ExitCode;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
 import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.common.component.ComponentStarter;
 import org.apache.bookkeeper.common.component.LifecycleComponent;
@@ -39,6 +54,10 @@ import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.UncheckedConfigurationException;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
@@ -48,6 +67,7 @@ import org.apache.bookkeeper.server.service.HttpService;
 import org.apache.bookkeeper.server.service.ScrubberService;
 import org.apache.bookkeeper.server.service.StatsProviderService;
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
@@ -299,9 +319,54 @@ public class Main {
         serverBuilder.addComponent(statsProviderService);
         log.info("Load lifecycle component : {}", StatsProviderService.class.getName());
 
-        // 2. build bookie server
+        // 2. Build metadata driver
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf.getServerConf(), rootStatsLogger);
+        serverBuilder.addComponent(new AutoCloseableLifecycleComponent("metadataDriver", metadataDriver));
+        RegistrationManager rm = metadataDriver.createRegistrationManager();
+        serverBuilder.addComponent(new AutoCloseableLifecycleComponent("registrationManager", rm));
+
+        // 3. Build ledger manager
+        LedgerManagerFactory lmFactory = metadataDriver.getLedgerManagerFactory();
+        serverBuilder.addComponent(new AutoCloseableLifecycleComponent("lmFactory", lmFactory));
+        LedgerManager ledgerManager = lmFactory.newLedgerManager();
+        serverBuilder.addComponent(new AutoCloseableLifecycleComponent("ledgerManager", ledgerManager));
+
+        // 4. Build bookie
+        StatsLogger bookieStats = rootStatsLogger.scope(BOOKIE_SCOPE);
+        DiskChecker diskChecker = BookieResources.createDiskChecker(conf.getServerConf());
+        LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
+                conf.getServerConf(), diskChecker, bookieStats.scope(LD_LEDGER_SCOPE));
+        LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
+                conf.getServerConf(), diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
+
+        CookieValidation cookieValidation = new LegacyCookieValidation(conf.getServerConf(), rm);
+        cookieValidation.checkCookies(storageDirectoriesFromConf(conf.getServerConf()));
+
+        ByteBufAllocatorWithOomHandler allocator = BookieResources.createAllocator(conf.getServerConf());
+
+        // bookie takes ownership of storage, so shuts it down
+        LedgerStorage storage = BookieResources.createLedgerStorage(
+                conf.getServerConf(), ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+
+        Bookie bookie;
+        if (conf.getServerConf().isForceReadOnlyBookie()) {
+            bookie = new ReadOnlyBookie(conf.getServerConf(), rm, storage,
+                                        diskChecker,
+                                        ledgerDirsManager, indexDirsManager,
+                                        bookieStats, allocator,
+                                        bookieServiceInfoProvider);
+        } else {
+            bookie = new BookieImpl(conf.getServerConf(), rm, storage,
+                                    diskChecker,
+                                    ledgerDirsManager, indexDirsManager,
+                                    bookieStats, allocator,
+                                    bookieServiceInfoProvider);
+        }
+
+        // 5. build bookie server
         BookieService bookieService =
-            new BookieService(conf, rootStatsLogger, bookieServiceInfoProvider);
+            new BookieService(conf, bookie, rootStatsLogger, allocator);
 
         serverBuilder.addComponent(bookieService);
         log.info("Load lifecycle component : {}", BookieService.class.getName());
@@ -313,7 +378,7 @@ public class Main {
                     conf, bookieService.getServer().getBookie().getLedgerStorage()));
         }
 
-        // 3. build auto recovery
+        // 6. build auto recovery
         if (conf.getServerConf().isAutoRecoveryDaemonEnabled()) {
             AutoRecoveryService autoRecoveryService =
                 new AutoRecoveryService(conf, rootStatsLogger.scope(REPLICATION_SCOPE));
@@ -322,12 +387,13 @@ public class Main {
             log.info("Load lifecycle component : {}", AutoRecoveryService.class.getName());
         }
 
-        // 4. build http service
+        // 7. build http service
         if (conf.getServerConf().isHttpServerEnabled()) {
             BKHttpServiceProvider provider = new BKHttpServiceProvider.Builder()
                 .setBookieServer(bookieService.getServer())
                 .setServerConfiguration(conf.getServerConf())
                 .setStatsProvider(statsProviderService.getStatsProvider())
+                .setLedgerManagerFactory(metadataDriver.getLedgerManagerFactory())
                 .build();
             HttpService httpService =
                 new HttpService(provider, conf, rootStatsLogger);
@@ -335,7 +401,7 @@ public class Main {
             log.info("Load lifecycle component : {}", HttpService.class.getName());
         }
 
-        // 5. build extra services
+        // 8. build extra services
         String[] extraComponents = conf.getServerConf().getExtraServerComponents();
         if (null != extraComponents) {
             try {
@@ -382,4 +448,35 @@ public class Main {
         return new BookieServiceInfo(componentInfoPublisher.getProperties(), endpoints);
     }
 
+    public static List<File> storageDirectoriesFromConf(ServerConfiguration conf) throws IOException {
+        List<File> dirs = new ArrayList<>();
+
+        File[] journalDirs = conf.getJournalDirs();
+        if (journalDirs != null) {
+            for (File j : journalDirs) {
+                File cur = BookieImpl.getCurrentDirectory(j);
+                BookieImpl.checkDirectoryStructure(cur);
+                dirs.add(cur);
+            }
+        }
+
+        File[] ledgerDirs = conf.getLedgerDirs();
+        if (ledgerDirs != null) {
+            for (File l : ledgerDirs) {
+                File cur = BookieImpl.getCurrentDirectory(l);
+                BookieImpl.checkDirectoryStructure(cur);
+                dirs.add(cur);
+            }
+        }
+        File[] indexDirs = conf.getIndexDirs();
+        if (indexDirs != null) {
+            for (File i : indexDirs) {
+                File cur = BookieImpl.getCurrentDirectory(i);
+                BookieImpl.checkDirectoryStructure(cur);
+                dirs.add(cur);
+            }
+        }
+        return dirs;
+    }
+
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
index 76705b9..588d432 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
@@ -36,6 +36,7 @@ import org.apache.bookkeeper.http.HttpServiceProvider;
 import org.apache.bookkeeper.http.service.ErrorHttpService;
 import org.apache.bookkeeper.http.service.HeartbeatService;
 import org.apache.bookkeeper.http.service.HttpEndpointService;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.replication.Auditor;
 import org.apache.bookkeeper.replication.AutoRecoveryMain;
@@ -76,17 +77,20 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
     private final StatsProvider statsProvider;
     private final BookieServer bookieServer;
     private final AutoRecoveryMain autoRecovery;
+    private final LedgerManagerFactory ledgerManagerFactory;
     private final ServerConfiguration serverConf;
     private final BookKeeperAdmin bka;
     private final ExecutorService executor;
 
     private BKHttpServiceProvider(BookieServer bookieServer,
                                   AutoRecoveryMain autoRecovery,
+                                  LedgerManagerFactory ledgerManagerFactory,
                                   ServerConfiguration serverConf,
                                   StatsProvider statsProvider)
         throws IOException, KeeperException, InterruptedException, BKException {
         this.bookieServer = bookieServer;
         this.autoRecovery = autoRecovery;
+        this.ledgerManagerFactory = ledgerManagerFactory;
         this.serverConf = serverConf;
         this.statsProvider = statsProvider;
         ClientConfiguration clientConfiguration = new ClientConfiguration(serverConf);
@@ -132,6 +136,7 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
 
         BookieServer bookieServer = null;
         AutoRecoveryMain autoRecovery = null;
+        LedgerManagerFactory ledgerManagerFactory = null;
         ServerConfiguration serverConf = null;
         StatsProvider statsProvider = null;
 
@@ -155,11 +160,17 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
             return this;
         }
 
+        public Builder setLedgerManagerFactory(LedgerManagerFactory ledgerManagerFactory) {
+            this.ledgerManagerFactory = ledgerManagerFactory;
+            return this;
+        }
+
         public BKHttpServiceProvider build()
             throws IOException, KeeperException, InterruptedException, BKException {
             return new BKHttpServiceProvider(
                 bookieServer,
                 autoRecovery,
+                ledgerManagerFactory,
                 serverConf,
                 statsProvider
             );
@@ -185,9 +196,9 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
             case DELETE_LEDGER:
                 return new DeleteLedgerService(configuration);
             case LIST_LEDGER:
-                return new ListLedgerService(configuration, bookieServer);
+                return new ListLedgerService(configuration, ledgerManagerFactory);
             case GET_LEDGER_META:
-                return new GetLedgerMetaService(configuration, bookieServer);
+                return new GetLedgerMetaService(configuration, ledgerManagerFactory);
             case READ_LEDGER_ENTRY:
                 return new ReadLedgerEntryService(configuration, bka);
 
@@ -219,7 +230,7 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
             case RECOVERY_BOOKIE:
                 return new RecoveryBookieService(configuration, bka, executor);
             case LIST_UNDER_REPLICATED_LEDGER:
-                return new ListUnderReplicatedLedgerService(configuration, bookieServer);
+                return new ListUnderReplicatedLedgerService(configuration, ledgerManagerFactory);
             case WHO_IS_AUDITOR:
                 return new WhoIsAuditorService(configuration, bka);
             case TRIGGER_AUDIT:
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ExpandStorageService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ExpandStorageService.java
index 3109bce..9f7a0a4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ExpandStorageService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ExpandStorageService.java
@@ -26,7 +26,9 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.http.HttpServer;
 import org.apache.bookkeeper.http.service.HttpEndpointService;
 import org.apache.bookkeeper.http.service.HttpServiceRequest;
@@ -79,11 +81,15 @@ public class ExpandStorageService implements HttpEndpointService {
             }
 
             try (MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(
-                URI.create(conf.getMetadataServiceUri())
-            )) {
-                driver.initialize(conf, () -> { }, NullStatsLogger.INSTANCE);
-                BookieImpl.checkEnvironmentWithStorageExpansion(conf, driver,
-                  Lists.newArrayList(journalDirectories), allLedgerDirs);
+                         URI.create(conf.getMetadataServiceUri()))) {
+                driver.initialize(conf, NullStatsLogger.INSTANCE);
+
+                try (RegistrationManager registrationManager = driver.createRegistrationManager()) {
+                    LegacyCookieValidation validation = new LegacyCookieValidation(conf, registrationManager);
+                    List<File> dirs = Lists.newArrayList(journalDirectories);
+                    dirs.addAll(allLedgerDirs);
+                    validation.checkCookies(dirs);
+                }
             } catch (BookieException e) {
                 LOG.error("Exception occurred while updating cookie for storage expansion", e);
                 response.setCode(HttpServer.StatusCode.INTERNAL_ERROR);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
index 451bf08..253c7fe 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
@@ -32,7 +32,6 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerMetadataSerDe;
-import org.apache.bookkeeper.proto.BookieServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,13 +44,13 @@ public class GetLedgerMetaService implements HttpEndpointService {
     static final Logger LOG = LoggerFactory.getLogger(GetLedgerMetaService.class);
 
     protected ServerConfiguration conf;
-    protected BookieServer bookieServer;
+    private final LedgerManagerFactory ledgerManagerFactory;
     private final LedgerMetadataSerDe serDe;
 
-    public GetLedgerMetaService(ServerConfiguration conf, BookieServer bookieServer) {
+    public GetLedgerMetaService(ServerConfiguration conf, LedgerManagerFactory ledgerManagerFactory) {
         checkNotNull(conf);
         this.conf = conf;
-        this.bookieServer = bookieServer;
+        this.ledgerManagerFactory = ledgerManagerFactory;
         this.serDe = new LedgerMetadataSerDe();
     }
 
@@ -63,8 +62,7 @@ public class GetLedgerMetaService implements HttpEndpointService {
         if (HttpServer.Method.GET == request.getMethod() && (params != null) && params.containsKey("ledger_id")) {
             Long ledgerId = Long.parseLong(params.get("ledger_id"));
 
-            LedgerManagerFactory mFactory = bookieServer.getBookie().getLedgerManagerFactory();
-            LedgerManager manager = mFactory.newLedgerManager();
+            LedgerManager manager = ledgerManagerFactory.newLedgerManager();
 
             // output <ledgerId: ledgerMetadata>
             Map<String, Object> output = Maps.newHashMap();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
index 45e692d..6254234 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
@@ -35,7 +35,6 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerMetadataSerDe;
-import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,13 +50,13 @@ public class ListLedgerService implements HttpEndpointService {
     static final Logger LOG = LoggerFactory.getLogger(ListLedgerService.class);
 
     protected ServerConfiguration conf;
-    protected BookieServer bookieServer;
+    protected LedgerManagerFactory ledgerManagerFactory;
     private final LedgerMetadataSerDe serDe;
 
-    public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) {
+    public ListLedgerService(ServerConfiguration conf, LedgerManagerFactory ledgerManagerFactory) {
         checkNotNull(conf);
         this.conf = conf;
-        this.bookieServer = bookieServer;
+        this.ledgerManagerFactory = ledgerManagerFactory;
         this.serDe = new LedgerMetadataSerDe();
 
     }
@@ -96,8 +95,7 @@ public class ListLedgerService implements HttpEndpointService {
             int pageIndex = (printMeta && params.containsKey("page"))
                 ? Integer.parseInt(params.get("page")) : -1;
 
-            LedgerManagerFactory mFactory = bookieServer.getBookie().getLedgerManagerFactory();
-            LedgerManager manager = mFactory.newLedgerManager();
+            LedgerManager manager = ledgerManagerFactory.newLedgerManager();
             LedgerManager.LedgerRangeIterator iter = manager.getLedgerRanges(0);
 
             // output <ledgerId: ledgerMetadata>
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListUnderReplicatedLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListUnderReplicatedLedgerService.java
index ef71a68..496bf0b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListUnderReplicatedLedgerService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListUnderReplicatedLedgerService.java
@@ -35,7 +35,6 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.meta.UnderreplicatedLedger;
-import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,12 +50,12 @@ public class ListUnderReplicatedLedgerService implements HttpEndpointService {
     static final Logger LOG = LoggerFactory.getLogger(ListUnderReplicatedLedgerService.class);
 
     protected ServerConfiguration conf;
-    protected BookieServer bookieServer;
+    private final LedgerManagerFactory ledgerManagerFactory;
 
-    public ListUnderReplicatedLedgerService(ServerConfiguration conf, BookieServer bookieServer) {
+    public ListUnderReplicatedLedgerService(ServerConfiguration conf, LedgerManagerFactory ledgerManagerFactory) {
         checkNotNull(conf);
         this.conf = conf;
-        this.bookieServer = bookieServer;
+        this.ledgerManagerFactory = ledgerManagerFactory;
     }
 
     /*
@@ -100,8 +99,8 @@ public class ListUnderReplicatedLedgerService implements HttpEndpointService {
                 boolean hasURLedgers = false;
                 List<Long> outputLedgers = null;
                 Map<Long, List<String>> outputLedgersWithMissingReplica = null;
-                LedgerManagerFactory mFactory = bookieServer.getBookie().getLedgerManagerFactory();
-                LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager();
+                LedgerUnderreplicationManager underreplicationManager =
+                    ledgerManagerFactory.newLedgerUnderreplicationManager();
                 Iterator<UnderreplicatedLedger> iter = underreplicationManager.listLedgersToRereplicate(predicate);
 
                 hasURLedgers = iter.hasNext();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
index 530dd0c..641bf6f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
@@ -23,39 +23,56 @@ import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.function.Supplier;
-import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher.EndpointInfo;
-import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.stats.StatsLogger;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * A {@link ServerLifecycleComponent} that starts the core bookie server.
  */
-@Slf4j
-public class BookieService extends ServerLifecycleComponent {
 
+public class BookieService extends ServerLifecycleComponent {
+    private static final Logger log = LoggerFactory.getLogger(BookieService.class);
     public static final String NAME = "bookie-server";
 
     private final BookieServer server;
+    private final ByteBufAllocatorWithOomHandler allocator;
 
     public BookieService(BookieConfiguration conf,
+                         Bookie bookie,
                          StatsLogger statsLogger,
-                         Supplier<BookieServiceInfo> bookieServiceInfoProvider)
+                         ByteBufAllocatorWithOomHandler allocator)
             throws Exception {
         super(NAME, conf, statsLogger);
-        this.server = new BookieServer(conf.getServerConf(), statsLogger, bookieServiceInfoProvider);
+        this.server = new BookieServer(conf.getServerConf(),
+                                       bookie,
+                                       statsLogger,
+                                       allocator);
+        this.allocator = allocator;
     }
 
     @Override
     public void setExceptionHandler(UncaughtExceptionHandler handler) {
         super.setExceptionHandler(handler);
         server.setExceptionHandler(handler);
+        allocator.setOomHandler((ex) -> {
+                try {
+                    log.error("Unable to allocate memory, exiting bookie", ex);
+                } finally {
+                    if (uncaughtExceptionHandler != null) {
+                        uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), ex);
+                    }
+                }
+            });
     }
 
     public BookieServer getServer() {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java
index 8b1183c..1f33152 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java
@@ -117,10 +117,14 @@ public class ConvertToInterleavedStorageCommand extends BookieCommand<ConvertToI
             }
         };
 
-        dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, null,
-            checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+        dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager,
+                             NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+        dbStorage.setCheckpointSource(checkpointSource);
+        dbStorage.setCheckpointer(checkpointer);
         interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager,
-            null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                                      NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+        interleavedStorage.setCheckpointSource(checkpointSource);
+        interleavedStorage.setCheckpointer(checkpointer);
         LedgerCache interleavedLedgerCache = interleavedStorage.getLedgerCache();
 
         int convertedLedgers = 0;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
index 5f6280f..fe98fda 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
@@ -18,8 +18,9 @@
  */
 package org.apache.bookkeeper.tools.cli.commands.bookie;
 
+import com.google.common.util.concurrent.UncheckedExecutionException;
 import java.io.File;
-
+import java.io.IOException;
 import org.apache.bookkeeper.bookie.Journal;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.bookie.LogMark;
@@ -50,18 +51,22 @@ public class LastMarkCommand extends BookieCommand<CliFlags> {
 
     @Override
     public boolean apply(ServerConfiguration conf, CliFlags flags) {
-        LedgerDirsManager dirsManager = new LedgerDirsManager(
-            conf, conf.getLedgerDirs(),
-            new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
-        File[] journalDirs = conf.getJournalDirs();
+        try {
+            LedgerDirsManager dirsManager = new LedgerDirsManager(
+                    conf, conf.getLedgerDirs(),
+                    new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
+            File[] journalDirs = conf.getJournalDirs();
 
-        for (int idx = 0; idx < journalDirs.length; idx++) {
-            Journal journal = new Journal(idx, journalDirs[idx], conf, dirsManager);
-            LogMark lastLogMark = journal.getLastLogMark().getCurMark();
-            LOG.info("LastLogMark : Journal Id - " + lastLogMark.getLogFileId() + "("
-                + Long.toHexString(lastLogMark.getLogFileId()) + ".txn), Pos - "
-                + lastLogMark.getLogFileOffset());
+            for (int idx = 0; idx < journalDirs.length; idx++) {
+                Journal journal = new Journal(idx, journalDirs[idx], conf, dirsManager);
+                LogMark lastLogMark = journal.getLastLogMark().getCurMark();
+                LOG.info("LastLogMark : Journal Id - " + lastLogMark.getLogFileId() + "("
+                                   + Long.toHexString(lastLogMark.getLogFileId()) + ".txn), Pos - "
+                                   + lastLogMark.getLogFileOffset());
+            }
+            return true;
+        } catch (IOException e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
         }
-        return true;
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java
index b2c63f4..c7c1a6d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java
@@ -174,7 +174,7 @@ public class ReadJournalCommand extends BookieCommand<ReadJournalCommand.ReadJou
         return true;
     }
 
-    private synchronized List<Journal> getJournals(ServerConfiguration conf) {
+    private synchronized List<Journal> getJournals(ServerConfiguration conf) throws IOException {
         if (null == journals) {
             journals = Lists.newArrayListWithCapacity(conf.getJournalDirs().length);
             int idx = 0;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java
index ca039a4..b2fcf0c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java
@@ -33,7 +33,9 @@ import lombok.experimental.Accessors;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
 import org.apache.bookkeeper.tools.framework.CliFlags;
@@ -234,9 +236,12 @@ public class AdminCommand extends BookieCommand<AdminCommand.AdminFlags> {
                 allLedgerDirs.addAll(Arrays.asList(indexDirectories));
             }
 
-            try {
-                BookieImpl.checkEnvironmentWithStorageExpansion(bkConf, driver, Arrays.asList(journalDirectories),
-                                                                allLedgerDirs);
+            try (RegistrationManager registrationManager = driver.createRegistrationManager()) {
+                LegacyCookieValidation validation = new LegacyCookieValidation(bkConf, registrationManager);
+                List<File> dirs = Lists.newArrayList();
+                dirs.addAll(Arrays.asList(journalDirectories));
+                dirs.addAll(allLedgerDirs);
+                validation.checkCookies(dirs);
                 return true;
             } catch (BookieException e) {
                 LOG.error("Exception while updating cookie for storage expansion", e);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
index 67cf0fa..2384999 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
@@ -33,18 +33,25 @@ import java.net.Socket;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
+
+import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.bookkeeper.server.conf.BookieConfiguration;
-import org.apache.bookkeeper.server.service.BookieService;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShimFactory;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -60,7 +67,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Local Bookkeeper.
  */
-public class LocalBookKeeper {
+public class LocalBookKeeper implements AutoCloseable {
     protected static final Logger LOG = LoggerFactory.getLogger(LocalBookKeeper.class);
     public static final int CONNECTION_TIMEOUT = 30000;
 
@@ -70,23 +77,30 @@ public class LocalBookKeeper {
 
     int numberOfBookies;
 
-    public LocalBookKeeper() {
-        this(3);
-    }
-
-    public LocalBookKeeper(int numberOfBookies) {
-        this(numberOfBookies, 5000, new ServerConfiguration(), defaultLocalBookiesConfigDir);
-    }
+//    public LocalBookKeeper() {
+//        this(3);
+//    }
+//
+//    public LocalBookKeeper(int numberOfBookies) {
+//        this(numberOfBookies, new ServerConfiguration(), defaultLocalBookiesConfigDir, true,
+//                "test", );
+//    }
 
     public LocalBookKeeper(
             int numberOfBookies,
-            int initialPort,
             ServerConfiguration baseConf,
-            String localBookiesConfigDirName) {
+            String localBookiesConfigDirName,
+            boolean stopOnExit, String dirSuffix,
+            String zkHost, int zkPort) {
         this.numberOfBookies = numberOfBookies;
-        this.initialPort = initialPort;
         this.localBookiesConfigDir = new File(localBookiesConfigDirName);
         this.baseConf = baseConf;
+        this.localBookies = new ArrayList<>();
+        this.stopOnExit = stopOnExit;
+        this.dirSuffix = dirSuffix;
+        this.zkHost = zkHost;
+        this.zkPort = zkPort;
+        this.dirsToCleanUp = new ArrayList<>();
         LOG.info("Running {} bookie(s) on zk ensemble = '{}:{}'.", this.numberOfBookies,
                 zooKeeperDefaultHost, zooKeeperDefaultPort);
     }
@@ -94,17 +108,20 @@ public class LocalBookKeeper {
     private static String zooKeeperDefaultHost = "127.0.0.1";
     private static int zooKeeperDefaultPort = 2181;
     private static int zkSessionTimeOut = 5000;
-    private static Integer bookieDefaultInitialPort = 5000;
     private static String defaultLocalBookiesConfigDir = "/tmp/localbookies-config";
 
     //BookKeeper variables
-    File[] journalDirs;
-    BookieServer[] bs;
-    ServerConfiguration[] bsConfs;
-    Integer initialPort = 5000;
+    List<LocalBookie> localBookies;
+    ZooKeeperServerShim zks;
+    String zkHost;
+    int zkPort;
+    String dirSuffix;
+    ByteBufAllocatorWithOomHandler allocator;
     private ServerConfiguration baseConf;
-
     File localBookiesConfigDir;
+    List<File> dirsToCleanUp;
+    boolean stopOnExit;
+
     /**
      * @param maxCC
      *          Max Concurrency of Client
@@ -130,7 +147,7 @@ public class LocalBookKeeper {
     }
 
     @SuppressWarnings("deprecation")
-    private void initializeZookeeper(String zkHost, int zkPort) throws IOException {
+    private void initializeZookeeper() throws IOException {
         LOG.info("Instantiate ZK Client");
         //initialize the zk client with values
         try (ZooKeeperClient zkc = ZooKeeperClient.newBuilder()
@@ -166,28 +183,12 @@ public class LocalBookKeeper {
         }
     }
 
-    private List<File> runBookies(String dirSuffix)
-            throws Exception {
-        List<File> tempDirs = new ArrayList<>();
-        try {
-            runBookies(tempDirs, dirSuffix);
-            return tempDirs;
-        } catch (Exception ioe) {
-            cleanupDirectories(tempDirs);
-            throw ioe;
-        }
-    }
-
     @SuppressWarnings("deprecation")
-    private void runBookies(List<File> tempDirs, String dirSuffix)
+    private void runBookies()
             throws Exception {
         LOG.info("Starting Bookie(s)");
         // Create Bookie Servers (B1, B2, B3)
 
-        journalDirs = new File[numberOfBookies];
-        bs = new BookieServer[numberOfBookies];
-        bsConfs = new ServerConfiguration[numberOfBookies];
-
         if (localBookiesConfigDir.exists() && localBookiesConfigDir.isFile()) {
             throw new IOException("Unable to create LocalBookiesConfigDir, since there is a file at "
                     + localBookiesConfigDir.getAbsolutePath());
@@ -196,79 +197,9 @@ public class LocalBookKeeper {
             throw new IOException(
                     "Unable to create LocalBookiesConfigDir - " + localBookiesConfigDir.getAbsolutePath());
         }
-
+        allocator = BookieResources.createAllocator(baseConf);
         for (int i = 0; i < numberOfBookies; i++) {
-            if (null == baseConf.getJournalDirNameWithoutDefault()) {
-                journalDirs[i] = IOUtils.createTempDir("localbookkeeper" + Integer.toString(i), dirSuffix);
-                tempDirs.add(journalDirs[i]);
-            } else {
-                journalDirs[i] = new File(baseConf.getJournalDirName(), "bookie" + Integer.toString(i));
-            }
-            if (journalDirs[i].exists()) {
-                if (journalDirs[i].isDirectory()) {
-                    FileUtils.deleteDirectory(journalDirs[i]);
-                } else if (!journalDirs[i].delete()) {
-                    throw new IOException("Couldn't cleanup bookie journal dir " + journalDirs[i]);
-                }
-            }
-            if (!journalDirs[i].mkdirs()) {
-                throw new IOException("Couldn't create bookie journal dir " + journalDirs[i]);
-            }
-
-            String [] ledgerDirs = baseConf.getLedgerDirWithoutDefault();
-            if ((null == ledgerDirs) || (0 == ledgerDirs.length)) {
-                ledgerDirs = new String[] { journalDirs[i].getPath() };
-            } else {
-                for (int l = 0; l < ledgerDirs.length; l++) {
-                    File dir = new File(ledgerDirs[l], "bookie" + Integer.toString(i));
-                    if (dir.exists()) {
-                        if (dir.isDirectory()) {
-                            FileUtils.deleteDirectory(dir);
-                        } else if (!dir.delete()) {
-                            throw new IOException("Couldn't cleanup bookie ledger dir " + dir);
-                        }
-                    }
-                    if (!dir.mkdirs()) {
-                        throw new IOException("Couldn't create bookie ledger dir " + dir);
-                    }
-                    ledgerDirs[l] = dir.getPath();
-                }
-            }
-
-            bsConfs[i] = new ServerConfiguration((ServerConfiguration) baseConf.clone());
-
-            // If the caller specified ephemeral ports then use ephemeral ports for all
-            // the bookies else use numBookie ports starting at initialPort
-            PortManager.initPort(initialPort);
-            if (0 == initialPort) {
-                bsConfs[i].setBookiePort(0);
-            } else {
-                bsConfs[i].setBookiePort(PortManager.nextFreePort());
-            }
-
-            if (null == baseConf.getMetadataServiceUriUnchecked()) {
-                bsConfs[i].setMetadataServiceUri(baseConf.getMetadataServiceUri());
-            }
-
-            bsConfs[i].setJournalDirName(journalDirs[i].getPath());
-            bsConfs[i].setLedgerDirNames(ledgerDirs);
-
-            // write config into file before start so we can know what's wrong if start failed
-            String fileName = BookieImpl.getBookieId(bsConfs[i]).toString() + ".conf";
-            serializeLocalBookieConfig(bsConfs[i], fileName);
-
-            // Mimic BookKeeper Main
-            final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher();
-            final Supplier<BookieServiceInfo> bookieServiceInfoProvider =
-                    () -> buildBookieServiceInfo(componentInfoPublisher);
-            BookieService bookieService = new BookieService(new BookieConfiguration(bsConfs[i]),
-                    NullStatsLogger.INSTANCE,
-                    bookieServiceInfoProvider
-            );
-            bs[i] = bookieService.getServer();
-            bookieService.publishInfo(componentInfoPublisher);
-            componentInfoPublisher.startupFinished();
-            bookieService.start();
+            runBookie(i);
         }
 
         /*
@@ -290,115 +221,114 @@ public class LocalBookKeeper {
         serializeLocalBookieConfig(baseConfWithCorrectZKServers, "baseconf.conf");
     }
 
-    public static void startLocalBookies(String zkHost,
-                                         int zkPort,
-                                         int numBookies,
-                                         boolean shouldStartZK,
-                                         int initialBookiePort)
-            throws Exception {
-        ServerConfiguration conf = new ServerConfiguration();
-        startLocalBookiesInternal(
-                conf, zkHost, zkPort, numBookies, shouldStartZK,
-                initialBookiePort, true, "test", null, defaultLocalBookiesConfigDir);
+    private void runBookie(int bookieIndex) throws Exception {
+        File journalDirs;
+        if (null == baseConf.getJournalDirNameWithoutDefault()) {
+            journalDirs = IOUtils.createTempDir("localbookkeeper" + bookieIndex, dirSuffix);
+            dirsToCleanUp.add(journalDirs);
+        } else {
+            journalDirs = new File(baseConf.getJournalDirName(), "bookie" + bookieIndex);
+        }
+        if (journalDirs.exists()) {
+            if (journalDirs.isDirectory()) {
+                FileUtils.deleteDirectory(journalDirs);
+            } else if (!journalDirs.delete()) {
+                throw new IOException("Couldn't cleanup bookie journal dir " + journalDirs);
+            }
+        }
+        if (!journalDirs.mkdirs()) {
+            throw new IOException("Couldn't create bookie journal dir " + journalDirs);
+        }
+
+        String[] ledgerDirs = baseConf.getLedgerDirWithoutDefault();
+        if ((null == ledgerDirs) || (0 == ledgerDirs.length)) {
+            ledgerDirs = new String[]{journalDirs.getPath()};
+        } else {
+            for (int l = 0; l < ledgerDirs.length; l++) {
+                File dir = new File(ledgerDirs[l], "bookie" + bookieIndex);
+                if (dir.exists()) {
+                    if (dir.isDirectory()) {
+                        FileUtils.deleteDirectory(dir);
+                    } else if (!dir.delete()) {
+                        throw new IOException("Couldn't cleanup bookie ledger dir " + dir);
+                    }
+                }
+                if (!dir.mkdirs()) {
+                    throw new IOException("Couldn't create bookie ledger dir " + dir);
+                }
+                dirsToCleanUp.add(dir);
+                ledgerDirs[l] = dir.getPath();
+            }
+        }
+        ServerConfiguration conf = new ServerConfiguration((ServerConfiguration) baseConf.clone());
+
+        conf.setBookiePort(PortManager.nextFreePort());
+
+        if (null == baseConf.getMetadataServiceUriUnchecked()) {
+            conf.setMetadataServiceUri(baseConf.getMetadataServiceUri());
+        }
+
+        conf.setJournalDirName(journalDirs.getPath());
+        conf.setLedgerDirNames(ledgerDirs);
+
+        // write config into file before start so we can know what's wrong if start failed
+        String fileName = BookieImpl.getBookieId(conf).toString() + ".conf";
+        serializeLocalBookieConfig(conf, fileName);
+
+        LocalBookie b = new LocalBookie(conf);
+        b.start();
+        localBookies.add(b);
     }
 
-    public static void startLocalBookies(String zkHost,
-                                         int zkPort,
-                                         int numBookies,
-                                         boolean shouldStartZK,
-                                         int initialBookiePort,
-                                         ServerConfiguration conf)
-            throws Exception {
-        startLocalBookiesInternal(
-                conf, zkHost, zkPort, numBookies, shouldStartZK,
-                initialBookiePort, true, "test", null, defaultLocalBookiesConfigDir);
+    private void setZooKeeperShim(ZooKeeperServerShim zks, File zkTmpDir) {
+        this.zks = zks;
+        this.dirsToCleanUp.add(zkTmpDir);
     }
 
-    public static void startLocalBookies(String zkHost,
-                                         int zkPort,
-                                         int numBookies,
-                                         boolean shouldStartZK,
-                                         int initialBookiePort,
-                                         String dirSuffix)
-            throws Exception {
-        ServerConfiguration conf = new ServerConfiguration();
-        startLocalBookiesInternal(
+    public static LocalBookKeeper getLocalBookies(String zkHost,
+                                                     int zkPort,
+                                                     int numBookies,
+                                                     boolean shouldStartZK,
+                                                     ServerConfiguration conf) throws Exception {
+        return getLocalBookiesInternal(
                 conf, zkHost, zkPort, numBookies, shouldStartZK,
-                initialBookiePort, true, dirSuffix, null, defaultLocalBookiesConfigDir);
+                true, "test", null, defaultLocalBookiesConfigDir);
     }
 
-    @SuppressWarnings("deprecation")
-    static void startLocalBookiesInternal(ServerConfiguration conf,
-                                          String zkHost,
-                                          int zkPort,
-                                          int numBookies,
-                                          boolean shouldStartZK,
-                                          int initialBookiePort,
-                                          boolean stopOnExit,
-                                          String dirSuffix,
-                                          String zkDataDir,
-                                          String localBookiesConfigDirName)
-            throws Exception {
+    private static LocalBookKeeper getLocalBookiesInternal(ServerConfiguration conf,
+                                                             String zkHost,
+                                                             int zkPort,
+                                                             int numBookies,
+                                                             boolean shouldStartZK,
+                                                             boolean stopOnExit,
+                                                             String dirSuffix,
+                                                             String zkDataDir,
+                                                             String localBookiesConfigDirName) throws Exception {
         conf.setMetadataServiceUri(
                 newMetadataServiceUri(
                         zkHost,
                         zkPort,
                         conf.getLedgerManagerLayoutStringFromFactoryClass(),
                         conf.getZkLedgersRootPath()));
-        LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, conf, localBookiesConfigDirName);
-        ZooKeeperServerShim zks = null;
-        File zkTmpDir = null;
-        List<File> bkTmpDirs = null;
-        try {
-            if (shouldStartZK) {
-                File zkDataDirFile = null;
-                if (zkDataDir != null) {
-                    zkDataDirFile = new File(zkDataDir);
-                    if (zkDataDirFile.exists() && zkDataDirFile.isFile()) {
-                        throw new IOException("Unable to create zkDataDir, since there is a file at "
-                                + zkDataDirFile.getAbsolutePath());
-                    }
-                    if (!zkDataDirFile.exists() && !zkDataDirFile.mkdirs()) {
-                        throw new IOException("Unable to create zkDataDir - " + zkDataDirFile.getAbsolutePath());
-                    }
-                }
-                zkTmpDir = IOUtils.createTempDir("zookeeper", dirSuffix, zkDataDirFile);
-                zkTmpDir.deleteOnExit();
-                zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
-            }
-
-            lb.initializeZookeeper(zkHost, zkPort);
-            bkTmpDirs = lb.runBookies(dirSuffix);
-
-            try {
-                while (true) {
-                    Thread.sleep(5000);
+        LocalBookKeeper lb = new LocalBookKeeper(numBookies, conf, localBookiesConfigDirName, stopOnExit,
+                dirSuffix, zkHost, zkPort);
+        if (shouldStartZK) {
+            File zkDataDirFile = null;
+            if (zkDataDir != null) {
+                zkDataDirFile = new File(zkDataDir);
+                if (zkDataDirFile.exists() && zkDataDirFile.isFile()) {
+                    throw new IOException("Unable to create zkDataDir, since there is a file at "
+                            + zkDataDirFile.getAbsolutePath());
                 }
-            } catch (InterruptedException ie) {
-                Thread.currentThread().interrupt();
-                if (stopOnExit) {
-                    lb.shutdownBookies();
-
-                    if (null != zks) {
-                        zks.stop();
-                    }
-                }
-                throw ie;
-            }
-        } catch (Exception e) {
-            LOG.error("Failed to run {} bookies : zk ensemble = '{}:{}'",
-                    numBookies, zkHost, zkPort, e);
-            throw e;
-        } finally {
-            if (stopOnExit) {
-                if (null != bkTmpDirs) {
-                    cleanupDirectories(bkTmpDirs);
-                }
-                if (null != zkTmpDir) {
-                    FileUtils.deleteDirectory(zkTmpDir);
+                if (!zkDataDirFile.exists() && !zkDataDirFile.mkdirs()) {
+                    throw new IOException("Unable to create zkDataDir - " + zkDataDirFile.getAbsolutePath());
                 }
             }
+            File zkTmpDir = IOUtils.createTempDir("zookeeper", dirSuffix, zkDataDirFile);
+            lb.setZooKeeperShim(LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir), zkTmpDir);
         }
+
+        return lb;
     }
 
     /**
@@ -472,8 +402,18 @@ public class LocalBookKeeper {
                 localBookiesConfigDirName = args[3];
             }
 
-            startLocalBookiesInternal(conf, zooKeeperDefaultHost, zooKeeperDefaultPort, numBookies, true,
-                    bookieDefaultInitialPort, false, "test", zkDataDir, localBookiesConfigDirName);
+            try (LocalBookKeeper lb = getLocalBookiesInternal(conf, zooKeeperDefaultHost, zooKeeperDefaultPort,
+                    numBookies, true, false, "test", zkDataDir,
+                    localBookiesConfigDirName)) {
+                try {
+                    while (true) {
+                        Thread.sleep(1000);
+                    }
+                } catch (InterruptedException ie) {
+                    Thread.currentThread().interrupt();
+                    throw ie;
+                }
+            }
         } catch (Exception e) {
             LOG.error("Exiting LocalBookKeeper because of exception in main method", e);
             /*
@@ -526,9 +466,86 @@ public class LocalBookKeeper {
         return false;
     }
 
-    public void shutdownBookies() {
-        for (BookieServer bookieServer: bs) {
-            bookieServer.shutdown();
+    public void start() throws Exception {
+        initializeZookeeper();
+        runBookies();
+    }
+
+    public void addBookie() throws Exception {
+        int bookieIndex = localBookies.size() + 1;
+        runBookie(bookieIndex);
+    }
+
+    public void removeBookie() throws Exception {
+        int index = localBookies.size() - 1;
+        LocalBookie bookie = localBookies.get(index);
+        bookie.shutdown();
+        localBookies.remove(index);
+    }
+
+    public void shutdownBookies() throws Exception {
+        for (LocalBookie b : localBookies) {
+            b.shutdown();
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (stopOnExit) {
+            shutdownBookies();
+
+            if (null != zks) {
+                zks.stop();
+            }
+        }
+
+        cleanupDirectories(dirsToCleanUp);
+    }
+
+    private class LocalBookie {
+        final BookieServer server;
+        final Bookie bookie;
+        final MetadataBookieDriver metadataDriver;
+        final RegistrationManager registrationManager;
+        final LedgerManagerFactory lmFactory;
+        final LedgerManager ledgerManager;
+
+        LocalBookie(ServerConfiguration conf) throws Exception {
+            metadataDriver = BookieResources.createMetadataDriver(conf, NullStatsLogger.INSTANCE);
+            registrationManager = metadataDriver.createRegistrationManager();
+            lmFactory = metadataDriver.getLedgerManagerFactory();
+            ledgerManager = lmFactory.newLedgerManager();
+
+            DiskChecker diskChecker = BookieResources.createDiskChecker(conf);
+            LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
+                    conf, diskChecker, NullStatsLogger.INSTANCE);
+            LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
+                    conf, diskChecker, NullStatsLogger.INSTANCE, ledgerDirsManager);
+            LedgerStorage storage = BookieResources.createLedgerStorage(
+                    conf, ledgerManager, ledgerDirsManager, indexDirsManager,
+                    NullStatsLogger.INSTANCE, allocator);
+
+            final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher();
+            final Supplier<BookieServiceInfo> bookieServiceInfoProvider =
+                    () -> buildBookieServiceInfo(componentInfoPublisher);
+
+            componentInfoPublisher.startupFinished();
+            bookie = new BookieImpl(conf, registrationManager, storage, diskChecker,
+                    ledgerDirsManager, indexDirsManager,
+                    NullStatsLogger.INSTANCE, allocator, bookieServiceInfoProvider);
+            server = new BookieServer(conf, bookie, NullStatsLogger.INSTANCE, allocator);
+        }
+
+        void start() throws Exception {
+            server.start();
+        }
+
+        void shutdown() throws Exception {
+            server.shutdown();
+            ledgerManager.close();
+            lmFactory.close();
+            registrationManager.close();
+            metadataDriver.close();
         }
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java
index 864aaf2..cf17533 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java
@@ -20,55 +20,30 @@
  */
 package org.apache.bookkeeper.util;
 
-import java.io.IOException;
 import java.net.ServerSocket;
-import java.util.concurrent.TimeUnit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * Port manager allows a base port to be specified on the commandline.
- * Tests will then use ports, counting up from this base port.
- * This allows multiple instances of the bookkeeper tests to run at once.
+ * Port manager provides free ports to allows multiple instances
+ * of the bookkeeper tests to run at once.
  */
 public class PortManager {
 
-    private static final Logger LOG = LoggerFactory.getLogger(PortManager.class);
-
-    private static int nextPort = 15000;
-
-    /**
-     * Init the base port.
-     *
-     * @param initPort initial port
-     */
-    public static void initPort(int initPort) {
-        nextPort = initPort;
-    }
-
     /**
-     * Return the available port.
+     * Return an available port.
      *
      * @return available port.
      */
     public static synchronized int nextFreePort() {
         int exceptionCount = 0;
         while (true) {
-            int port = nextPort++;
-            try (ServerSocket ignored = new ServerSocket(port)) {
-                // Give it some time to truly close the connection
-                TimeUnit.MILLISECONDS.sleep(100);
-                return port;
-            } catch (IOException ioe) {
+            try (ServerSocket ss = new ServerSocket(0)) {
+                return ss.getLocalPort();
+            } catch (Exception e) {
                 exceptionCount++;
                 if (exceptionCount > 100) {
-                    throw new RuntimeException("Unable to allocate socket port", ioe);
+                    throw new RuntimeException("Unable to allocate socket port", e);
                 }
-            } catch (InterruptedException ie) {
-                LOG.error("Failed to allocate socket port", ie);
-                Thread.currentThread().interrupt();
             }
         }
     }
-
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java
index c5affe8..5efe0af 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java
@@ -23,7 +23,6 @@ package org.apache.bookkeeper.bookie;
 
 import static org.junit.Assert.assertEquals;
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 import java.util.UUID;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
@@ -45,8 +44,8 @@ public class AdvertisedAddressTest extends BookKeeperClusterTestCase {
         super(0);
     }
 
-    private String newDirectory(boolean createCurDir) throws IOException {
-        File d = createTempDir("cookie", "tmpdir");
+    private String newDirectory(boolean createCurDir) throws Exception {
+        File d = tmpDirs.createNew("cookie", "tmpdir");
         if (createCurDir) {
             new File(d, "current").mkdirs();
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
index 3b0fc7a..e94027b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
@@ -26,26 +26,24 @@ import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistra
 import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE;
 import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME;
 import static org.apache.bookkeeper.util.TestUtils.countNumOfFiles;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.hasProperty;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
@@ -64,12 +62,10 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Supplier;
-
 import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException;
 import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException;
 import org.apache.bookkeeper.bookie.Journal.LastLogMark;
@@ -98,18 +94,14 @@ import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat;
 import org.apache.bookkeeper.replication.AutoRecoveryMain;
-import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException;
-import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.replication.ReplicationStats;
 import org.apache.bookkeeper.server.Main;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.server.service.AutoRecoveryService;
 import org.apache.bookkeeper.server.service.BookieService;
 import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-import org.apache.bookkeeper.tls.SecurityException;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.LoggerOutput;
 import org.apache.bookkeeper.util.PortManager;
@@ -118,11 +110,9 @@ import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
-import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import org.powermock.reflect.Whitebox;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.LoggingEvent;
@@ -140,7 +130,6 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     public final TestName runtime = new TestName();
     @Rule
     public LoggerOutput loggerOutput = new LoggerOutput();
-    ZKMetadataBookieDriver driver;
 
     public BookieInitializationTest() {
         super(0);
@@ -151,14 +140,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         String ledgersPath = "/ledgers" + runtime.getMethodName();
         super.setUp(ledgersPath);
         zkUtil.createBKEnsemble(ledgersPath);
-        driver = new ZKMetadataBookieDriver();
     }
 
     @Override
     public void tearDown() throws Exception {
-        if (driver != null) {
-            driver.close();
-        }
         super.tearDown();
     }
 
@@ -176,8 +161,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      * Tests that journal replay works correctly when bookie crashes and starts up in RO mode.
      */
     private void testJournalReplayForBookieRestartInReadOnlyMode(int numOfJournalDirs) throws Exception {
-        File tmpLedgerDir = createTempDir("DiskCheck", "test");
-        File tmpJournalDir = createTempDir("DiskCheck", "test");
+        File tmpLedgerDir = tmpDirs.createNew("DiskCheck", "test");
+        File tmpJournalDir = tmpDirs.createNew("DiskCheck", "test");
 
         String[] journalDirs = new String[numOfJournalDirs];
         for (int i = 0; i < numOfJournalDirs; i++) {
@@ -235,8 +220,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setDiskUsageThreshold(0.001f)
                 .setDiskUsageWarnThreshold(0.0f).setReadOnlyModeEnabled(true).setIsForceGCAllowWhenNoSpace(true)
                 .setMinUsableSizeForIndexFileCreation(5 * 1024);
-
-        server = new BookieServer(conf);
+        server = new BookieServer(
+                conf,
+                TestBookieImpl.buildReadOnly(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
 
         for (int i = 0; i < journalDirs.length; i++) {
             Journal journal = ((BookieImpl) server.getBookie()).journals.get(i);
@@ -277,7 +264,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 assertTrue((countNumOfFiles(conf.getLedgerDirs(), "idx") - idxBefore) <= 0);
             }
 
-            server = new BookieServer(conf);
+            server = new BookieServer(
+                    conf,
+                    TestBookieImpl.buildReadOnly(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         }
         bkClient.close();
     }
@@ -288,31 +278,27 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testExitCodeZK_REG_FAIL() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(tmpDir.getPath())
             .setLedgerDirNames(new String[] { tmpDir.getPath() })
             .setMetadataServiceUri(metadataServiceUri);
 
-        RegistrationManager rm = mock(RegistrationManager.class);
+
+        // simulating ZooKeeper exception by assigning a closed zk client to bk
+        MetadataBookieDriver metadataDriver = spy(BookieResources.createMetadataDriver(conf, NullStatsLogger.INSTANCE));
+        RegistrationManager rm = spy(metadataDriver.createRegistrationManager());
         doThrow(new MetadataStoreException("mocked exception"))
             .when(rm)
             .registerBookie(any(BookieId.class), anyBoolean(), any(BookieServiceInfo.class));
-
-        // simulating ZooKeeper exception by assigning a closed zk client to bk
-        BookieServer bkServer = new BookieServer(conf) {
-            @Override
-            protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator,
-                     Supplier<BookieServiceInfo> bookieServiceInfoProvider)
-                    throws IOException, KeeperException, InterruptedException,
-                    BookieException {
-                Bookie bookie = new TestBookieImpl(conf);
-                MetadataBookieDriver driver = Whitebox.getInternalState(bookie, "metadataDriver");
-                ((ZKMetadataBookieDriver) driver).setRegManager(rm);
-                return bookie;
-            }
-        };
+        doReturn(rm)
+            .when(metadataDriver).createRegistrationManager();
+        TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+            .withMetadataDriver(metadataDriver)
+            .withRegistrationManager(rm).build();
+        BookieServer bkServer = new BookieServer(conf, new TestBookieImpl(resources),
+                                                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
 
         bkServer.start();
         bkServer.join();
@@ -327,19 +313,20 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setListeningInterface(null);
 
         BookieId bookieId = BookieImpl.getBookieId(conf);
-
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-        try (StateManager manager = new BookieStateManager(conf, driver)) {
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf, rm)) {
             manager.registerBookie(true).get();
             assertTrue(
                 "Bookie registration node doesn't exists!",
-                driver.getRegistrationManager().isBookieRegistered(bookieId));
+                rm.isBookieRegistered(bookieId));
 
             // test register bookie again if the registeration node is created by itself.
             manager.registerBookie(true).get();
             assertTrue(
                 "Bookie registration node doesn't exists!",
-                driver.getRegistrationManager().isBookieRegistered(bookieId));
+                rm.isBookieRegistered(bookieId));
         }
     }
 
@@ -357,9 +344,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         String bookieId = BookieImpl.getBookieAddress(conf).toString();
         final String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf)
             + "/" + AVAILABLE_NODE + "/" + bookieId;
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+        metadataDriver.initialize(conf, NullStatsLogger.INSTANCE);
 
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-        try (StateManager manager = new BookieStateManager(conf, driver)) {
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf, rm)) {
             manager.registerBookie(true).get();
         }
         Stat bkRegNode1 = zkc.exists(bkRegPath, false);
@@ -368,7 +358,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // simulating bookie restart, on restart bookie will create new
         // zkclient and doing the registration.
         try (MetadataBookieDriver newDriver = new ZKMetadataBookieDriver()) {
-            newDriver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
+            newDriver.initialize(conf, NullStatsLogger.INSTANCE);
 
             try (ZooKeeperClient newZk = createNewZKClient()) {
                 // deleting the znode, so that the bookie registration should
@@ -382,7 +372,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                         LOG.error("Failed to delete the znode :" + bkRegPath, e);
                     }
                 }).start();
-                try (StateManager newMgr = new BookieStateManager(conf, newDriver)) {
+
+                try (RegistrationManager newRm = newDriver.createRegistrationManager();
+                     StateManager newMgr = new BookieStateManager(conf, newRm)) {
                     newMgr.registerBookie(true).get();
                 } catch (IOException e) {
                     Throwable t = e.getCause();
@@ -415,11 +407,14 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         final BookieId bookieId =
                 BookieId.parse(InetAddress.getLocalHost().getCanonicalHostName() + ":" + conf.getBookiePort());
 
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-        try (StateManager manager = new BookieStateManager(conf, driver)) {
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf, rm)) {
             manager.registerBookie(true).get();
             assertTrue("Bookie registration node doesn't exists!",
-                driver.getRegistrationManager().isBookieRegistered(bookieId));
+                rm.isBookieRegistered(bookieId));
         }
     }
 
@@ -433,11 +428,14 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
         final BookieId bookieId = BookieId.parse(InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0]
             + ":" + conf.getBookiePort());
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-        try (StateManager manager = new BookieStateManager(conf, driver)) {
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf, rm)) {
             manager.registerBookie(true).get();
             assertTrue("Bookie registration node doesn't exists!",
-                driver.getRegistrationManager().isBookieRegistered(bookieId));
+                rm.isBookieRegistered(bookieId));
         }
     }
 
@@ -456,11 +454,14 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 + conf.getBookiePort());
         String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + "/" + AVAILABLE_NODE + "/" + bookieId;
 
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-        try (StateManager manager = new BookieStateManager(conf, driver)) {
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf, rm)) {
             manager.registerBookie(true).get();
             assertTrue("Bookie registration node doesn't exists!",
-                driver.getRegistrationManager().isBookieRegistered(bookieId));
+                rm.isBookieRegistered(bookieId));
         }
         Stat bkRegNode1 = zkc.exists(bkRegPath, false);
         assertNotNull("Bookie registration has been failed",
@@ -469,8 +470,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // simulating bookie restart, on restart bookie will create new
         // zkclient and doing the registration.
         try (MetadataBookieDriver newDriver = new ZKMetadataBookieDriver()) {
-            newDriver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-            try (StateManager newMgr = new BookieStateManager(conf, newDriver)) {
+            newDriver.initialize(conf, NullStatsLogger.INSTANCE);
+
+            try (RegistrationManager newRm = newDriver.createRegistrationManager();
+                 StateManager newMgr = new BookieStateManager(conf, newRm)) {
                 newMgr.registerBookie(true).get();
                 fail("Should throw NodeExistsException as the znode is not getting expired");
             } catch (ExecutionException ee) {
@@ -513,7 +516,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 + ":" + conf.getBookiePort());
         String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + "/" + AVAILABLE_NODE + "/" + bookieId;
 
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
 
         Endpoint endpoint = new Endpoint("test", 1281, "localhost", "bookie-rpc",
                 Collections.emptyList(), Collections.emptyList());
@@ -523,11 +527,13 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
         LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
                 conf, conf.getLedgerDirs(), diskChecker);
-        try (StateManager manager = new BookieStateManager(conf,
-                NullStatsLogger.INSTANCE, driver, ledgerDirsManager, supplier)) {
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager();
+             StateManager manager = new BookieStateManager(conf,
+                NullStatsLogger.INSTANCE,
+                rm, ledgerDirsManager, supplier)) {
             manager.registerBookie(true).get();
             assertTrue("Bookie registration node doesn't exists!",
-                    driver.getRegistrationManager().isBookieRegistered(bookieId));
+                    rm.isBookieRegistered(bookieId));
         }
         Stat bkRegNode = zkc.exists(bkRegPath, false);
         assertNotNull("Bookie registration has been failed", bkRegNode);
@@ -550,7 +556,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testUserNotPermittedToStart() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = PortManager.nextFreePort();
@@ -564,8 +570,11 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
         boolean sawException = false;
         try {
-            bs1 = new BookieServer(conf);
-            Assert.fail("Bookkeeper should not have started since current user isn't in permittedStartupUsers");
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
+            fail("Bookkeeper should not have started since current user isn't in permittedStartupUsers");
         } catch (AccessControlException buae) {
             sawException = true;
         } finally {
@@ -582,7 +591,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testUserPermittedToStart() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = PortManager.nextFreePort();
@@ -597,10 +606,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         String userString = "larry,,,curly ," + System.getProperty("user.name") + " ,moe";
         conf.setPermittedStartupUsers(userString);
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             bs1.start();
         } catch (AccessControlException buae) {
-            Assert.fail("Bookkeeper should have started since current user is in permittedStartupUsers");
+            fail("Bookkeeper should have started since current user is in permittedStartupUsers");
         } finally {
             if (bs1 != null && bs1.isRunning()) {
                 bs1.shutdown();
@@ -611,10 +622,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         userString = "larry ,curly, moe," + System.getProperty("user.name") + ",";
         conf.setPermittedStartupUsers(userString);
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             bs1.start();
         } catch (AccessControlException buae) {
-            Assert.fail("Bookkeeper should have started since current user is in permittedStartupUsers");
+            fail("Bookkeeper should have started since current user is in permittedStartupUsers");
         } finally {
             if (bs1 != null && bs1.isRunning()) {
                 bs1.shutdown();
@@ -628,7 +641,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testUserPermittedToStartWithMissingProperty() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         LOG.info("{}", conf);
@@ -640,10 +653,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(new String[] { tmpDir.getPath() });
         BookieServer bs1 = null;
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             bs1.start();
         } catch (AccessControlException buae) {
-            Assert.fail("Bookkeeper should have started since permittedStartupUser is not specified");
+            fail("Bookkeeper should have started since permittedStartupUser is not specified");
         } finally {
             if (bs1 != null && bs1.isRunning()) {
                 bs1.shutdown();
@@ -657,7 +672,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testDuplicateBookieServerStartup() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = PortManager.nextFreePort();
@@ -665,12 +680,16 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setJournalDirName(tmpDir.getPath())
             .setLedgerDirNames(new String[] { tmpDir.getPath() })
             .setMetadataServiceUri(metadataServiceUri);
-        BookieServer bs1 = new BookieServer(conf);
+        BookieServer bs1 = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bs1.start();
         BookieServer bs2 = null;
         // starting bk server with same conf
         try {
-            bs2 = new BookieServer(conf);
+            bs2 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             bs2.start();
             fail("Should throw BindException, as the bk server is already running!");
         } catch (BindException e) {
@@ -688,7 +707,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
     @Test
     public void testBookieServiceExceptionHandler() throws Exception {
-        File tmpDir = createTempDir("bookie", "exception-handler");
+        File tmpDir = tmpDirs.createNew("bookie", "exception-handler");
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = PortManager.nextFreePort();
         conf.setBookiePort(port)
@@ -697,7 +716,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setMetadataServiceUri(metadataServiceUri);
 
         BookieConfiguration bkConf = new BookieConfiguration(conf);
-        BookieService service = new BookieService(bkConf, NullStatsLogger.INSTANCE, BookieServiceInfo.NO_INFO);
+        BookieService service = new BookieService(
+                bkConf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE,
+                BookieResources.createAllocator(conf));
         CompletableFuture<Void> startFuture = ComponentStarter.startComponent(service);
 
         // shutdown the bookie service
@@ -725,10 +747,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
     @Test
     public void testBookieStartException() throws Exception {
-        File journalDir = createTempDir("bookie", "journal");
+        File journalDir = tmpDirs.createNew("bookie", "journal");
         BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
-        File ledgerDir = createTempDir("bookie", "ledger");
+        File ledgerDir = tmpDirs.createNew("bookie", "ledger");
         BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         /*
@@ -751,19 +773,21 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 .setLedgerStorageClass(MockInterleavedLedgerStorage.class.getName());
 
         BookieConfiguration bkConf = new BookieConfiguration(conf);
-        driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
-
-        /*
-         * create cookie and write it to JournalDir/LedgerDir.
-         */
-        Cookie.Builder cookieBuilder = Cookie.generateCookie(conf);
-        Cookie cookie = cookieBuilder.build();
-        cookie.writeToDirectory(new File(journalDir, "current"));
-        cookie.writeToDirectory(new File(ledgerDir, "current"));
-        Versioned<byte[]> newCookie = new Versioned<>(
-                cookie.toString().getBytes(UTF_8), Version.NEW
-        );
-        driver.getRegistrationManager().writeCookie(BookieImpl.getBookieId(conf), newCookie);
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+        try (RegistrationManager rm = metadataDriver.createRegistrationManager()) {
+            /*
+             * create cookie and write it to JournalDir/LedgerDir.
+             */
+            Cookie.Builder cookieBuilder = Cookie.generateCookie(conf);
+            Cookie cookie = cookieBuilder.build();
+            cookie.writeToDirectory(new File(journalDir, "current"));
+            cookie.writeToDirectory(new File(ledgerDir, "current"));
+            Versioned<byte[]> newCookie = new Versioned<>(
+                    cookie.toString().getBytes(UTF_8), Version.NEW
+            );
+            rm.writeCookie(BookieImpl.getBookieId(conf), newCookie);
+        }
 
         /*
          * Create LifecycleComponent for BookieServer and start it.
@@ -798,10 +822,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testNegativeLengthEntryBookieShutdown() throws Exception {
-        File journalDir = createTempDir("bookie", "journal");
+        File journalDir = tmpDirs.createNew("bookie", "journal");
         BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
-        File ledgerDir = createTempDir("bookie", "ledger");
+        File ledgerDir = tmpDirs.createNew("bookie", "ledger");
         BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         writeV5Journal(BookieImpl.getCurrentDirectory(journalDir), 5,
@@ -846,8 +870,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testBookieServerStartupOnEphemeralPorts() throws Exception {
-        File tmpDir1 = createTempDir("bookie", "test1");
-        File tmpDir2 = createTempDir("bookie", "test2");
+        File tmpDir1 = tmpDirs.createNew("bookie", "test1");
+        File tmpDir2 = tmpDirs.createNew("bookie", "test2");
 
         ServerConfiguration conf1 = TestBKConfiguration.newServerConfiguration();
         conf1.setBookiePort(0)
@@ -856,7 +880,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 new String[] { tmpDir1.getPath() })
             .setMetadataServiceUri(null);
         assertEquals(0, conf1.getBookiePort());
-        BookieServer bs1 = new BookieServer(conf1);
+        BookieServer bs1 = new BookieServer(
+                conf1, new TestBookieImpl(conf1),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bs1.start();
         assertFalse(0 == conf1.getBookiePort());
 
@@ -867,7 +893,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(
                 new String[] { tmpDir2.getPath() })
             .setMetadataServiceUri(null);
-        BookieServer bs2 = new BookieServer(conf2);
+        BookieServer bs2 = new BookieServer(
+                conf2, new TestBookieImpl(conf2),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
         bs2.start();
         assertFalse(0 == conf2.getBookiePort());
 
@@ -882,7 +911,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     public void testStartBookieWithoutZKServer() throws Exception {
         zkUtil.killCluster();
 
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(tmpDir.getPath())
@@ -903,7 +932,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testStartBookieWithoutZKInitialized() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
         final String zkRoot = "/ledgers2";
 
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
@@ -930,7 +959,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testWithDiskFullReadOnlyDisabledOrForceGCAllowDisabled() throws Exception {
-        File tmpDir = createTempDir("DiskCheck", "test");
+        File tmpDir = tmpDirs.createNew("DiskCheck", "test");
         long usableSpace = tmpDir.getUsableSpace();
         long totalSpace = tmpDir.getTotalSpace();
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
@@ -985,7 +1014,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testWithDiskFullReadOnlyEnabledAndForceGCAllowAllowed() throws Exception {
-        File tmpDir = createTempDir("DiskCheck", "test");
+        File tmpDir = tmpDirs.createNew("DiskCheck", "test");
         long usableSpace = tmpDir.getUsableSpace();
         long totalSpace = tmpDir.getTotalSpace();
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
@@ -1013,24 +1042,18 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     class MockBookieServer extends BookieServer {
         ServerConfiguration conf;
 
-        public MockBookieServer(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException,
-                BookieException, UnavailableException, CompatibilityException, SecurityException {
-            super(conf);
+        public MockBookieServer(ServerConfiguration conf) throws Exception {
+            super(conf,
+                  new MockBookieWithNoopShutdown(conf),
+                  NullStatsLogger.INSTANCE,
+                  UnpooledByteBufAllocator.DEFAULT);
             this.conf = conf;
         }
-
-        @Override
-        protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator,
-                     Supplier<BookieServiceInfo> bookieServiceInfoProvider)
-                throws IOException, KeeperException, InterruptedException, BookieException {
-            return new MockBookieWithNoopShutdown(conf, NullStatsLogger.INSTANCE);
-        }
     }
 
-    class MockBookieWithNoopShutdown extends BookieImpl {
-        public MockBookieWithNoopShutdown(ServerConfiguration conf, StatsLogger statsLogger)
-                throws IOException, KeeperException, InterruptedException, BookieException {
-            super(conf, statsLogger, UnpooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO);
+    class MockBookieWithNoopShutdown extends TestBookieImpl {
+        public MockBookieWithNoopShutdown(ServerConfiguration conf) throws Exception {
+            super(conf);
         }
 
         // making Bookie Shutdown no-op. Ideally for this testcase we need to
@@ -1046,7 +1069,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
     @Test
     public void testWithDiskFullAndAbilityToCreateNewIndexFile() throws Exception {
-        File tmpDir = createTempDir("DiskCheck", "test");
+        File tmpDir = tmpDirs.createNew("DiskCheck", "test");
 
         final ServerConfiguration conf = newServerConfiguration()
             .setJournalDirName(tmpDir.getPath())
@@ -1078,7 +1101,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setDiskUsageThreshold(0.001f)
                 .setDiskUsageWarnThreshold(0.0f).setReadOnlyModeEnabled(true).setIsForceGCAllowWhenNoSpace(true)
                 .setMinUsableSizeForIndexFileCreation(Long.MAX_VALUE);
-        server = new BookieServer(conf);
+        server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
         // Now we are trying to start the Bookie, which tries to replay the
         // Journal. While replaying the Journal it tries to create the IndexFile
         // for the ledger (whose entries are not flushed). but since we set
@@ -1092,10 +1118,13 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // value. So if index dirs are full then it will consider the dirs which
         // have atleast MinUsableSizeForIndexFileCreation usable space for the
         // creation of new Index file.
-        conf.setMinUsableSizeForIndexFileCreation(5 * 1024);
-        server = new BookieServer(conf);
+        conf.setMinUsableSizeForIndexFileCreation(1 * 1024);
+        server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
         server.start();
-        Thread.sleep((conf.getDiskCheckInterval() * 2) + 100);
+        Thread.sleep((conf.getDiskCheckInterval() * 2) + 1000);
         assertTrue("Bookie should be up and running", server.getBookie().isRunning());
         assertTrue(server.getBookie().isReadOnly());
         server.shutdown();
@@ -1107,7 +1136,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testWithDiskError() throws Exception {
-        File parent = createTempDir("DiskCheck", "test");
+        File parent = tmpDirs.createNew("DiskCheck", "test");
         File child = File.createTempFile("DiskCheck", "test", parent);
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(child.getPath())
@@ -1135,8 +1164,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testAllowDiskPartitionDuplicationDisabled() throws Exception {
-        File tmpDir1 = createTempDir("bookie", "test");
-        File tmpDir2 = createTempDir("bookie", "test");
+        File tmpDir1 = tmpDirs.createNew("bookie", "test");
+        File tmpDir2 = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = PortManager.nextFreePort();
@@ -1150,7 +1179,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setAllowMultipleDirsUnderSameDiskPartition(false);
         BookieServer bs1 = null;
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1160,8 +1191,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             }
         }
 
-        tmpDir1 = createTempDir("bookie", "test");
-        tmpDir2 = createTempDir("bookie", "test");
+        tmpDir1 = tmpDirs.createNew("bookie", "test");
+        tmpDir2 = tmpDirs.createNew("bookie", "test");
         port = PortManager.nextFreePort();
         // multiple indexdirs in same diskpartition
         conf.setMetadataServiceUri(metadataServiceUri)
@@ -1173,7 +1204,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setAllowMultipleDirsUnderSameDiskPartition(false);
         bs1 = null;
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1183,8 +1216,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             }
         }
 
-        tmpDir1 = createTempDir("bookie", "test");
-        tmpDir2 = createTempDir("bookie", "test");
+        tmpDir1 = tmpDirs.createNew("bookie", "test");
+        tmpDir2 = tmpDirs.createNew("bookie", "test");
         port = PortManager.nextFreePort();
         // multiple journaldirs in same diskpartition
         conf.setMetadataServiceUri(metadataServiceUri)
@@ -1196,7 +1229,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setAllowMultipleDirsUnderSameDiskPartition(false);
         bs1 = null;
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1213,12 +1248,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testAllowDiskPartitionDuplicationAllowed() throws Exception {
-        File tmpDir1 = createTempDir("bookie", "test");
-        File tmpDir2 = createTempDir("bookie", "test");
-        File tmpDir3 = createTempDir("bookie", "test");
-        File tmpDir4 = createTempDir("bookie", "test");
-        File tmpDir5 = createTempDir("bookie", "test");
-        File tmpDir6 = createTempDir("bookie", "test");
+        File tmpDir1 = tmpDirs.createNew("bookie", "test");
+        File tmpDir2 = tmpDirs.createNew("bookie", "test");
+        File tmpDir3 = tmpDirs.createNew("bookie", "test");
+        File tmpDir4 = tmpDirs.createNew("bookie", "test");
+        File tmpDir5 = tmpDirs.createNew("bookie", "test");
+        File tmpDir6 = tmpDirs.createNew("bookie", "test");
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         int port = 12555;
@@ -1231,7 +1266,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setAllowMultipleDirsUnderSameDiskPartition(true);
         BookieServer bs1 = null;
         try {
-            bs1 = new BookieServer(conf);
+            bs1 = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
         } catch (DiskPartitionDuplicationException dpde) {
             fail("Bookkeeper should have started since AllowMultipleDirsUnderSameDiskPartition is enabled");
         } finally {
@@ -1255,14 +1293,16 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     @Test(timeout = 10000)
     public void testPersistBookieStatus() throws Exception {
         // enable persistent bookie status
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(tmpDir.getPath())
             .setLedgerDirNames(new String[] { tmpDir.getPath() })
             .setReadOnlyModeEnabled(true)
             .setPersistBookieStatusEnabled(true)
             .setMetadataServiceUri(metadataServiceUri);
-        BookieServer bookieServer = new BookieServer(conf);
+        BookieServer bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         Bookie bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1272,7 +1312,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
         // restart bookie should start in read only mode
         bookieServer.shutdown();
-        bookieServer = new BookieServer(conf);
+        bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
@@ -1280,7 +1322,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookie.getStateManager().transitionToWritableMode().get();
         // restart bookie should start in writable mode
         bookieServer.shutdown();
-        bookieServer = new BookieServer(conf);
+        bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1292,7 +1336,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
      */
     @Test(timeout = 10000)
     public void testReadOnlyBookieShouldIgnoreBookieStatus() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(tmpDir.getPath())
             .setLedgerDirNames(new String[] { tmpDir.getPath() })
@@ -1300,7 +1344,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setPersistBookieStatusEnabled(true)
             .setMetadataServiceUri(metadataServiceUri);
         // start new bookie
-        BookieServer bookieServer = new BookieServer(conf);
+        BookieServer bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         Bookie bookie = bookieServer.getBookie();
         // persist bookie status
@@ -1312,7 +1358,12 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         final ServerConfiguration readOnlyConf = TestBKConfiguration.newServerConfiguration();
         readOnlyConf.loadConf(conf);
         readOnlyConf.setForceReadOnlyBookie(true);
-        bookieServer = new BookieServer(readOnlyConf);
+
+        bookieServer = new BookieServer(
+                readOnlyConf,
+                TestBookieImpl.buildReadOnly(readOnlyConf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
@@ -1331,7 +1382,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         File[] tmpLedgerDirs = new File[3];
         String[] filePath = new String[tmpLedgerDirs.length];
         for (int i = 0; i < tmpLedgerDirs.length; i++) {
-            tmpLedgerDirs[i] = createTempDir("bookie", "test" + i);
+            tmpLedgerDirs[i] = tmpDirs.createNew("bookie", "test" + i);
             filePath[i] = tmpLedgerDirs[i].getPath();
         }
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -1341,7 +1392,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setPersistBookieStatusEnabled(true)
             .setMetadataServiceUri(metadataServiceUri);
         // start a new bookie
-        BookieServer bookieServer = new BookieServer(conf);
+        BookieServer bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         // transition in to read only and persist the status on disk
         Bookie bookie = (BookieImpl) bookieServer.getBookie();
@@ -1354,7 +1407,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         corruptFile(new File(ledgerDirs.get(1), BOOKIE_STATUS_FILENAME));
         // restart the bookie should be in read only mode
         bookieServer.shutdown();
-        bookieServer = new BookieServer(conf);
+        bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
@@ -1370,7 +1425,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         File[] tmpLedgerDirs = new File[3];
         String[] filePath = new String[tmpLedgerDirs.length];
         for (int i = 0; i < tmpLedgerDirs.length; i++) {
-            tmpLedgerDirs[i] = createTempDir("bookie", "test" + i);
+            tmpLedgerDirs[i] = tmpDirs.createNew("bookie", "test" + i);
             filePath[i] = tmpLedgerDirs[i].getPath();
         }
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -1380,7 +1435,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setPersistBookieStatusEnabled(true)
             .setMetadataServiceUri(metadataServiceUri);
         // start a new bookie
-        BookieServer bookieServer = new BookieServer(conf);
+        BookieServer bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         // transition in to read only and persist the status on disk
         Bookie bookie = (BookieImpl) bookieServer.getBookie();
@@ -1395,7 +1452,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         status.writeToDirectories(dirs);
         // restart the bookie should start in writable state
         bookieServer.shutdown();
-        bookieServer = new BookieServer(conf);
+        bookieServer = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1420,7 +1479,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
 
     @Test
     public void testIOVertexHTTPServerEndpointForBookieWithPrometheusProvider() throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
 
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(tmpDir.getPath()).setLedgerDirNames(new String[] { tmpDir.getPath() })
@@ -1463,7 +1522,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         @SuppressWarnings("unchecked")
         Map<String, Object> configMap = om.readValue(url, Map.class);
         if (configMap.isEmpty() || !configMap.containsKey("bookiePort")) {
-            Assert.fail("Failed to map configurations to valid JSON entries.");
+            fail("Failed to map configurations to valid JSON entries.");
         }
         stackComponentFuture.cancel(true);
     }
@@ -1512,7 +1571,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         @SuppressWarnings("unchecked")
         Map<String, Object> configMap = om.readValue(url, Map.class);
         if (configMap.isEmpty() || !configMap.containsKey("metadataServiceUri")) {
-            Assert.fail("Failed to map configurations to valid JSON entries.");
+            fail("Failed to map configurations to valid JSON entries.");
         }
         stackComponentFuture.cancel(true);
     }
@@ -1530,7 +1589,7 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             runFunctionWithRegistrationManager(conf, rm -> {
                 try {
                     bookieConnectAfterCookieDeleteWorker(conf, rm);
-                } catch (BookieException | IOException | InterruptedException e) {
+                } catch (Exception e) {
                     fail("Test failed to run: " + e.getMessage());
                 }
                 return null;
@@ -1541,10 +1600,10 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     }
 
     private void bookieConnectAfterCookieDeleteWorker(ServerConfiguration conf, RegistrationManager rm)
-            throws BookieException, InterruptedException, IOException {
+            throws Exception {
 
-        File tmpLedgerDir = createTempDir("BootupTest", "test");
-        File tmpJournalDir = createTempDir("BootupTest", "test");
+        File tmpLedgerDir = tmpDirs.createNew("BootupTest", "test");
+        File tmpJournalDir = tmpDirs.createNew("BootupTest", "test");
         Integer numOfJournalDirs = 2;
 
         String[] journalDirs = new String[numOfJournalDirs];
@@ -1555,7 +1614,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setJournalDirsName(journalDirs);
         conf.setLedgerDirNames(new String[] { tmpLedgerDir.getPath() });
 
-        Bookie b = new TestBookieImpl(conf);
+        LifecycleComponent server = Main.buildBookieServer(new BookieConfiguration(conf));
+        server.start();
 
         final BookieId bookieAddress = BookieImpl.getBookieId(conf);
 
@@ -1563,16 +1623,16 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         Versioned<Cookie> rmCookie = Cookie.readFromRegistrationManager(rm, bookieAddress);
 
         // Shutdown bookie
-        b.shutdown();
+        server.stop();
 
         // Remove cookie from registration manager
         rmCookie.getValue().deleteFromRegistrationManager(rm, conf, rmCookie.getVersion());
 
         try {
-            b = new TestBookieImpl(conf);
-            Assert.fail("Bookie should not have come up. Cookie no present in metadata store.");
+            Main.buildBookieServer(new BookieConfiguration(conf));
+            fail("Bookie should not have been buildable. Cookie no present in metadata store.");
         } catch (Exception e) {
-            LOG.info("As expected Bookie fails to come up without a cookie in metadata store.");
+            LOG.info("As expected Bookie fails to be built without a cookie in metadata store.");
         }
     }
 
@@ -1585,8 +1645,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
             conf.setMetadataServiceUri("//ledgers");
             try {
-                new BookieServer(conf);
-                Assert.fail("Bookie metadata initialization must fail with metadata service uri: //ledgers");
+                new BookieServer(conf, new TestBookieImpl(conf),
+                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                fail("Bookie metadata initialization must fail with metadata service uri: //ledgers");
             } catch (NullPointerException e) {
                 assertTrue(e.getMessage().contains("Invalid metadata service uri : //ledgers"));
             }
@@ -1596,8 +1657,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
             conf.setMetadataServiceUri("");
             try {
-                new BookieServer(conf);
-                Assert.fail("Bookie metadata initialization must fail with empty metadata service uri");
+                new BookieServer(conf, new TestBookieImpl(conf),
+                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                fail("Bookie metadata initialization must fail with empty metadata service uri");
             } catch (NullPointerException e) {
                 assertTrue(e.getMessage().contains("Invalid metadata service uri :"));
             }
@@ -1608,8 +1670,9 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setMetadataServiceUri(uri);
         try {
-            new BookieServer(conf);
-            Assert.fail("Bookie metadata initialization must fail with an invalid metadata service uri: " + uri);
+            new BookieServer(conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+            fail("Bookie metadata initialization must fail with an invalid metadata service uri: " + uri);
         } catch (MetadataStoreException e) {
             // ok
         }
@@ -1630,24 +1693,21 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     public void testBookieIdChange() throws Exception {
         // By default, network info is set as Bookie Id and it is stored in the Cookie.
         final ServerConfiguration conf = newServerConfiguration();
-        BookieServer server = new MockBookieServer(conf);
+        LifecycleComponent server = Main.buildBookieServer(new BookieConfiguration(conf));
         server.start();
-        assertNotNull(server.getBookieId().toString());
-        server.shutdown();
+        server.stop();
 
         // If BookieID is set, it takes precedence over network info. Because of that, the new Bookie start
         // should fail with an InvalidCookieException, as now the custom BookieID takes precedence.
         String customBookieId = "customId";
         conf.setBookieId(customBookieId);
         try {
-            server = new MockBookieServer(conf);
+            Main.buildBookieServer(new BookieConfiguration(conf));
         } catch (BookieException.InvalidCookieException e) {
             // This is the expected case, as the customBookieId prevails over the default one.
         } catch (Exception e) {
             // Unexpected exception, failing.
-            Assert.fail();
+            fail();
         }
-        assertEquals(customBookieId, server.getBookieId().toString());
-        server.shutdown();
     }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java
index f1e2dd6..0aceb5e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java
@@ -59,7 +59,7 @@ import org.slf4j.LoggerFactory;
  * Test the bookie journal.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest(JournalChannel.class)
+@PrepareForTest({JournalChannel.class, FileChannelProvider.class})
 public class BookieJournalTest {
     private static final Logger LOG = LoggerFactory.getLogger(BookieJournalTest.class);
 
@@ -821,13 +821,17 @@ public class BookieJournalTest {
                 .setMetadataServiceUri(null);
 
         Journal.JournalScanner journalScanner = new DummyJournalScan();
+        BookieFileChannel bookieFileChannel = PowerMockito.mock(BookieFileChannel.class);
         FileChannel fileChannel = PowerMockito.mock(FileChannel.class);
+        FileChannelProvider fileChannelProvider = PowerMockito.mock(FileChannelProvider.class);
 
         PowerMockito.when(fileChannel.position(Mockito.anyLong()))
                 .thenThrow(new IOException());
 
-        PowerMockito.mockStatic(JournalChannel.class);
-        PowerMockito.when(JournalChannel.openFileChannel(Mockito.any(RandomAccessFile.class))).thenReturn(fileChannel);
+        PowerMockito.mockStatic(FileChannelProvider.class);
+        PowerMockito.when(FileChannelProvider.newProvider(Mockito.any())).thenReturn(fileChannelProvider);
+        Mockito.when(fileChannelProvider.open(Mockito.any(), Mockito.any())).thenReturn(bookieFileChannel);
+        Mockito.when(bookieFileChannel.getFileChannel()).thenReturn(fileChannel);
 
         BookieImpl b = new TestBookieImpl(conf);
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
index 2da712f..8b793ad 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
@@ -151,7 +151,7 @@ public class BookieShellTest {
             .thenReturn(new Versioned<>(cookie.toString().getBytes(UTF_8), version));
 
         this.driver = mock(MetadataBookieDriver.class);
-        when(driver.getRegistrationManager())
+        when(driver.createRegistrationManager())
             .thenReturn(rm);
 
         PowerMockito.mockStatic(MetadataDrivers.class);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java
index ad19d99..3f9a420 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java
@@ -145,9 +145,9 @@ public class BookieStorageThresholdTest extends BookKeeperClusterTestCase {
     public void testStorageThresholdCompaction() throws Exception {
         stopAllBookies();
         ServerConfiguration conf = newServerConfiguration();
-        File ledgerDir1 = createTempDir("ledger", "test1");
-        File ledgerDir2 = createTempDir("ledger", "test2");
-        File journalDir = createTempDir("journal", "test");
+        File ledgerDir1 = tmpDirs.createNew("ledger", "test1");
+        File ledgerDir2 = tmpDirs.createNew("ledger", "test2");
+        File journalDir = tmpDirs.createNew("journal", "test");
         String[] ledgerDirNames = new String[]{
             ledgerDir1.getPath(),
             ledgerDir2.getPath()
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java
index a2df9d3..873bb33 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java
@@ -66,7 +66,7 @@ public class BookieWriteToJournalTest {
     class NoOpJournalReplayBookie extends TestBookieImpl {
 
         public NoOpJournalReplayBookie(ServerConfiguration conf)
-                throws IOException, InterruptedException, BookieException {
+                throws Exception {
             super(conf);
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
index 4e4e74b..7302444 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
@@ -301,11 +301,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
                     lm,
                     dirManager,
                     dirManager,
-                    null,
-                    cp,
-                    Checkpointer.NULL,
                     NullStatsLogger.INSTANCE,
                     UnpooledByteBufAllocator.DEFAULT);
+                storage.setCheckpointSource(cp);
+                storage.setCheckpointer(Checkpointer.NULL);
+
                 storage.start();
                 long startTime = System.currentTimeMillis();
                 storage.gcThread.enableForceGC();
@@ -450,7 +450,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
 
         // entry logs ([0,1,2].log) should be compacted.
-        for (File ledgerDirectory : tmpDirs) {
+        for (File ledgerDirectory : tmpDirs.getDirs()) {
             assertFalse("Found entry log file ([0,1,2].log that should have not been compacted in ledgerDirectory: "
                             + ledgerDirectory, TestUtils.hasLogFiles(ledgerDirectory, true, 0, 1, 2));
         }
@@ -524,7 +524,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
 
         // entry logs ([0,1,2].log) should be compacted.
-        for (File ledgerDirectory : tmpDirs) {
+        for (File ledgerDirectory : tmpDirs.getDirs()) {
             assertFalse("Found entry log file ([0,1,2].log that should have not been compacted in ledgerDirectory: "
                     + ledgerDirectory, TestUtils.hasLogFiles(ledgerDirectory, true, 0, 1, 2));
         }
@@ -592,26 +592,26 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         //                 E3 (flushed): Entry log should have been garbage collected.
         //                 E4 (un-flushed): Entry log should exist as un-flushed entry logs are not considered for GC.
         assertTrue("Not found entry log files [0, 1, 4].log that should not have been compacted in: "
-                + tmpDirs.get(0), TestUtils.hasAllLogFiles(tmpDirs.get(0), 0, 1, 4));
+                + tmpDirs.getDirs().get(0), TestUtils.hasAllLogFiles(tmpDirs.getDirs().get(0), 0, 1, 4));
         assertTrue("Found entry log files [2, 3].log that should have been compacted in ledgerDirectory: "
-                + tmpDirs.get(0), TestUtils.hasNoneLogFiles(tmpDirs.get(0), 2, 3));
+                + tmpDirs.getDirs().get(0), TestUtils.hasNoneLogFiles(tmpDirs.getDirs().get(0), 2, 3));
 
         // Now, let's mark E1 as flushed, as its ledger L1 has been deleted already. In this case, the GC algorithm
         // should consider it for deletion.
         getGCThread().entryLogger.recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(1L);
         getGCThread().triggerGC(true, false, false).get();
         assertTrue("Found entry log file 1.log that should have been compacted in ledgerDirectory: "
-                + tmpDirs.get(0), TestUtils.hasNoneLogFiles(tmpDirs.get(0), 1));
+                + tmpDirs.getDirs().get(0), TestUtils.hasNoneLogFiles(tmpDirs.getDirs().get(0), 1));
 
         // Once removed the ledger L0, then deleting E0 is fine (only if it has been flushed).
         bkc.deleteLedger(lhs[0].getId());
         getGCThread().triggerGC(true, false, false).get();
         assertTrue("Found entry log file 0.log that should not have been compacted in ledgerDirectory: "
-                + tmpDirs.get(0), TestUtils.hasAllLogFiles(tmpDirs.get(0), 0));
+                + tmpDirs.getDirs().get(0), TestUtils.hasAllLogFiles(tmpDirs.getDirs().get(0), 0));
         getGCThread().entryLogger.recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(0L);
         getGCThread().triggerGC(true, false, false).get();
         assertTrue("Found entry log file 0.log that should have been compacted in ledgerDirectory: "
-                + tmpDirs.get(0), TestUtils.hasNoneLogFiles(tmpDirs.get(0), 0));
+                + tmpDirs.getDirs().get(0), TestUtils.hasNoneLogFiles(tmpDirs.getDirs().get(0), 0));
     }
 
     @Test
@@ -683,9 +683,9 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         // We need at least 2 ledger dirs because compaction will flush ledger cache, and will
         // trigger relocateIndexFileAndFlushHeader. If we only have one ledger dir, compaction will always fail
         // when there's no writeable ledger dir.
-        File ledgerDir1 = createTempDir("ledger", "test1");
-        File ledgerDir2 = createTempDir("ledger", "test2");
-        File journalDir = createTempDir("journal", "test");
+        File ledgerDir1 = tmpDirs.createNew("ledger", "test1");
+        File ledgerDir2 = tmpDirs.createNew("ledger", "test2");
+        File journalDir = tmpDirs.createNew("journal", "test");
         String[] ledgerDirNames = new String[]{
             ledgerDir1.getPath(),
             ledgerDir2.getPath()
@@ -843,7 +843,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         assertTrue(getGCThread().lastMajorCompactionTime > lastMajorCompactionTime);
 
         // entry logs ([0,1,2].log) should be compacted
-        for (File ledgerDirectory : tmpDirs) {
+        for (File ledgerDirectory : tmpDirs.getDirs()) {
             assertFalse("Found entry log file ([0,1,2].log that should have not been compacted in ledgerDirectory: "
                     + ledgerDirectory, TestUtils.hasLogFiles(ledgerDirectory, true, 0, 1, 2));
         }
@@ -1122,7 +1122,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         final Set<Long> ledgers = Collections.newSetFromMap(new ConcurrentHashMap<Long, Boolean>());
         LedgerManager manager = getLedgerManager(ledgers);
 
-        File tmpDir = createTempDir("bkTest", ".dir");
+        File tmpDir = tmpDirs.createNew("bkTest", ".dir");
         File curDir = BookieImpl.getCurrentDirectory(tmpDir);
         BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] {tmpDir.toString()});
@@ -1170,11 +1170,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
             manager,
             dirs,
             dirs,
-            null,
-            checkpointSource,
-            Checkpointer.NULL,
             NullStatsLogger.INSTANCE,
             UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
+
         ledgers.add(1L);
         ledgers.add(2L);
         ledgers.add(3L);
@@ -1196,11 +1196,12 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         storage.initialize(
             conf,
             manager,
-            dirs, dirs, null,
-            checkpointSource,
-            Checkpointer.NULL,
+            dirs, dirs,
             NullStatsLogger.INSTANCE,
             UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
+
         storage.start();
         for (int i = 0; i < 10; i++) {
             if (!log0.exists()) {
@@ -1222,11 +1223,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
             manager,
             dirs,
             dirs,
-            null,
-            checkpointSource,
-            Checkpointer.NULL,
             NullStatsLogger.INSTANCE,
             UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
+
         storage.getEntry(1, 1); // entry should exist
     }
 
@@ -1306,7 +1307,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
     public void testWhenNoLogsToCompact() throws Exception {
         tearDown(); // I dont want the test infrastructure
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
-        File tmpDir = createTempDir("bkTest", ".dir");
+        File tmpDir = tmpDirs.createNew("bkTest", ".dir");
         File curDir = BookieImpl.getCurrentDirectory(tmpDir);
         BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
@@ -1334,11 +1335,10 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
             manager,
             dirs,
             dirs,
-            null,
-            checkpointSource,
-            Checkpointer.NULL,
             NullStatsLogger.INSTANCE,
             UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
 
         double threshold = 0.1;
         long limit = 0;
@@ -1358,7 +1358,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
                 return c;
             });
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
-        File tmpDir = createTempDir("bkTest", ".dir");
+        File tmpDir = tmpDirs.createNew("bkTest", ".dir");
         File curDir = BookieImpl.getCurrentDirectory(tmpDir);
         BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
@@ -1383,8 +1383,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
             }
         };
         InterleavedLedgerStorage storage = new InterleavedLedgerStorage();
-        storage.initialize(conf, manager, dirs, dirs, null, checkpointSource,
-            Checkpointer.NULL, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        storage.initialize(conf, manager, dirs, dirs,
+                           NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
+
 
         for (long ledger = 0; ledger <= 10; ledger++) {
             ledgers.add(ledger);
@@ -1401,8 +1404,9 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         storage.shutdown();
 
         storage = new InterleavedLedgerStorage();
-        storage.initialize(conf, manager, dirs, dirs, null, checkpointSource,
-                           Checkpointer.NULL, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        storage.initialize(conf, manager, dirs, dirs, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(checkpointSource);
+        storage.setCheckpointer(Checkpointer.NULL);
 
         long startingEntriesCount = storage.gcThread.entryLogger.getLeastUnflushedLogId()
             - storage.gcThread.scannedLogId;
@@ -1476,11 +1480,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
             lm,
             dirManager,
             dirManager,
-            null,
-            cp,
-            Checkpointer.NULL,
             stats.getStatsLogger("storage"),
             UnpooledByteBufAllocator.DEFAULT);
+        storage.setCheckpointSource(cp);
+        storage.setCheckpointer(Checkpointer.NULL);
+
         storage.start();
 
         int majorCompactions = stats.getCounter("storage.gc." + MAJOR_COMPACTION_COUNT).get().intValue();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
index 0bb4e77..0bbdeab 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
@@ -32,6 +32,8 @@ import com.google.common.collect.Sets;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -52,22 +54,27 @@ import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.commons.io.FileUtils;
 import org.junit.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Test cookies.
  */
 public class CookieTest extends BookKeeperClusterTestCase {
+    private static final Logger log = LoggerFactory.getLogger(CookieTest.class);
+
     final int bookiePort = PortManager.nextFreePort();
 
     public CookieTest() {
         super(0);
     }
 
-    private String newDirectory() throws IOException {
+    private String newDirectory() throws Exception {
         return newDirectory(true);
     }
 
-    private String newDirectory(boolean createCurDir) throws IOException {
-        File d = createTempDir("cookie", "tmpdir");
+    private String newDirectory(boolean createCurDir) throws Exception {
+        File d = tmpDirs.createNew("cookie", "tmpdir");
         if (createCurDir) {
             new File(d, "current").mkdirs();
         }
@@ -83,33 +90,54 @@ public class CookieTest extends BookKeeperClusterTestCase {
         baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         this.metadataBookieDriver = MetadataDrivers.getBookieDriver(
             URI.create(baseConf.getMetadataServiceUri()));
-        this.metadataBookieDriver.initialize(baseConf, () -> {}, NullStatsLogger.INSTANCE);
-        this.rm = metadataBookieDriver.getRegistrationManager();
+        this.metadataBookieDriver.initialize(baseConf, NullStatsLogger.INSTANCE);
+        this.rm = metadataBookieDriver.createRegistrationManager();
     }
 
     @Override
     public void tearDown() throws Exception {
         super.tearDown();
+        if (rm != null) {
+            rm.close();
+        }
         if (metadataBookieDriver != null) {
             metadataBookieDriver.close();
         }
     }
 
+    private static List<File> currentDirectoryList(File[] dirs) {
+        return Arrays.asList(BookieImpl.getCurrentDirectories(dirs));
+    }
+
+    private void validateConfig(ServerConfiguration conf) throws Exception {
+        List<File> dirs = new ArrayList<>();
+        for (File f : conf.getJournalDirs()) {
+            File cur = BookieImpl.getCurrentDirectory(f);
+            dirs.add(cur);
+            BookieImpl.checkDirectoryStructure(cur);
+        }
+        for (File f : conf.getLedgerDirs()) {
+            File cur = BookieImpl.getCurrentDirectory(f);
+            dirs.add(cur);
+            BookieImpl.checkDirectoryStructure(cur);
+        }
+        LegacyCookieValidation cookieValidation = new LegacyCookieValidation(conf, rm);
+        cookieValidation.checkCookies(dirs);
+
+    }
+
     /**
      * Test starting bookie with clean state.
      */
     @Test
     public void testCleanStart() throws Exception {
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
-        conf.setJournalDirName(newDirectory(false))
-            .setLedgerDirNames(new String[] { newDirectory(false) })
+        conf.setJournalDirName(newDirectory(true))
+            .setLedgerDirNames(new String[] { newDirectory(true) })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        try {
-            Bookie b = new TestBookieImpl(conf);
-        } catch (Exception e) {
-            fail("Should not reach here.");
-        }
+
+        validateConfig(conf);
     }
 
     /**
@@ -140,7 +168,8 @@ public class CookieTest extends BookKeeperClusterTestCase {
         c2.writeToDirectory(new File(ledgerDir, "current"));
 
         try {
-            Bookie b = new TestBookieImpl(conf2);
+            validateConfig(conf2);
+
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -163,13 +192,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setLedgerDirNames(new String[] { ledgerDirs[0], ledgerDirs[1] });
         try {
-            Bookie b2 = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -177,16 +204,14 @@ public class CookieTest extends BookKeeperClusterTestCase {
 
         conf.setJournalDirName(newDirectory()).setLedgerDirNames(ledgerDirs);
         try {
-            Bookie b2 = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
         }
 
         conf.setJournalDirName(journalDir);
-        b = new TestBookieImpl(conf);
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
     }
 
     /**
@@ -204,15 +229,13 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         File cookieFile =
             new File(BookieImpl.getCurrentDirectory(new File(journalDir)), BookKeeperConstants.VERSION_FILENAME);
         assertTrue(cookieFile.delete());
         try {
-            new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -234,15 +257,13 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         File cookieFile =
             new File(BookieImpl.getCurrentDirectory(new File(ledgerDirs[0])), BookKeeperConstants.VERSION_FILENAME);
         assertTrue(cookieFile.delete());
         try {
-            new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -264,22 +285,18 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setLedgerDirNames(new String[] { ledgerDir0, newDirectory() });
         try {
-            Bookie b2 = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
         }
 
         conf.setLedgerDirNames(new String[] { ledgerDir0 });
-        b = new TestBookieImpl(conf);
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
     }
 
     /**
@@ -299,10 +316,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setAllowStorageExpansion(true)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        BookieImpl b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
-        b = null;
+        validateConfig(conf);
 
         // add a few additional ledger dirs
         String[] lPaths = new String[] {ledgerDir0, newDirectory(), newDirectory()};
@@ -315,12 +329,12 @@ public class CookieTest extends BookKeeperClusterTestCase {
         conf.setIndexDirName(iPaths);
 
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
         } catch (BookieException.InvalidCookieException ice) {
             fail("Should have been able to start the bookie");
         }
 
-        List<File> l = b.getLedgerDirsManager().getAllLedgerDirs();
+        List<File> l = currentDirectoryList(conf.getLedgerDirs());
         HashSet<String> bookieLedgerDirs = Sets.newHashSet();
         for (File f : l) {
             // Using the parent path because the bookie creates a 'current'
@@ -331,7 +345,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
                    + bookieLedgerDirs,
                    configuredLedgerDirs.equals(bookieLedgerDirs));
 
-        l = b.getIndexDirsManager().getAllLedgerDirs();
+        l = currentDirectoryList(conf.getIndexDirs());
         HashSet<String> bookieIndexDirs = Sets.newHashSet();
         for (File f : l) {
             bookieIndexDirs.add(f.getParent());
@@ -340,14 +354,12 @@ public class CookieTest extends BookKeeperClusterTestCase {
                    + bookieIndexDirs,
                    configuredIndexDirs.equals(bookieIndexDirs));
 
-        b.shutdown();
-
         // Make sure that substituting an older ledger directory
         // is not allowed.
         String[] lPaths2 = new String[] { lPaths[0], lPaths[1], newDirectory() };
         conf.setLedgerDirNames(lPaths2);
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Should not have been able to start the bookie");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -358,7 +370,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
         lPaths2 = new String[] { lPaths[0], lPaths[1] };
         conf.setLedgerDirNames(lPaths2);
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Should not have been able to start the bookie");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -382,10 +394,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setAllowStorageExpansion(true)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
-        b = null;
+        validateConfig(conf);
 
         // add an additional ledger dir
         String[] lPaths = new String[] {ledgerDir0, newDirectory()};
@@ -397,7 +406,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
         assertTrue(currentDir.list().length == 1);
 
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -413,7 +422,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
         assertTrue(currentDir.list().length == 1);
 
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -434,13 +443,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         FileUtils.deleteDirectory(new File(ledgerDir0));
         try {
-            Bookie b2 = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -458,13 +465,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setBookiePort(3182);
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -484,9 +489,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(newDirectory())
@@ -494,7 +497,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            b = new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -514,17 +517,17 @@ public class CookieTest extends BookKeeperClusterTestCase {
         BookKeeperAdmin.format(adminConf, false, true);
 
         ServerConfiguration bookieConf = TestBKConfiguration.newServerConfiguration();
-        bookieConf.setJournalDirName(newDirectory(false))
-            .setLedgerDirNames(new String[] { newDirectory(false) })
+        bookieConf.setJournalDirName(newDirectory(true))
+            .setLedgerDirNames(new String[] { newDirectory(true) })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         // Bookie should start successfully for fresh env.
-        new TestBookieImpl(bookieConf);
+        validateConfig(bookieConf);
 
         // Format metadata one more time.
         BookKeeperAdmin.format(adminConf, false, true);
         try {
-            new TestBookieImpl(bookieConf);
+            validateConfig(bookieConf);
             fail("Bookie should not start with previous instance id.");
         } catch (BookieException.InvalidCookieException e) {
             assertTrue(
@@ -535,7 +538,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
         // Now format the Bookie and restart.
         BookieImpl.format(bookieConf, false, true);
         // After bookie format bookie should be able to start again.
-        new TestBookieImpl(bookieConf);
+        validateConfig(bookieConf);
     }
 
     /**
@@ -544,8 +547,8 @@ public class CookieTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testV2data() throws Exception {
-        File journalDir = initV2JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV2LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -553,11 +556,18 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            Bookie b = new TestBookieImpl(conf);
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
             fail("Shouldn't have been able to start");
-        } catch (BookieException.InvalidCookieException ice) {
+        } catch (IOException ioe) {
             // correct behaviour
-            assertTrue("wrong exception", ice.getCause().getMessage().contains("upgrade needed"));
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
+        }
+        try {
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
+            fail("Shouldn't have been able to start");
+        } catch (IOException ioe) {
+            // correct behaviour
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
         }
     }
 
@@ -567,8 +577,8 @@ public class CookieTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testV1data() throws Exception {
-        File journalDir = initV1JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV1LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV1JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV1LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -576,11 +586,19 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            Bookie b = new TestBookieImpl(conf);
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
             fail("Shouldn't have been able to start");
-        } catch (BookieException.InvalidCookieException ice) {
+        } catch (IOException ioe) {
+            // correct behaviour
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
+        }
+
+        try {
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
+            fail("Shouldn't have been able to start");
+        } catch (IOException ioe) {
             // correct behaviour
-            assertTrue("wrong exception", ice.getCause().getMessage().contains("upgrade needed"));
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
         }
     }
 
@@ -598,13 +616,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setUseHostNameAsBookieID(true);
         try {
-            new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Should not start a bookie with hostname if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -625,13 +641,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         conf.setUseHostNameAsBookieID(false);
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setAdvertisedAddress("unknown");
         try {
-            new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Should not start a bookie with ip if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -652,13 +666,11 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         conf.setUseHostNameAsBookieID(true);
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
 
         conf.setUseHostNameAsBookieID(false);
         try {
-            new TestBookieImpl(conf);
+            validateConfig(conf);
             fail("Should not start a bookie with ip if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -671,8 +683,8 @@ public class CookieTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testV2dataWithHostNameAsBookieID() throws Exception {
-        File journalDir = initV2JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV2LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -680,13 +692,19 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            conf.setUseHostNameAsBookieID(true);
-            new TestBookieImpl(conf);
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
             fail("Shouldn't have been able to start");
-        } catch (BookieException.InvalidCookieException ice) {
+        } catch (IOException ioe) {
+            // correct behaviour
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
+        }
+
+        try {
+            BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
+            fail("Shouldn't have been able to start");
+        } catch (IOException ioe) {
             // correct behaviour
-            assertTrue("wrong exception",
-                    ice.getCause().getMessage().contains("upgrade needed"));
+            assertTrue("wrong exception", ioe.getMessage().contains("upgrade needed"));
         }
     }
 
@@ -702,9 +720,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
         Versioned<Cookie> zkCookie = Cookie.readFromRegistrationManager(rm, conf);
         Version version1 = zkCookie.getVersion();
         assertTrue("Invalid type expected ZkVersion type",
@@ -734,9 +750,7 @@ public class CookieTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new TestBookieImpl(conf); // should work fine
-        b.start();
-        b.shutdown();
+        validateConfig(conf);
         Versioned<Cookie> zkCookie = Cookie.readFromRegistrationManager(rm, conf);
         Cookie cookie = zkCookie.getValue();
         cookie.deleteFromRegistrationManager(rm, conf, zkCookie.getVersion());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
index 861c363..707c435 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
@@ -68,6 +68,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.FixMethodOrder;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runners.MethodSorters;
 import org.slf4j.Logger;
@@ -276,7 +277,7 @@ public class EntryLogTest {
     /**
      * Test that EntryLogger Should fail with FNFE, if entry logger directories does not exist.
      */
-    @Test
+    @Ignore // no longer valid as LedgerDirsManager creates the directory as needed
     public void testEntryLoggerShouldThrowFNFEIfDirectoriesDoesNotExist()
             throws Exception {
         File tmpDir = createTempDir("bkTest", ".dir");
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java
index afe5ec8..6c17bd6 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java
@@ -128,7 +128,7 @@ public class GcOverreplicatedLedgerTest extends LedgerManagerTestCase {
         try {
             final String metadataServiceUriStr = conf.getMetadataServiceUri();
             final MetadataBookieDriver driver = MetadataDrivers.getBookieDriver(URI.create(metadataServiceUriStr));
-            driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
+            driver.initialize(conf, NullStatsLogger.INSTANCE);
             return driver;
         } catch (MetadataException me) {
             throw new BookieException.MetadataStoreException("Failed to initialize metadata bookie driver", me);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java
index d372288..e9a02c2 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java
@@ -161,8 +161,9 @@ public class InterleavedLedgerStorageTest {
                 conf, ledgerDirsManager, null, NullStatsLogger.INSTANCE);
         interleavedStorage.initializeWithEntryLogger(
                 conf, null, ledgerDirsManager, ledgerDirsManager,
-                null, checkpointSource, checkpointer, entryLogger,
-                statsProvider.getStatsLogger(BOOKIE_SCOPE));
+                entryLogger, statsProvider.getStatsLogger(BOOKIE_SCOPE));
+        interleavedStorage.setCheckpointer(checkpointer);
+        interleavedStorage.setCheckpointSource(checkpointSource);
 
         // Insert some ledger & entries in the interleaved storage
         for (long entryId = 0; entryId < numWrites; entryId++) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java
index b1f1b38..ba8e1b9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java
@@ -612,6 +612,8 @@ public class LedgerCacheTest {
         final AtomicLong injectFlushExceptionForLedger;
         final AtomicInteger numOfTimesFlushSnapshotCalled = new AtomicInteger(0);
         static final long FORALLLEDGERS = -1;
+        ServerConfiguration conf;
+        StatsLogger statsLogger;
 
         public FlushTestSortedLedgerStorage() {
             super();
@@ -642,9 +644,6 @@ public class LedgerCacheTest {
                                LedgerManager ledgerManager,
                                LedgerDirsManager ledgerDirsManager,
                                LedgerDirsManager indexDirsManager,
-                               StateManager stateManager,
-                               CheckpointSource checkpointSource,
-                               Checkpointer checkpointer,
                                StatsLogger statsLogger,
                                ByteBufAllocator allocator) throws IOException {
             super.initialize(
@@ -652,11 +651,15 @@ public class LedgerCacheTest {
                 ledgerManager,
                 ledgerDirsManager,
                 indexDirsManager,
-                stateManager,
-                checkpointSource,
-                checkpointer,
                 statsLogger,
                 allocator);
+            this.conf = conf;
+            this.statsLogger = statsLogger;
+        }
+
+        @Override
+        public void setCheckpointSource(CheckpointSource checkpointSource) {
+            super.setCheckpointSource(checkpointSource);
             if (this.memTable instanceof EntryMemTableWithParallelFlusher) {
                 this.memTable = new EntryMemTableWithParallelFlusher(conf, checkpointSource, statsLogger) {
                     @Override
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
index eaa7505..41e3e08 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
@@ -23,6 +23,8 @@ package org.apache.bookkeeper.bookie;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
 
+import io.netty.buffer.PooledByteBufAllocator;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
@@ -53,6 +55,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.ZooKeeperUtil;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.bookkeeper.util.PortManager;
@@ -217,7 +220,9 @@ public class LedgerStorageCheckpointTest {
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
-        BookieServer server = new BookieServer(conf);
+        BookieServer server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -346,7 +351,9 @@ public class LedgerStorageCheckpointTest {
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
-        BookieServer server = new BookieServer(conf);
+        BookieServer server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -424,7 +431,9 @@ public class LedgerStorageCheckpointTest {
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
-        BookieServer server = new BookieServer(conf);
+        BookieServer server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -487,7 +496,9 @@ public class LedgerStorageCheckpointTest {
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
-        BookieServer server = new BookieServer(conf);
+        BookieServer server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -591,7 +602,9 @@ public class LedgerStorageCheckpointTest {
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
-        BookieServer server = new BookieServer(conf);
+        BookieServer server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -668,7 +681,9 @@ public class LedgerStorageCheckpointTest {
 
         // now we are restarting BookieServer
         conf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName());
-        server = new BookieServer(conf);
+        server = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
         server.start();
         BookKeeper newBKClient = new BookKeeper(clientConf);
         // since Bookie checkpointed successfully before shutdown/crash,
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
index 0b429ad..bdd7af3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
@@ -110,14 +110,11 @@ public class SlowInterleavedLedgerStorage extends InterleavedLedgerStorage {
                            LedgerManager ledgerManager,
                            LedgerDirsManager ledgerDirsManager,
                            LedgerDirsManager indexDirsManager,
-                           StateManager stateManager,
-                           CheckpointSource checkpointSource,
-                           Checkpointer checkpointer,
                            StatsLogger statsLogger,
                            ByteBufAllocator allocator)
             throws IOException {
         super.initialize(conf, ledgerManager, ledgerDirsManager, indexDirsManager,
-                stateManager, checkpointSource, checkpointer, statsLogger, allocator);
+                         statsLogger, allocator);
         // do not want to add these to config class, reading throw "raw" interface
         long getDelay = conf.getLong(PROP_SLOW_STORAGE_GET_DELAY, 0);
         long addDelay = conf.getLong(PROP_SLOW_STORAGE_ADD_DELAY, 0);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
index 48d1038..a60ab7e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
@@ -140,11 +140,10 @@ public class SortedLedgerStorageCheckpointTest extends LedgerStorageTestBase {
             mock(LedgerManager.class),
             ledgerDirsManager,
             ledgerDirsManager,
-            null,
-            checkpointSrc,
-            checkpointer,
             NullStatsLogger.INSTANCE,
             UnpooledByteBufAllocator.DEFAULT);
+        this.storage.setCheckpointer(checkpointer);
+        this.storage.setCheckpointSource(checkpointSrc);
     }
 
     @After
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java
index 2fc8961..58c72f9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java
@@ -107,8 +107,10 @@ public class SortedLedgerStorageTest {
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
         ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
-        sortedLedgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource,
-                checkpointer, statsProvider.getStatsLogger(BOOKIE_SCOPE), UnpooledByteBufAllocator.DEFAULT);
+        sortedLedgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
+                                       statsProvider.getStatsLogger(BOOKIE_SCOPE), UnpooledByteBufAllocator.DEFAULT);
+        sortedLedgerStorage.setCheckpointSource(checkpointSource);
+        sortedLedgerStorage.setCheckpointer(checkpointer);
     }
 
     @Test
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java
index ec5b9ef..d6a262c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java
@@ -23,12 +23,10 @@ package org.apache.bookkeeper.bookie;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.File;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
-import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -61,7 +59,7 @@ public class StateManagerTest extends BookKeeperClusterTestCase {
     public void setUp() throws Exception {
         super.setUp();
         zkUtil.createBKEnsemble("/" + runtime.getMethodName());
-        File tmpDir = createTempDir("stateManger", "test");
+        File tmpDir = tmpDirs.createNew("stateManger", "test");
         conf.setJournalDirName(tmpDir.getPath())
                 .setLedgerDirNames(new String[] { tmpDir.getPath() })
                 .setJournalDirName(tmpDir.toString())
@@ -81,35 +79,41 @@ public class StateManagerTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testNormalBookieTransitions() throws Exception {
-        BookieStateManager stateManager = new BookieStateManager(conf, driver);
-        driver.initialize(conf, () -> {
-            stateManager.forceToUnregistered();
-            // schedule a re-register operation
-            stateManager.registerBookie(false);
-        }, NullStatsLogger.INSTANCE);
-
-        stateManager.initState();
-        stateManager.registerBookie(true).get();
-
-        assertTrue(stateManager.isRunning());
-        assertTrue(stateManager.isRegistered());
+        driver.initialize(conf, NullStatsLogger.INSTANCE);
+        try (RegistrationManager rm = driver.createRegistrationManager();
+             BookieStateManager stateManager = new BookieStateManager(conf, rm)) {
+            rm.addRegistrationListener(() -> {
+                stateManager.forceToUnregistered();
+                // schedule a re-register operation
+                stateManager.registerBookie(false);
+            });
+            stateManager.initState();
+            stateManager.registerBookie(true).get();
 
-        stateManager.transitionToReadOnlyMode().get();
-        assertTrue(stateManager.isReadOnly());
+            assertTrue(stateManager.isRunning());
+            assertTrue(stateManager.isRegistered());
 
-        stateManager.transitionToWritableMode().get();
-        assertTrue(stateManager.isRunning());
-        assertFalse(stateManager.isReadOnly());
+            stateManager.transitionToReadOnlyMode().get();
+            assertTrue(stateManager.isReadOnly());
 
-        stateManager.close();
-        assertFalse(stateManager.isRunning());
+            stateManager.transitionToWritableMode().get();
+            assertTrue(stateManager.isRunning());
+            assertFalse(stateManager.isReadOnly());
+            stateManager.close();
+            assertFalse(stateManager.isRunning());
+        }
     }
 
     @Test
     public void testReadOnlyDisableBookieTransitions() throws Exception {
         conf.setReadOnlyModeEnabled(false);
         // readOnly disabled bk stateManager
-        BookieStateManager stateManager = new BookieStateManager(conf, driver);
+        driver.initialize(
+            conf,
+            NullStatsLogger.INSTANCE);
+
+        RegistrationManager rm = driver.createRegistrationManager();
+        BookieStateManager stateManager = new BookieStateManager(conf, rm);
         // simulate sync shutdown logic in bookie
         stateManager.setShutdownHandler(new StateManager.ShutdownHandler() {
             @Override
@@ -125,14 +129,11 @@ public class StateManagerTest extends BookKeeperClusterTestCase {
                 }
             }
         });
-        driver.initialize(
-            conf,
-            () -> {
+        rm.addRegistrationListener(() -> {
                 stateManager.forceToUnregistered();
                 // schedule a re-register operation
                 stateManager.registerBookie(false);
-            },
-            NullStatsLogger.INSTANCE);
+            });
 
         stateManager.initState();
         stateManager.registerBookie(true).get();
@@ -149,15 +150,18 @@ public class StateManagerTest extends BookKeeperClusterTestCase {
     @Test
     public void testReadOnlyBookieTransitions() throws Exception{
         // readOnlybk, which use override stateManager impl
-        File tmpDir = createTempDir("stateManger", "test-readonly");
+        File tmpDir = tmpDirs.createNew("stateManger", "test-readonly");
         final ServerConfiguration readOnlyConf = TestBKConfiguration.newServerConfiguration();
         readOnlyConf.setJournalDirName(tmpDir.getPath())
                 .setLedgerDirNames(new String[] { tmpDir.getPath() })
                 .setJournalDirName(tmpDir.toString())
                 .setMetadataServiceUri(zkUtil.getMetadataServiceUri())
                 .setForceReadOnlyBookie(true);
-        ReadOnlyBookie readOnlyBookie = new ReadOnlyBookie(readOnlyConf, NullStatsLogger.INSTANCE,
-                UnpooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO);
+        driver.initialize(readOnlyConf, NullStatsLogger.INSTANCE);
+
+        ReadOnlyBookie readOnlyBookie = TestBookieImpl.buildReadOnly(
+                new TestBookieImpl.ResourceBuilder(readOnlyConf)
+                .withMetadataDriver(driver).build());
         readOnlyBookie.start();
         assertTrue(readOnlyBookie.isRunning());
         assertTrue(readOnlyBookie.isReadOnly());
@@ -175,15 +179,18 @@ public class StateManagerTest extends BookKeeperClusterTestCase {
      */
     @Test
     public void testRegistration() throws Exception {
-        BookieStateManager stateManager = new BookieStateManager(conf, driver);
         driver.initialize(
             conf,
-            () -> {
+            NullStatsLogger.INSTANCE);
+
+        RegistrationManager rm = driver.createRegistrationManager();
+        BookieStateManager stateManager = new BookieStateManager(conf, rm);
+        rm.addRegistrationListener(() -> {
                 stateManager.forceToUnregistered();
                 // schedule a re-register operation
                 stateManager.registerBookie(false);
-            },
-            NullStatsLogger.INSTANCE);
+            });
+
         // simulate sync shutdown logic in bookie
         stateManager.setShutdownHandler(new StateManager.ShutdownHandler() {
             @Override
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
index 2de8395..0817829 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
@@ -273,13 +273,16 @@ public class SyncThreadTest {
             LedgerManager ledgerManager,
             LedgerDirsManager ledgerDirsManager,
             LedgerDirsManager indexDirsManager,
-            StateManager stateManager,
-            CheckpointSource checkpointSource,
-            Checkpointer checkpointer,
             StatsLogger statsLogger,
             ByteBufAllocator allocator)
                 throws IOException {
         }
+        @Override
+        public void setStateManager(StateManager stateManager) {}
+        @Override
+        public void setCheckpointSource(CheckpointSource checkpointSource) {}
+        @Override
+        public void setCheckpointer(Checkpointer checkpointer) {}
 
         @Override
         public void deleteLedger(long ledgerId) throws IOException {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java
index 751f99b..960651b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java
@@ -20,18 +20,177 @@
  */
 package org.apache.bookkeeper.bookie;
 
-import io.netty.buffer.PooledByteBufAllocator;
-import java.io.IOException;
+import io.netty.buffer.UnpooledByteBufAllocator;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.meta.NullMetadataBookieDriver;
 import org.apache.bookkeeper.proto.SimpleBookieServiceInfoProvider;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.DiskChecker;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Test wrapper for BookieImpl that chooses defaults for dependencies.
  */
 public class TestBookieImpl extends BookieImpl {
-    public TestBookieImpl(ServerConfiguration conf) throws IOException, InterruptedException, BookieException {
-        super(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
-                new SimpleBookieServiceInfoProvider(conf));
+    private static final Logger log = LoggerFactory.getLogger(TestBookieImpl.class);
+
+    private final Resources resources;
+
+    public TestBookieImpl(ServerConfiguration conf) throws Exception {
+        this(new ResourceBuilder(conf).build());
+    }
+
+    public TestBookieImpl(Resources resources) throws Exception {
+        super(resources.conf,
+                resources.registrationManager,
+                resources.storage,
+                resources.diskChecker,
+                resources.ledgerDirsManager,
+                resources.indexDirsManager,
+                NullStatsLogger.INSTANCE,
+                UnpooledByteBufAllocator.DEFAULT,
+                new SimpleBookieServiceInfoProvider(resources.conf));
+        this.resources = resources;
+    }
+
+    public static ReadOnlyBookie buildReadOnly(Resources resources) throws Exception {
+        return new ReadOnlyBookie(resources.conf,
+                resources.registrationManager,
+                resources.storage,
+                resources.diskChecker,
+                resources.ledgerDirsManager,
+                resources.indexDirsManager,
+                NullStatsLogger.INSTANCE,
+                UnpooledByteBufAllocator.DEFAULT,
+                new SimpleBookieServiceInfoProvider(resources.conf));
+    }
+
+    public static ReadOnlyBookie buildReadOnly(ServerConfiguration conf) throws Exception {
+        return buildReadOnly(new ResourceBuilder(conf).build());
+    }
+
+    @Override
+    int shutdown(int exitCode) {
+        int ret = super.shutdown(exitCode);
+        resources.cleanup();
+        return ret;
+    }
+
+    /**
+     * Manages bookie resources including their cleanup.
+     */
+    public static class Resources {
+        private final ServerConfiguration conf;
+        private final MetadataBookieDriver metadataDriver;
+        private final  RegistrationManager registrationManager;
+        private final LedgerManagerFactory ledgerManagerFactory;
+        private final LedgerManager ledgerManager;
+        private final LedgerStorage storage;
+        private final DiskChecker diskChecker;
+        private final LedgerDirsManager ledgerDirsManager;
+        private final LedgerDirsManager indexDirsManager;
+
+        Resources(ServerConfiguration conf,
+                  MetadataBookieDriver metadataDriver,
+                  RegistrationManager registrationManager,
+                  LedgerManagerFactory ledgerManagerFactory,
+                  LedgerManager ledgerManager,
+                  LedgerStorage storage,
+                  DiskChecker diskChecker,
+                  LedgerDirsManager ledgerDirsManager,
+                  LedgerDirsManager indexDirsManager) {
+            this.conf = conf;
+            this.metadataDriver = metadataDriver;
+            this.registrationManager = registrationManager;
+            this.ledgerManagerFactory = ledgerManagerFactory;
+            this.ledgerManager = ledgerManager;
+            this.storage = storage;
+            this.diskChecker = diskChecker;
+            this.ledgerDirsManager = ledgerDirsManager;
+            this.indexDirsManager = indexDirsManager;
+        }
+
+        void cleanup() {
+            try {
+                ledgerManager.close();
+            } catch (Exception e) {
+                log.warn("Error shutting down ledger manager", e);
+            }
+            try {
+                ledgerManagerFactory.close();
+            } catch (Exception e) {
+                log.warn("Error shutting down ledger manager factory", e);
+            }
+            registrationManager.close();
+            try {
+                metadataDriver.close();
+            } catch (Exception e) {
+                log.warn("Error shutting down metadata driver", e);
+            }
+        }
+    }
+
+    /**
+     * Builder for resources.
+     */
+    public static class ResourceBuilder {
+        private final ServerConfiguration conf;
+        private MetadataBookieDriver metadataBookieDriver;
+        private RegistrationManager registrationManager;
+
+        public ResourceBuilder(ServerConfiguration conf) {
+            this.conf = conf;
+        }
+
+        public ResourceBuilder withMetadataDriver(MetadataBookieDriver driver) {
+            this.metadataBookieDriver = driver;
+            return this;
+        }
+
+        public ResourceBuilder withRegistrationManager(RegistrationManager registrationManager) {
+            this.registrationManager = registrationManager;
+            return this;
+        }
+
+        Resources build() throws Exception {
+            if (metadataBookieDriver == null) {
+                if (conf.getMetadataServiceUri() == null) {
+                    metadataBookieDriver = new NullMetadataBookieDriver();
+                } else {
+                    metadataBookieDriver = BookieResources.createMetadataDriver(conf, NullStatsLogger.INSTANCE);
+                }
+            }
+            if (registrationManager == null) {
+                registrationManager = metadataBookieDriver.createRegistrationManager();
+            }
+            LedgerManagerFactory ledgerManagerFactory = metadataBookieDriver.getLedgerManagerFactory();
+            LedgerManager ledgerManager = ledgerManagerFactory.newLedgerManager();
+
+            DiskChecker diskChecker = BookieResources.createDiskChecker(conf);
+            LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
+                    conf, diskChecker, NullStatsLogger.INSTANCE);
+            LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
+                    conf, diskChecker, NullStatsLogger.INSTANCE, ledgerDirsManager);
+
+            LedgerStorage storage = BookieResources.createLedgerStorage(
+                    conf, ledgerManager, ledgerDirsManager, indexDirsManager,
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+
+            return new Resources(conf,
+                                 metadataBookieDriver,
+                                 registrationManager,
+                                 ledgerManagerFactory,
+                                 ledgerManager,
+                                 storage,
+                                 diskChecker,
+                                 ledgerDirsManager,
+                                 indexDirsManager);
+        }
     }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java
index a5e2e13..04f07ff 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java
@@ -32,7 +32,7 @@ import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
-import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.server.Main;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.versioning.Version;
@@ -51,9 +51,10 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
 
     MetadataBookieDriver driver;
     RegistrationManager rm;
+    ServerConfiguration conf;
 
     public UpdateCookieCmdTest() {
-        super(1);
+        super(0);
         useUUIDasBookieId = false;
     }
 
@@ -64,13 +65,20 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
         baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         driver = MetadataDrivers.getBookieDriver(
             URI.create(baseConf.getMetadataServiceUri()));
-        driver.initialize(baseConf, () -> {}, NullStatsLogger.INSTANCE);
-        rm = driver.getRegistrationManager();
+        driver.initialize(baseConf, NullStatsLogger.INSTANCE);
+        rm = driver.createRegistrationManager();
+
+        conf = newServerConfiguration();
+        LegacyCookieValidation validation = new LegacyCookieValidation(conf, rm);
+        validation.checkCookies(Main.storageDirectoriesFromConf(conf));
     }
 
     @Override
     public void tearDown() throws Exception {
         super.tearDown();
+        if (rm != null) {
+            rm.close();
+        }
         if (driver != null) {
             driver.close();
         }
@@ -102,9 +110,9 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
         updateCookie("-b", "ip", false);
 
         // start bookie to ensure everything works fine
-        ServerConfiguration conf = confByIndex(0);
-        BookieServer restartBookie = startAndAddBookie(conf).getServer();
-        restartBookie.shutdown();
+        conf.setUseHostNameAsBookieID(false);
+        LegacyCookieValidation validation = new LegacyCookieValidation(conf, rm);
+        validation.checkCookies(Main.storageDirectoriesFromConf(conf));
     }
 
     /**
@@ -113,7 +121,7 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
     @Test
     public void testUpdateCookieWithInvalidOption() throws Exception {
         String[] argv = new String[] { "updatecookie", "-b", "invalidBookieID" };
-        final ServerConfiguration conf = confByIndex(0);
+        final ServerConfiguration conf = this.conf;
         updateCookie(argv, -1, conf);
 
         argv = new String[] { "updatecookie", "-b" };
@@ -143,7 +151,7 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
         updateCookie("-b", "hostname", true);
 
         // creates cookie with ipaddress
-        ServerConfiguration conf = confByIndex(0);
+        final ServerConfiguration conf = this.conf;
         conf.setUseHostNameAsBookieID(true); // sets to hostname
         Cookie cookie = Cookie.readFromRegistrationManager(rm, conf).getValue();
         Cookie.Builder cookieBuilder = Cookie.newBuilder(cookie);
@@ -175,17 +183,13 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
     @Test
     public void testDuplicateUpdateCookieIpAddress() throws Exception {
         String[] argv = new String[] { "updatecookie", "-b", "ip" };
-        final ServerConfiguration conf = confByIndex(0);
+        final ServerConfiguration conf = this.conf;
         conf.setUseHostNameAsBookieID(true);
         updateCookie(argv, -1, conf);
     }
 
     @Test
     public void testWhenNoCookieExists() throws Exception {
-        ServerConfiguration conf = confByIndex(0);
-        BookieServer bks = serverByIndex(0);
-        bks.shutdown();
-
         String zkCookiePath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf)
             + "/" + COOKIE_NODE + "/" + BookieImpl.getBookieAddress(conf);
         Assert.assertNotNull("Cookie path doesn't still exists!", zkc.exists(zkCookiePath, false));
@@ -213,10 +217,6 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
 
     private void updateCookie(String option, String optionVal, boolean useHostNameAsBookieID, boolean useShortHostName)
             throws Exception {
-        ServerConfiguration conf = new ServerConfiguration(confByIndex(0));
-        BookieServer bks = serverByIndex(0);
-        bks.shutdown();
-
         conf.setUseHostNameAsBookieID(!useHostNameAsBookieID);
         Cookie cookie = Cookie.readFromRegistrationManager(rm, conf).getValue();
         final boolean previousBookieID = cookie.isBookieHostCreatedFromIp();
@@ -252,9 +252,6 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase {
 
     private void updateCookie(String[] argv, int exitCode, ServerConfiguration conf) throws KeeperException,
             InterruptedException, IOException, UnknownHostException, Exception {
-        BookieServer bks = serverByIndex(0);
-        bks.shutdown();
-
         LOG.info("Perform updatecookie command");
         BookieShell bkShell = new BookieShell();
         bkShell.setConf(conf);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java
index 118b97c..1c48295 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java
@@ -30,6 +30,7 @@ import io.netty.buffer.Unpooled;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintStream;
 import java.io.RandomAccessFile;
@@ -41,6 +42,9 @@ import org.apache.bookkeeper.client.ClientUtil;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.bookkeeper.util.PortManager;
@@ -153,55 +157,76 @@ public class UpgradeTest extends BookKeeperClusterTestCase {
             .setLedgerDirNames(new String[] { ledgerDir })
             .setBookiePort(bookiePort);
         Bookie b = null;
-        try {
-            b = new TestBookieImpl(conf);
+
+        try (MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                     conf, NullStatsLogger.INSTANCE);
+             RegistrationManager rm = metadataDriver.createRegistrationManager()) {
+            TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+                .withMetadataDriver(metadataDriver).withRegistrationManager(rm).build();
+            b = new TestBookieImpl(resources);
             fail("Shouldn't have been able to start");
-        } catch (BookieException.InvalidCookieException e) {
+        } catch (IOException e) {
             // correct behaviour
             assertTrue("wrong exception", e.getMessage().contains("upgrade needed"));
         }
 
         FileSystemUpgrade.upgrade(conf); // should work fine
-        b = new TestBookieImpl(conf);
-        b.start();
-        b.shutdown();
+        try (MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                     conf, NullStatsLogger.INSTANCE);
+             RegistrationManager rm = metadataDriver.createRegistrationManager()) {
+            TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+                .withMetadataDriver(metadataDriver).withRegistrationManager(rm).build();
+            b = new TestBookieImpl(resources);
+            b.start();
+            b.shutdown();
+        }
         b = null;
 
         FileSystemUpgrade.rollback(conf);
-        try {
-            b = new TestBookieImpl(conf);
+        try (MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                     conf, NullStatsLogger.INSTANCE);
+             RegistrationManager rm = metadataDriver.createRegistrationManager()) {
+            TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+                .withMetadataDriver(metadataDriver).withRegistrationManager(rm).build();
+            b = new TestBookieImpl(resources);
             fail("Shouldn't have been able to start");
-        } catch (BookieException.InvalidCookieException e) {
+        } catch (IOException e) {
             // correct behaviour
             assertTrue("wrong exception", e.getMessage().contains("upgrade needed"));
         }
 
         FileSystemUpgrade.upgrade(conf);
         FileSystemUpgrade.finalizeUpgrade(conf);
-        b = new TestBookieImpl(conf);
-        b.start();
-        b.shutdown();
+        try (MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                     conf, NullStatsLogger.INSTANCE);
+             RegistrationManager rm = metadataDriver.createRegistrationManager()) {
+            TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+                .withMetadataDriver(metadataDriver).withRegistrationManager(rm).build();
+            b = new TestBookieImpl(resources);
+            b.start();
+            b.shutdown();
+        }
         b = null;
     }
 
     @Test
     public void testUpgradeV1toCurrent() throws Exception {
-        File journalDir = initV1JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV1LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV1JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV1LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
         testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath());
     }
 
     @Test
     public void testUpgradeV2toCurrent() throws Exception {
-        File journalDir = initV2JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV2LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
         testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath());
     }
 
     @Test
     public void testUpgradeCurrent() throws Exception {
-        File journalDir = initV2JournalDirectory(createTempDir("bookie", "journal"));
-        File ledgerDir = initV2LedgerDirectory(createTempDir("bookie", "ledger"));
+        File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal"));
+        File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger"));
         testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath());
 
         // Upgrade again
@@ -211,7 +236,13 @@ public class UpgradeTest extends BookKeeperClusterTestCase {
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         FileSystemUpgrade.upgrade(conf); // should work fine with current directory
-        Bookie b = new TestBookieImpl(conf);
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                conf, NullStatsLogger.INSTANCE);
+        RegistrationManager rm = metadataDriver.createRegistrationManager();
+
+        TestBookieImpl.Resources resources = new TestBookieImpl.ResourceBuilder(conf)
+                .withMetadataDriver(metadataDriver).withRegistrationManager(rm).build();
+        Bookie b = new TestBookieImpl(resources);
         b.start();
         b.shutdown();
     }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java
index 14c1e5c..ec13fbc 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java
@@ -90,8 +90,10 @@ public class ConversionRollbackTest {
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
 
         DbLedgerStorage dbStorage = new DbLedgerStorage();
-        dbStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer,
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        dbStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
+                             NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        dbStorage.setCheckpointer(checkpointer);
+        dbStorage.setCheckpointSource(checkpointSource);
 
         // Insert some ledger & entries in the dbStorage
         for (long ledgerId = 0; ledgerId < 5; ledgerId++) {
@@ -121,7 +123,9 @@ public class ConversionRollbackTest {
         // Verify that interleaved storage index has the same entries
         InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage();
         interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
-                null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                                      NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        interleavedStorage.setCheckpointSource(checkpointSource);
+        interleavedStorage.setCheckpointer(checkpointer);
 
         Set<Long> ledgers = Sets.newTreeSet(interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE));
         Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java
index 1890ac4..38d1705 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java
@@ -89,7 +89,9 @@ public class ConversionTest {
 
         InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage();
         interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
-                null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                                      NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        interleavedStorage.setCheckpointSource(checkpointSource);
+        interleavedStorage.setCheckpointer(checkpointer);
 
         // Insert some ledger & entries in the interleaved storage
         for (long ledgerId = 0; ledgerId < 5; ledgerId++) {
@@ -119,12 +121,16 @@ public class ConversionTest {
         // Verify that db index has the same entries
         DbLedgerStorage dbStorage = new DbLedgerStorage();
         dbStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
-                null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                             NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        dbStorage.setCheckpointer(checkpointer);
+        dbStorage.setCheckpointSource(checkpointSource);
 
         interleavedStorage = new InterleavedLedgerStorage();
         interleavedStorage.initialize(conf, null, ledgerDirsManager,
-                ledgerDirsManager, null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE,
+                ledgerDirsManager, NullStatsLogger.INSTANCE,
                 UnpooledByteBufAllocator.DEFAULT);
+        interleavedStorage.setCheckpointSource(checkpointSource);
+        interleavedStorage.setCheckpointer(checkpointer);
 
         Set<Long> ledgers = Sets.newTreeSet(dbStorage.getActiveLedgersInRange(0, Long.MAX_VALUE));
         Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java
index ef1d524..b6ffcf1 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java
@@ -31,10 +31,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException;
 import org.apache.bookkeeper.bookie.BookieImpl;
-import org.apache.bookkeeper.bookie.CheckpointSource;
-import org.apache.bookkeeper.bookie.Checkpointer;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
-import org.apache.bookkeeper.bookie.StateManager;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
@@ -57,23 +54,21 @@ public class DbLedgerStorageWriteCacheTest {
         @Override
         protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf,
                 LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager,
-                StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer,
                 StatsLogger statsLogger, ScheduledExecutorService gcExecutor,
                 long writeCacheSize, long readCacheSize)
                 throws IOException {
             return new MockedSingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager,
-                    stateManager, checkpointSource, checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize,
-                    readCacheSize);
+                                                            statsLogger, allocator, gcExecutor, writeCacheSize,
+                                                            readCacheSize);
         }
 
         private static class MockedSingleDirectoryDbLedgerStorage extends SingleDirectoryDbLedgerStorage {
             public MockedSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager,
-                    LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager,
-                    CheckpointSource checkpointSource, Checkpointer checkpointer, StatsLogger statsLogger,
+                    LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger,
                     ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize,
                     long readCacheSize) throws IOException {
-                super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, stateManager, checkpointSource,
-                        checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize);
+                super(conf, ledgerManager, ledgerDirsManager, indexDirsManager,
+                      statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize);
             }
 
           @Override
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildTest.java
index e178d71..f3d3dec 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildTest.java
@@ -33,9 +33,6 @@ import java.util.UUID;
 
 import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieShell;
-import org.apache.bookkeeper.bookie.CheckpointSource;
-import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
-import org.apache.bookkeeper.bookie.Checkpointer;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
@@ -65,29 +62,6 @@ public class LedgersIndexRebuildTest {
     private ServerConfiguration conf;
     private File tmpDir;
 
-    CheckpointSource checkpointSource = new CheckpointSource() {
-        @Override
-        public Checkpoint newCheckpoint() {
-            return Checkpoint.MAX;
-        }
-
-        @Override
-        public void checkpointComplete(Checkpoint checkpoint, boolean compact) throws IOException {
-        }
-    };
-
-    Checkpointer checkpointer = new Checkpointer() {
-        @Override
-        public void startCheckpoint(Checkpoint checkpoint) {
-            // No-op
-        }
-
-        @Override
-        public void start() {
-            // no-op
-        }
-    };
-
     @Before
     public void setUp() throws IOException {
         tmpDir = File.createTempFile("bkTest", ".dir");
@@ -140,7 +114,7 @@ public class LedgersIndexRebuildTest {
         ledgerStorage = new DbLedgerStorage();
         LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
-        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer,
+        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
 
        for (long ledgerId = 0; ledgerId < ledgerCount; ledgerId++) {
@@ -165,7 +139,7 @@ public class LedgersIndexRebuildTest {
                 .thenReturn(bookieAddress);
 
         DbLedgerStorage ledgerStorage = new DbLedgerStorage();
-        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer,
+        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
 
         return ledgerStorage;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java
index 1b296fb..b9f9737 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java
@@ -88,8 +88,10 @@ public class LocationsIndexRebuildTest {
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
 
         DbLedgerStorage ledgerStorage = new DbLedgerStorage();
-        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer,
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
+                                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        ledgerStorage.setCheckpointer(checkpointer);
+        ledgerStorage.setCheckpointSource(checkpointSource);
 
         // Insert some ledger & entries in the storage
         for (long ledgerId = 0; ledgerId < 5; ledgerId++) {
@@ -118,8 +120,10 @@ public class LocationsIndexRebuildTest {
 
         // Verify that db index has the same entries
         ledgerStorage = new DbLedgerStorage();
-        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer,
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager,
+                                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+        ledgerStorage.setCheckpointSource(checkpointSource);
+        ledgerStorage.setCheckpointer(checkpointer);
 
         Set<Long> ledgers = Sets.newTreeSet(ledgerStorage.getActiveLedgersInRange(0, Long.MAX_VALUE));
         Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java
index 2001441..591761b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java
@@ -46,6 +46,9 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.component.ComponentStarter;
@@ -55,6 +58,8 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.UnderreplicatedLedger;
 import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
@@ -64,6 +69,7 @@ import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.server.Main;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger;
 import org.apache.bookkeeper.util.BookKeeperConstants;
@@ -178,18 +184,19 @@ public class BookKeeperAdminTest extends BookKeeperClusterTestCase {
 
     @Test
     public void testBookieInit() throws Exception {
-        int bookieindex = 0;
-        ServerConfiguration confOfExistingBookie = confByIndex(bookieindex);
-        Assert.assertFalse("initBookie shouldn't have succeeded, since bookie is still running with that configuration",
-                BookKeeperAdmin.initBookie(confOfExistingBookie));
-        killBookie(bookieindex);
-        Assert.assertFalse("initBookie shouldn't have succeeded, since previous bookie is not formatted yet",
-                BookKeeperAdmin.initBookie(confOfExistingBookie));
-
-        File[] journalDirs = confOfExistingBookie.getJournalDirs();
-        for (File journalDir : journalDirs) {
-            FileUtils.deleteDirectory(journalDir);
+        ServerConfiguration confOfExistingBookie = newServerConfiguration();
+        BookieId bookieId = BookieImpl.getBookieId(confOfExistingBookie);
+        try (MetadataBookieDriver driver = BookieResources.createMetadataDriver(
+                confOfExistingBookie, NullStatsLogger.INSTANCE);
+             RegistrationManager rm = driver.createRegistrationManager()) {
+            CookieValidation cookieValidation = new LegacyCookieValidation(confOfExistingBookie, rm);
+            cookieValidation.checkCookies(Main.storageDirectoriesFromConf(confOfExistingBookie));
+            rm.registerBookie(bookieId, false /* readOnly */, BookieServiceInfo.EMPTY);
+            Assert.assertFalse(
+                    "initBookie shouldn't have succeeded, since bookie is still running with that configuration",
+                    BookKeeperAdmin.initBookie(confOfExistingBookie));
         }
+
         Assert.assertFalse("initBookie shouldn't have succeeded, since previous bookie is not formatted yet completely",
                 BookKeeperAdmin.initBookie(confOfExistingBookie));
 
@@ -208,11 +215,10 @@ public class BookKeeperAdminTest extends BookKeeperClusterTestCase {
         }
         Assert.assertFalse("initBookie shouldn't have succeeded, since cookie in ZK is not deleted yet",
                 BookKeeperAdmin.initBookie(confOfExistingBookie));
-        String bookieId = BookieImpl.getBookieId(confOfExistingBookie).toString();
         String bookieCookiePath =
             ZKMetadataDriverBase.resolveZkLedgersRootPath(confOfExistingBookie)
                 + "/" + BookKeeperConstants.COOKIE_NODE
-                + "/" + bookieId;
+                + "/" + bookieId.toString();
         zkc.delete(bookieCookiePath, -1);
 
         Assert.assertTrue("initBookie shouldn't succeeded",
@@ -665,7 +671,7 @@ public class BookKeeperAdminTest extends BookKeeperClusterTestCase {
     }
 
     private void testBookieServiceInfo(boolean readonly, boolean legacy) throws Exception {
-        File tmpDir = createTempDir("bookie", "test");
+        File tmpDir = tmpDirs.createNew("bookie", "test");
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(tmpDir.getPath())
                 .setLedgerDirNames(new String[]{tmpDir.getPath()})
@@ -690,7 +696,7 @@ public class BookKeeperAdminTest extends BookKeeperClusterTestCase {
             String regPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(bkConf) + "/" + AVAILABLE_NODE;
             regPath = readonly
                     ? regPath + READONLY + "/" + bookieId
-                    : regPath + "/" + bookieId;
+                    : regPath + "/" + bookieId.toString();
             // deleting the metadata, so that the bookie registration should
             // continue successfully with legacy BookieServiceInfo
             zkc.setData(regPath, new byte[]{}, -1);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
index 1ecf4a7..e56c19b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
@@ -49,7 +49,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
-import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException.BKLedgerClosedException;
@@ -64,7 +63,6 @@ import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.test.TestStatsProvider;
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.KeeperException;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -1501,7 +1499,7 @@ public class BookieWriteLedgerTest extends
         ByteBuf localBuf;
 
         public CorruptReadBookie(ServerConfiguration conf)
-                throws IOException, KeeperException, InterruptedException, BookieException {
+                throws Exception {
             super(conf);
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
index 550e445..6aa1ed7 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
@@ -65,7 +65,6 @@ import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -518,7 +517,7 @@ public class ParallelLedgerRecoveryTest extends BookKeeperClusterTestCase {
                 new LinkedBlockingQueue<WriteCallbackEntry>();
 
         public DelayResponseBookie(ServerConfiguration conf)
-                throws IOException, KeeperException, InterruptedException, BookieException {
+                throws Exception {
             super(conf);
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java
index 47df744..ef17059 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java
@@ -311,7 +311,7 @@ public class SlowBookieTest extends BookKeeperClusterTestCase {
         BookKeeper bkc = new BookKeeper(conf);
 
         byte[] pwd = new byte[] {};
-        final LedgerHandle lh = bkc.createLedger(4, 3, 1, BookKeeper.DigestType.CRC32, pwd);
+        final LedgerHandle lh = bkc.createLedger(4, 3, 2, BookKeeper.DigestType.CRC32, pwd);
         final AtomicBoolean finished = new AtomicBoolean(false);
         final AtomicBoolean failTest = new AtomicBoolean(false);
         Thread t = new Thread() {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
index dc7a6a9..662e2ae 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.bookkeeper.bookie.Bookie;
-import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.SortedLedgerStorage;
@@ -42,7 +41,6 @@ import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -82,7 +80,7 @@ public class TestReadLastConfirmedAndEntry extends BookKeeperClusterTestCase {
         final boolean stallOrRespondNull;
 
         public FakeBookie(ServerConfiguration conf, long expectedEntryToFail, boolean stallOrRespondNull)
-                throws InterruptedException, BookieException, KeeperException, IOException {
+                throws Exception {
             super(conf);
             this.expectedEntryToFail = expectedEntryToFail;
             this.stallOrRespondNull = stallOrRespondNull;
@@ -180,7 +178,7 @@ public class TestReadLastConfirmedAndEntry extends BookKeeperClusterTestCase {
 
         public SlowReadLacBookie(ServerConfiguration conf,
                                  long lacToSlowRead, CountDownLatch readLatch)
-                throws IOException, KeeperException, InterruptedException, BookieException {
+                throws Exception {
             super(conf);
             this.lacToSlowRead = lacToSlowRead;
             this.readLatch = readLatch;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
index e514121..01d64a9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
@@ -100,7 +100,7 @@ public class UpdateLedgerCmdTest extends BookKeeperClusterTestCase {
         String[] argv = new String[] { "updateBookieInLedger", "-sb", srcBookie.toString(), "-db",
                 destBookie.toString(), "-v", "true", "-p", "2" };
         final ServerConfiguration conf = confByIndex(0);
-        serverByIndex(0).shutdown();
+        killBookie(0);
         updateLedgerCmd(argv, 0, conf);
         int updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, srcBookie);
         assertEquals("Failed to update the ledger metadata with new bookie-address", 0, updatedLedgersCount);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java
index 47e46e5..5ceee0b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java
@@ -28,6 +28,7 @@ import java.util.Enumeration;
 
 import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
+import org.apache.bookkeeper.util.PortManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,8 +52,8 @@ public class TestBKConfiguration {
         confReturn.setJournalFlushWhenQueueEmpty(true);
         // enable journal format version
         confReturn.setJournalFormatVersionToWrite(5);
-        confReturn.setAllowEphemeralPorts(true);
-        confReturn.setBookiePort(0);
+        confReturn.setAllowEphemeralPorts(false);
+        confReturn.setBookiePort(PortManager.nextFreePort());
         confReturn.setGcWaitTime(1000);
         confReturn.setDiskUsageThreshold(0.999f);
         confReturn.setDiskUsageWarnThreshold(0.99f);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java
index 5e4ea24..4e1b06b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java
@@ -54,7 +54,7 @@ public class TestZkRegistrationManager {
             ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
             conf.setMetadataServiceUri("zk+hierarchical://localhost:2181/test/ledgers");
             zkc = localZkServer.getZooKeeperClient();
-            ZKRegistrationManager zkRegistrationManager = new ZKRegistrationManager(conf, zkc, () -> {});
+            ZKRegistrationManager zkRegistrationManager = new ZKRegistrationManager(conf, zkc);
             zkRegistrationManager.prepareFormat();
             assertNotNull(zkc.exists("/test/ledgers", false));
         } finally {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
index d463d4e..ae11f0b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
@@ -577,14 +577,18 @@ public class GcLedgersTest extends LedgerManagerTestCase {
             LedgerManager ledgerManager,
             LedgerDirsManager ledgerDirsManager,
             LedgerDirsManager indexDirsManager,
-            StateManager stateManager,
-            CheckpointSource checkpointSource,
-            Checkpointer checkpointer,
             StatsLogger statsLogger,
             ByteBufAllocator allocator) throws IOException {
         }
 
         @Override
+        public void setStateManager(StateManager stateManager) {}
+        @Override
+        public void setCheckpointSource(CheckpointSource checkpointSource) {}
+        @Override
+        public void setCheckpointer(Checkpointer checkpointer) {}
+
+        @Override
         public void start() {
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
index cb85f4e..3796250 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
@@ -174,14 +174,18 @@ public abstract class LedgerManagerTestCase extends BookKeeperClusterTestCase {
             LedgerManager ledgerManager,
             LedgerDirsManager ledgerDirsManager,
             LedgerDirsManager indexDirsManager,
-            StateManager stateManager,
-            CheckpointSource checkpointSource,
-            Checkpointer checkpointer,
             StatsLogger statsLogger,
             ByteBufAllocator allocator) throws IOException {
         }
 
         @Override
+        public void setStateManager(StateManager stateManager) {}
+        @Override
+        public void setCheckpointSource(CheckpointSource checkpointSource) {}
+        @Override
+        public void setCheckpointer(Checkpointer checkpointer) {}
+
+        @Override
         public void start() {
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java
index 593c520..90f956a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java
@@ -36,7 +36,6 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationClient;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.meta.MetadataDrivers.MetadataBookieDriverInfo;
 import org.apache.bookkeeper.meta.MetadataDrivers.MetadataClientDriverInfo;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
@@ -107,13 +106,12 @@ public class MetadataDriversTest {
     abstract static class TestBookieDriver implements MetadataBookieDriver {
         @Override
         public MetadataBookieDriver initialize(ServerConfiguration conf,
-                                               RegistrationListener listener,
                                                StatsLogger statsLogger) throws MetadataException {
             return this;
         }
 
         @Override
-        public RegistrationManager getRegistrationManager() {
+        public RegistrationManager createRegistrationManager() {
             return mock(RegistrationManager.class);
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriverTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriverTest.java
index ca3279d..055ab60 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriverTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataBookieDriverTest.java
@@ -18,13 +18,11 @@
  */
 package org.apache.bookkeeper.meta.zk;
 
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.same;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
@@ -33,6 +31,7 @@ import org.apache.bookkeeper.discover.ZKRegistrationManager;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
 import org.apache.zookeeper.ZooKeeper;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -58,41 +57,37 @@ public class ZKMetadataBookieDriverTest extends ZKMetadataDriverTestBase {
         driver = new ZKMetadataBookieDriver();
     }
 
+    @After
+    public void teardown() {
+        driver.close();
+    }
+
     @Test
     public void testGetRegManager() throws Exception {
         RegistrationListener listener = mock(RegistrationListener.class);
-        driver.initialize(conf, listener, NullStatsLogger.INSTANCE);
+        driver.initialize(conf, NullStatsLogger.INSTANCE);
 
         assertSame(conf, driver.serverConf);
-        assertSame(listener, driver.listener);
-        assertNull(driver.regManager);
 
         ZKRegistrationManager mockRegManager = PowerMockito.mock(ZKRegistrationManager.class);
 
         PowerMockito.whenNew(ZKRegistrationManager.class)
             .withParameterTypes(
                 ServerConfiguration.class,
-                ZooKeeper.class,
-                RegistrationListener.class)
+                ZooKeeper.class)
             .withArguments(
                 any(ServerConfiguration.class),
-                any(ZooKeeper.class),
-                any(RegistrationListener.class))
+                any(ZooKeeper.class))
             .thenReturn(mockRegManager);
 
-        RegistrationManager manager = driver.getRegistrationManager();
-        assertSame(mockRegManager, manager);
-        assertSame(mockRegManager, driver.regManager);
+        try (RegistrationManager manager = driver.createRegistrationManager()) {
+            assertSame(mockRegManager, manager);
 
-        PowerMockito.verifyNew(ZKRegistrationManager.class, times(1))
-            .withArguments(
-                same(conf),
-                same(mockZkc),
-                same(listener));
-
-        driver.close();
-        verify(mockRegManager, times(1)).close();
-        assertNull(driver.regManager);
+            PowerMockito.verifyNew(ZKRegistrationManager.class, times(1))
+                    .withArguments(
+                            same(conf),
+                            same(mockZkc));
+        }
     }
 
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
index 4a120c4..18ace93 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
@@ -128,7 +128,6 @@ public class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
             URI.create(confByIndex(0).getMetadataServiceUri()));
         driver.initialize(
             confByIndex(0),
-            () -> {},
             NullStatsLogger.INSTANCE);
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java
index 254de9a..7608e8e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java
@@ -83,8 +83,7 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         super.setUp();
         StaticDNSResolver.reset();
         driver = MetadataDrivers.getBookieDriver(URI.create(confByIndex(0).getMetadataServiceUri()));
-        driver.initialize(confByIndex(0), () -> {
-        }, NullStatsLogger.INSTANCE);
+        driver.initialize(confByIndex(0), NullStatsLogger.INSTANCE);
     }
 
     @After
@@ -101,14 +100,14 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         int numOfBookies = 5;
         List<BookieId> bookieAddresses = new ArrayList<>();
         BookieSocketAddress bookieAddress;
-        RegistrationManager regManager = driver.getRegistrationManager();
-
-        // all the numOfBookies (5) are going to be in different racks
-        for (int i = 0; i < numOfBookies; i++) {
-            bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181);
-            StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), "/rack" + (i));
-            bookieAddresses.add(bookieAddress.toBookieId());
-            regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY);
+        try (RegistrationManager regManager = driver.createRegistrationManager()) {
+            // all the numOfBookies (5) are going to be in different racks
+            for (int i = 0; i < numOfBookies; i++) {
+                bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181);
+                StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), "/rack" + (i));
+                bookieAddresses.add(bookieAddress.toBookieId());
+                regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY);
+            }
         }
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
@@ -216,12 +215,12 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         int numOfBookies = 5;
         int numOfLedgersNotAdheringToPlacementPolicy = 0;
         List<BookieId> bookieAddresses = new ArrayList<>();
-        RegistrationManager regManager = driver.getRegistrationManager();
-
-        for (int i = 0; i < numOfBookies; i++) {
-            BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
-            bookieAddresses.add(bookieAddress);
-            regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+        try (RegistrationManager regManager = driver.createRegistrationManager()) {
+            for (int i = 0; i < numOfBookies; i++) {
+                BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
+                bookieAddresses.add(bookieAddress);
+                regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+            }
         }
 
         // only three racks
@@ -313,12 +312,12 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         int underreplicatedLedgerRecoveryGracePeriod = timeElapsed ? 1 : 1000;
         int numOfURLedgersElapsedRecoveryGracePeriod = 0;
         List<BookieId> bookieAddresses = new ArrayList<BookieId>();
-        RegistrationManager regManager = driver.getRegistrationManager();
-
-        for (int i = 0; i < numOfBookies; i++) {
-            BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
-            bookieAddresses.add(bookieAddress);
-            regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+        try (RegistrationManager regManager = driver.createRegistrationManager()) {
+            for (int i = 0; i < numOfBookies; i++) {
+                BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
+                bookieAddresses.add(bookieAddress);
+                regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+            }
         }
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
@@ -432,12 +431,12 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         int numOfBookies = 7;
         int numOfLedgersNotAdheringToPlacementPolicy = 0;
         List<BookieId> bookieAddresses = new ArrayList<>();
-        RegistrationManager regManager = driver.getRegistrationManager();
-
-        for (int i = 0; i < numOfBookies; i++) {
-            BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
-            bookieAddresses.add(bookieAddress);
-            regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+        try (RegistrationManager regManager = driver.createRegistrationManager()) {
+            for (int i = 0; i < numOfBookies; i++) {
+                BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId();
+                bookieAddresses.add(bookieAddress);
+                regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY);
+            }
         }
 
         // only three racks
@@ -537,19 +536,19 @@ public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase {
         int numOfLedgersNotAdheringToPlacementPolicy = 0;
         int numOfLedgersSoftlyAdheringToPlacementPolicy = 0;
         List<BookieId> bookieAddresses = new ArrayList<BookieId>();
-        RegistrationManager regManager = driver.getRegistrationManager();
-
-        /*
-         * 6 bookies - 3 zones and 2 uds
-         */
-        for (int i = 0; i < numOfBookies; i++) {
-            BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181);
-            bookieAddresses.add(bookieAddress.toBookieId());
-            regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY);
-            String zone = "/zone" + (i % 3);
-            String upgradeDomain = "/ud" + (i % 2);
-            String networkLocation = zone + upgradeDomain;
-            StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), networkLocation);
+        try (RegistrationManager regManager = driver.createRegistrationManager()) {
+            /*
+             * 6 bookies - 3 zones and 2 uds
+             */
+            for (int i = 0; i < numOfBookies; i++) {
+                BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181);
+                bookieAddresses.add(bookieAddress.toBookieId());
+                regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY);
+                String zone = "/zone" + (i % 3);
+                String upgradeDomain = "/ud" + (i % 2);
+                String networkLocation = zone + upgradeDomain;
+                StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), networkLocation);
+            }
         }
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java
index 550147b..279f4b5 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java
@@ -77,6 +77,7 @@ import org.junit.Test;
  */
 public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     private MetadataBookieDriver driver;
+    private RegistrationManager regManager;
 
     public AuditorReplicasCheckTest() {
         super(1);
@@ -89,13 +90,16 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
         super.setUp();
         StaticDNSResolver.reset();
         driver = MetadataDrivers.getBookieDriver(URI.create(confByIndex(0).getMetadataServiceUri()));
-        driver.initialize(confByIndex(0), () -> {
-        }, NullStatsLogger.INSTANCE);
+        driver.initialize(confByIndex(0), NullStatsLogger.INSTANCE);
+        regManager = driver.createRegistrationManager();
     }
 
     @After
     @Override
     public void tearDown() throws Exception {
+        if (null != regManager) {
+            regManager.close();
+        }
         if (null != driver) {
             driver.close();
         }
@@ -178,7 +182,7 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
         servConf.setAuditorPeriodicReplicasCheckInterval(1000);
     }
 
-    List<BookieId> addAndRegisterBookies(RegistrationManager regManager, int numOfBookies)
+    List<BookieId> addAndRegisterBookies(int numOfBookies)
             throws BookieException {
         BookieId bookieAddress;
         List<BookieId> bookieAddresses = new ArrayList<BookieId>();
@@ -271,12 +275,11 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     @Test
     public void testReplicasCheckForBookieHandleNotAvailable() throws Exception {
         int numOfBookies = 5;
-        RegistrationManager regManager = driver.getRegistrationManager();
         MultiKeyMap<String, AvailabilityOfEntriesOfLedger> returnAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, AvailabilityOfEntriesOfLedger>();
         MultiKeyMap<String, Integer> errorReturnValueForGetAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, Integer>();
-        List<BookieId> bookieAddresses = addAndRegisterBookies(regManager, numOfBookies);
+        List<BookieId> bookieAddresses = addAndRegisterBookies(numOfBookies);
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
         LedgerManager lm = mFactory.newLedgerManager();
@@ -369,12 +372,11 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     @Test
     public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception {
         int numOfBookies = 5;
-        RegistrationManager regManager = driver.getRegistrationManager();
         MultiKeyMap<String, AvailabilityOfEntriesOfLedger> returnAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, AvailabilityOfEntriesOfLedger>();
         MultiKeyMap<String, Integer> errorReturnValueForGetAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, Integer>();
-        List<BookieId> bookieAddresses = addAndRegisterBookies(regManager, numOfBookies);
+        List<BookieId> bookieAddresses = addAndRegisterBookies(numOfBookies);
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
         LedgerManager lm = mFactory.newLedgerManager();
@@ -505,12 +507,11 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     @Test
     public void testReplicasCheckForLedgersFoundHavingLessThanAQReplicasOfAnEntry() throws Exception {
         int numOfBookies = 5;
-        RegistrationManager regManager = driver.getRegistrationManager();
         MultiKeyMap<String, AvailabilityOfEntriesOfLedger> returnAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, AvailabilityOfEntriesOfLedger>();
         MultiKeyMap<String, Integer> errorReturnValueForGetAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, Integer>();
-        List<BookieId> bookieAddresses = addAndRegisterBookies(regManager, numOfBookies);
+        List<BookieId> bookieAddresses = addAndRegisterBookies(numOfBookies);
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
         LedgerManager lm = mFactory.newLedgerManager();
@@ -660,12 +661,11 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     @Test
     public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() throws Exception {
         int numOfBookies = 5;
-        RegistrationManager regManager = driver.getRegistrationManager();
         MultiKeyMap<String, AvailabilityOfEntriesOfLedger> returnAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, AvailabilityOfEntriesOfLedger>();
         MultiKeyMap<String, Integer> errorReturnValueForGetAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, Integer>();
-        List<BookieId> bookieAddresses = addAndRegisterBookies(regManager, numOfBookies);
+        List<BookieId> bookieAddresses = addAndRegisterBookies(numOfBookies);
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
         LedgerManager lm = mFactory.newLedgerManager();
@@ -813,12 +813,11 @@ public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase {
     @Test
     public void testReplicasCheckForLedgersWithEmptySegments() throws Exception {
         int numOfBookies = 5;
-        RegistrationManager regManager = driver.getRegistrationManager();
         MultiKeyMap<String, AvailabilityOfEntriesOfLedger> returnAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, AvailabilityOfEntriesOfLedger>();
         MultiKeyMap<String, Integer> errorReturnValueForGetAvailabilityOfEntriesOfLedger =
                 new MultiKeyMap<String, Integer>();
-        List<BookieId> bookieAddresses = addAndRegisterBookies(regManager, numOfBookies);
+        List<BookieId> bookieAddresses = addAndRegisterBookies(numOfBookies);
 
         LedgerManagerFactory mFactory = driver.getLedgerManagerFactory();
         LedgerManager lm = mFactory.newLedgerManager();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
index ace70f8..bb30235 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
@@ -133,7 +133,6 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
             URI.create(baseConf.getMetadataServiceUri()));
         this.driver.initialize(
             baseConf,
-            () -> {},
             NullStatsLogger.INSTANCE);
         // initialize urReplicationManager
         mFactory = driver.getLedgerManagerFactory();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestBookieBoot.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestBookieBoot.java
new file mode 100644
index 0000000..182fe63
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestBookieBoot.java
@@ -0,0 +1,132 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.server;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+
+import java.io.File;
+import java.net.Socket;
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.ExitCode;
+import org.apache.bookkeeper.client.BookKeeperTestClient;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.bookkeeper.client.api.WriteHandle;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.apache.bookkeeper.util.PortManager;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests that a bookie can boot via the main method
+ * and serve read and write requests.
+ */
+public class TestBookieBoot extends BookKeeperClusterTestCase {
+    private static final Logger log = LoggerFactory.getLogger(TestBookieBoot.class);
+    public TestBookieBoot() throws Exception {
+        super(0);
+    }
+
+    @Test
+    public void testBootFromConfig() throws Exception {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setMetadataServiceUri(this.metadataServiceUri);
+        conf.setAllowLoopback(true);
+        conf.setBookiePort(PortManager.nextFreePort());
+        conf.setLedgerStorageClass("org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage");
+
+        File storageDir = tmpDirs.createNew("bookie", "storage");
+        conf.setLedgerDirNames(new String[] { storageDir.toString() });
+        conf.setJournalDirName(storageDir.toString());
+
+        PropertiesConfiguration propsConf = new PropertiesConfiguration();
+        for (Iterator<String> iter = conf.getKeys(); iter.hasNext(); ) {
+            String key = iter.next();
+            propsConf.setProperty(key, conf.getProperty(key));
+        }
+
+        File confFile = File.createTempFile("test", "conf");
+        propsConf.save(confFile);
+
+        log.info("Conf: {}", confFile);
+
+        CompletableFuture<Integer> promise = new CompletableFuture<>();
+        Thread t = new Thread(() -> {
+            try {
+                int ret = Main.doMain(new String[] {"-c", confFile.toString()});
+                promise.complete(ret);
+            } catch (Exception e) {
+                promise.completeExceptionally(e);
+            }
+        }, "bookie-main");
+        t.start();
+
+        BookieSocketAddress addr = BookieImpl.getBookieAddress(conf);
+        BookKeeperTestClient bkc = new BookKeeperTestClient(baseClientConf);
+        bkc.waitForWritableBookie(addr.toBookieId()).get();
+
+        boolean connected = false;
+        for (int i = 0; i < 100 && t.isAlive(); i++) {
+            try (Socket s = new Socket(addr.getSocketAddress().getAddress(), addr.getPort())) {
+                connected = true;
+                break;
+            } catch (Exception e) {
+                // expected, will retry
+            }
+            Thread.sleep(100);
+        }
+        assertThat(connected, equalTo(true));
+
+        long ledgerId;
+        try (WriteHandle wh = bkc.newCreateLedgerOp().withEnsembleSize(1)
+                .withWriteQuorumSize(1).withAckQuorumSize(1)
+                .withDigestType(DigestType.CRC32C)
+                .withPassword(new byte[0])
+                .execute().get()) {
+            ledgerId = wh.getId();
+            wh.append("foobar".getBytes(UTF_8));
+        }
+
+        try (ReadHandle rh = bkc.newOpenLedgerOp().withLedgerId(ledgerId)
+                .withDigestType(DigestType.CRC32C)
+                .withPassword(new byte[0])
+                .withRecovery(true)
+                .execute().get()) {
+            assertThat(rh.getLastAddConfirmed(), equalTo(0L));
+            try (LedgerEntries entries = rh.read(0, 0)) {
+                assertThat(new String(entries.getEntry(0).getEntryBytes(), UTF_8), equalTo("foobar"));
+            }
+        }
+
+        t.interrupt();
+        assertThat(promise.get(10, TimeUnit.SECONDS), equalTo(ExitCode.OK));
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java
index fd4be6d..73e602c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java
@@ -19,21 +19,24 @@
 
 package org.apache.bookkeeper.server;
 
-import static org.apache.bookkeeper.server.Main.buildBookieServer;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.when;
 import static org.powermock.api.mockito.PowerMockito.whenNew;
 
 import java.io.IOException;
 
-import java.util.function.Supplier;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.LifecycleComponentStack;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.NullMetadataBookieDriver;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
@@ -50,7 +53,7 @@ import org.powermock.modules.junit4.PowerMockRunner;
  * Unit test of {@link Main}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest(BookieService.class)
+@PrepareForTest({BookieService.class, BookieResources.class, Main.class})
 public class TestMain {
 
     static class TestComponent extends ServerLifecycleComponent {
@@ -75,24 +78,34 @@ public class TestMain {
 
     @Test
     public void testBuildBookieServer() throws Exception {
+        PowerMockito.mockStatic(BookieResources.class);
+        when(BookieResources.createMetadataDriver(any(), any()))
+            .thenReturn(new NullMetadataBookieDriver());
+        when(BookieResources.createAllocator(any())).thenReturn(
+                PowerMockito.mock(ByteBufAllocatorWithOomHandler.class));
+
         ServerConfiguration serverConf = new ServerConfiguration()
             .setAutoRecoveryDaemonEnabled(false)
             .setHttpServerEnabled(false)
             .setExtraServerComponents(new String[] { TestComponent.class.getName() });
         BookieConfiguration conf = new BookieConfiguration(serverConf);
 
+        whenNew(BookieImpl.class).withAnyArguments().thenReturn(PowerMockito.mock(BookieImpl.class));
+        whenNew(LegacyCookieValidation.class)
+            .withAnyArguments().thenReturn(PowerMockito.mock(LegacyCookieValidation.class));
+
         BookieServer mockServer = PowerMockito.mock(BookieServer.class);
         whenNew(BookieServer.class)
-            .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class))
+            .withAnyArguments()
             .thenReturn(mockServer);
 
         BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281);
         when(mockServer.getLocalAddress()).thenReturn(bookieAddress);
         when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId());
 
-        LifecycleComponentStack stack = buildBookieServer(conf);
-        assertEquals(3, stack.getNumComponents());
-        assertTrue(stack.getComponent(2) instanceof TestComponent);
+        LifecycleComponentStack stack = Main.buildBookieServer(conf);
+        assertEquals(7, stack.getNumComponents());
+        assertTrue(stack.getComponent(6) instanceof TestComponent);
 
         stack.start();
         verify(mockServer, times(1)).start();
@@ -105,6 +118,10 @@ public class TestMain {
 
     @Test
     public void testIgnoreExtraServerComponentsStartupFailures() throws Exception {
+        PowerMockito.mockStatic(BookieResources.class);
+        when(BookieResources.createMetadataDriver(any(), any()))
+            .thenReturn(new NullMetadataBookieDriver());
+
         ServerConfiguration serverConf = new ServerConfiguration()
             .setAutoRecoveryDaemonEnabled(false)
             .setHttpServerEnabled(false)
@@ -112,17 +129,21 @@ public class TestMain {
             .setIgnoreExtraServerComponentsStartupFailures(true);
         BookieConfiguration conf = new BookieConfiguration(serverConf);
 
+        whenNew(BookieImpl.class).withAnyArguments().thenReturn(PowerMockito.mock(BookieImpl.class));
+        whenNew(LegacyCookieValidation.class)
+            .withAnyArguments().thenReturn(PowerMockito.mock(LegacyCookieValidation.class));
+
         BookieServer mockServer = PowerMockito.mock(BookieServer.class);
         whenNew(BookieServer.class)
-            .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class))
+            .withAnyArguments()
             .thenReturn(mockServer);
 
         BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281);
         when(mockServer.getLocalAddress()).thenReturn(bookieAddress);
         when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId());
 
-        LifecycleComponentStack stack = buildBookieServer(conf);
-        assertEquals(2, stack.getNumComponents());
+        LifecycleComponentStack stack = Main.buildBookieServer(conf);
+        assertEquals(6, stack.getNumComponents());
 
         stack.start();
         verify(mockServer, times(1)).start();
@@ -135,6 +156,10 @@ public class TestMain {
 
     @Test
     public void testExtraServerComponentsStartupFailures() throws Exception {
+        PowerMockito.mockStatic(BookieResources.class);
+        when(BookieResources.createMetadataDriver(any(), any()))
+            .thenReturn(new NullMetadataBookieDriver());
+
         ServerConfiguration serverConf = new ServerConfiguration()
             .setAutoRecoveryDaemonEnabled(false)
             .setHttpServerEnabled(false)
@@ -142,9 +167,13 @@ public class TestMain {
             .setIgnoreExtraServerComponentsStartupFailures(false);
         BookieConfiguration conf = new BookieConfiguration(serverConf);
 
+        whenNew(BookieImpl.class).withAnyArguments().thenReturn(PowerMockito.mock(BookieImpl.class));
+        whenNew(LegacyCookieValidation.class)
+            .withAnyArguments().thenReturn(PowerMockito.mock(LegacyCookieValidation.class));
+
         BookieServer mockServer = PowerMockito.mock(BookieServer.class);
         whenNew(BookieServer.class)
-            .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class))
+                .withAnyArguments()
             .thenReturn(mockServer);
 
         BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281);
@@ -152,7 +181,7 @@ public class TestMain {
         when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId());
 
         try {
-            buildBookieServer(conf);
+            Main.buildBookieServer(conf);
             fail("Should fail to start bookie server if `ignoreExtraServerComponentsStartupFailures` is set to false");
         } catch (RuntimeException re) {
             assertTrue(re.getCause() instanceof ClassNotFoundException);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
index 35f97e7..a00b1cd 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
@@ -35,6 +35,7 @@ import java.util.concurrent.Future;
 
 import lombok.Cleanup;
 
+import org.apache.bookkeeper.bookie.BookieResources;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.ClientUtil;
 import org.apache.bookkeeper.client.LedgerHandle;
@@ -49,10 +50,12 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.replication.AuditorElector;
 import org.apache.bookkeeper.server.http.service.BookieInfoService;
 import org.apache.bookkeeper.server.http.service.BookieStateService.BookieState;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.junit.Before;
 import org.junit.Test;
@@ -72,7 +75,7 @@ public class TestHttpService extends BookKeeperClusterTestCase {
     public TestHttpService() {
         super(numberOfBookies);
         try {
-            File tmpDir = createTempDir("bookie_http", "test");
+            File tmpDir = tmpDirs.createNew("bookie_http", "test");
             baseConf.setJournalDirName(tmpDir.getPath())
               .setLedgerDirNames(
                 new String[]{tmpDir.getPath()});
@@ -88,9 +91,13 @@ public class TestHttpService extends BookKeeperClusterTestCase {
         baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         baseClientConf.setStoreSystemtimeAsLedgerCreationTime(true);
 
+        MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver(
+                baseConf, NullStatsLogger.INSTANCE);
+
         this.bkHttpServiceProvider = new BKHttpServiceProvider.Builder()
             .setBookieServer(serverByIndex(numberOfBookies - 1))
             .setServerConfiguration(baseConf)
+            .setLedgerManagerFactory(metadataDriver.getLedgerManagerFactory())
             .build();
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java
index c83fb6c..5344480 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java
@@ -33,6 +33,8 @@ import java.util.Base64;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
+import org.apache.bookkeeper.bookie.BookieResources;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
@@ -40,7 +42,9 @@ import org.apache.bookkeeper.http.HttpServer;
 import org.apache.bookkeeper.http.service.HttpServiceRequest;
 import org.apache.bookkeeper.http.service.HttpServiceResponse;
 import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.StatsProvider;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.apache.bookkeeper.test.TestStatsProvider;
 import org.apache.commons.lang3.RandomUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -60,7 +64,10 @@ public class ListLedgerServiceTest extends BookKeeperClusterTestCase {
     @Before
     public void setUp() throws Exception {
         super.setUp();
-        listLedgerService = new ListLedgerService(confByIndex(0), serverByIndex(0));
+        StatsProvider provider = new TestStatsProvider();
+        listLedgerService = new ListLedgerService(confByIndex(0),
+                BookieResources.createMetadataDriver(confByIndex(0),
+                        provider.getStatsLogger("")).getLedgerManagerFactory());
     }
 
     @Test
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
index 21c636c..bb66b7a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
@@ -21,11 +21,13 @@
 
 package org.apache.bookkeeper.test;
 
-
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
 import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE;
 import static org.junit.Assert.assertFalse;
+
 import com.google.common.base.Stopwatch;
-import io.netty.buffer.ByteBufAllocator;
 import java.io.File;
 import java.io.IOException;
 import java.net.UnknownHostException;
@@ -40,18 +42,27 @@ import java.util.concurrent.Future;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
-import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.metastore.InMemoryMetaStore;
 import org.apache.bookkeeper.net.BookieId;
@@ -59,9 +70,10 @@ import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.replication.Auditor;
 import org.apache.bookkeeper.replication.AutoRecoveryMain;
-import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.server.Main;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.PortManager;
-import org.apache.commons.io.FileUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.junit.After;
@@ -91,7 +103,7 @@ public abstract class BookKeeperClusterTestCase {
     protected String metadataServiceUri;
 
     // BookKeeper related variables
-    protected final List<File> tmpDirs = new LinkedList<>();
+    protected final TmpDirs tmpDirs = new TmpDirs();
     private final List<ServerTester> servers = new LinkedList<>();
 
     protected int numBookies;
@@ -105,6 +117,7 @@ public abstract class BookKeeperClusterTestCase {
      */
     protected final ServerConfiguration baseConf = TestBKConfiguration.newServerConfiguration();
     protected final ClientConfiguration baseClientConf = TestBKConfiguration.newClientConfiguration();
+    private final ByteBufAllocatorWithOomHandler allocator = BookieResources.createAllocator(baseConf);
 
     private boolean isAutoRecoveryEnabled;
 
@@ -197,7 +210,7 @@ public abstract class BookKeeperClusterTestCase {
         }
         // cleanup temp dirs
         try {
-            cleanupTempDirs();
+            tmpDirs.cleanup();
         } catch (Exception e) {
             LOG.error("Got Exception while trying to cleanupTempDirs", e);
             tearDownException = e;
@@ -208,12 +221,6 @@ public abstract class BookKeeperClusterTestCase {
         }
     }
 
-    protected File createTempDir(String prefix, String suffix) throws IOException {
-        File dir = IOUtils.createTempDir(prefix, suffix);
-        tmpDirs.add(dir);
-        return dir;
-    }
-
     /**
      * Start zookeeper cluster.
      *
@@ -270,14 +277,8 @@ public abstract class BookKeeperClusterTestCase {
         servers.clear();
     }
 
-    protected void cleanupTempDirs() throws Exception {
-        for (File f : tmpDirs) {
-            FileUtils.deleteDirectory(f);
-        }
-    }
-
     protected ServerConfiguration newServerConfiguration() throws Exception {
-        File f = createTempDir("bookie", "test");
+        File f = tmpDirs.createNew("bookie", "test");
 
         int port;
         if (baseConf.isEnableLocalTransport() || !baseConf.getAllowEphemeralPorts()) {
@@ -798,18 +799,55 @@ public abstract class BookKeeperClusterTestCase {
     /**
      * Class to encapsulate all the test objects.
      */
-    public static class ServerTester {
+    public class ServerTester {
         private final ServerConfiguration conf;
         private final TestStatsProvider provider;
+        private final Bookie bookie;
         private final BookieServer server;
         private final BookieSocketAddress address;
+        private final MetadataBookieDriver metadataDriver;
+        private final RegistrationManager registrationManager;
+        private final LedgerManagerFactory lmFactory;
+        private final LedgerManager ledgerManager;
+        private final LedgerStorage storage;
+
         private AutoRecoveryMain autoRecovery;
 
         ServerTester(ServerConfiguration conf) throws Exception {
             this.conf = conf;
             provider = new TestStatsProvider();
 
-            server = new BookieServer(conf, provider.getStatsLogger(""), null);
+            StatsLogger rootStatsLogger = provider.getStatsLogger("");
+            StatsLogger bookieStats = rootStatsLogger.scope(BOOKIE_SCOPE);
+
+            metadataDriver = BookieResources.createMetadataDriver(conf, bookieStats);
+            registrationManager = metadataDriver.createRegistrationManager();
+            lmFactory = metadataDriver.getLedgerManagerFactory();
+            ledgerManager = lmFactory.newLedgerManager();
+
+            LegacyCookieValidation cookieValidation = new LegacyCookieValidation(
+                    conf, registrationManager);
+            cookieValidation.checkCookies(Main.storageDirectoriesFromConf(conf));
+
+            DiskChecker diskChecker = BookieResources.createDiskChecker(conf);
+            LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
+                    conf, diskChecker, bookieStats.scope(LD_LEDGER_SCOPE));
+            LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
+                    conf, diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
+            storage = BookieResources.createLedgerStorage(
+                    conf, ledgerManager, ledgerDirsManager, indexDirsManager,
+                    bookieStats, allocator);
+
+            if (conf.isForceReadOnlyBookie()) {
+                bookie = new ReadOnlyBookie(conf, registrationManager, storage,
+                                            diskChecker, ledgerDirsManager, indexDirsManager,
+                                            bookieStats, allocator, BookieServiceInfo.NO_INFO);
+            } else {
+                bookie = new BookieImpl(conf, registrationManager, storage,
+                                        diskChecker, ledgerDirsManager, indexDirsManager,
+                                        bookieStats, allocator, BookieServiceInfo.NO_INFO);
+            }
+            server = new BookieServer(conf, bookie, rootStatsLogger, allocator);
             address = BookieImpl.getBookieAddress(conf);
 
             autoRecovery = null;
@@ -819,15 +857,14 @@ public abstract class BookKeeperClusterTestCase {
             this.conf = conf;
             provider = new TestStatsProvider();
 
-            server = new BookieServer(conf, provider.getStatsLogger(""), null) {
-                    @Override
-                    protected Bookie newBookie(ServerConfiguration conf,
-                                               ByteBufAllocator allocator,
-                                               Supplier<BookieServiceInfo> bookieServiceInfoProvider) {
-                        return b;
-                    }
-                };
+            metadataDriver = null;
+            registrationManager = null;
+            ledgerManager = null;
+            lmFactory = null;
+            storage = null;
 
+            bookie = b;
+            server = new BookieServer(conf, b, provider.getStatsLogger(""), allocator);
             address = BookieImpl.getBookieAddress(conf);
 
             autoRecovery = null;
@@ -873,6 +910,19 @@ public abstract class BookKeeperClusterTestCase {
         void shutdown() throws Exception {
             server.shutdown();
 
+            if (ledgerManager != null) {
+                ledgerManager.close();
+            }
+            if (lmFactory != null) {
+                lmFactory.close();
+            }
+            if (registrationManager != null) {
+                registrationManager.close();
+            }
+            if (metadataDriver != null) {
+                metadataDriver.close();
+            }
+
             if (autoRecovery != null) {
                 LOG.debug("Shutdown auto recovery for bookieserver: {}", address);
                 autoRecovery.shutdown();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
index 81376fb..eca8856 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
@@ -38,6 +38,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 
+import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.BookKeeperClientStats;
@@ -90,7 +91,9 @@ public class BookieClientTest {
             .setLedgerDirNames(new String[] { tmpDir.getPath() })
             .setMetadataServiceUri(null);
 
-        bs = new BookieServer(conf);
+        bs = new BookieServer(
+                conf, new TestBookieImpl(conf),
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
         bs.start();
         eventLoopGroup = new NioEventLoopGroup();
         executor = OrderedExecutor.newBuilder()
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
index 3751401..6b4e1ad 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
@@ -25,11 +25,15 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import io.netty.buffer.UnpooledByteBufAllocator;
+
 import java.io.File;
 import java.util.HashSet;
 
+import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.util.PortManager;
 import org.junit.Test;
 
@@ -50,7 +54,7 @@ public class BookieZKExpireTest extends BookKeeperClusterTestCase {
     public void testBookieServerZKExpireBehaviour() throws Exception {
         BookieServer server = null;
         try {
-            File f = createTempDir("bookieserver", "test");
+            File f = tmpDirs.createNew("bookieserver", "test");
 
             HashSet<Thread> threadset = new HashSet<Thread>();
             int threadCount = Thread.activeCount();
@@ -63,7 +67,9 @@ public class BookieZKExpireTest extends BookKeeperClusterTestCase {
             }
 
             ServerConfiguration conf = newServerConfiguration(PortManager.nextFreePort(), f, new File[] { f });
-            server = new BookieServer(conf);
+            server = new BookieServer(
+                    conf, new TestBookieImpl(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
             server.start();
 
             int secondsToWait = 5;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java
index 68ab21b..65a7035 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java
@@ -23,6 +23,7 @@ package org.apache.bookkeeper.test;
 import static org.junit.Assert.assertFalse;
 
 import java.io.File;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -122,6 +123,7 @@ public class LedgerDeleteTest extends BookKeeperClusterTestCase {
         // restart bookies to force rolling entry log files
         restartBookies();
 
+        List<File> ledgerDirectories = bookieLedgerDirs();
         // Delete all of these ledgers from the BookKeeper client
         for (LedgerHandle lh : lhs) {
             bkc.deleteLedger(lh.getId());
@@ -130,7 +132,7 @@ public class LedgerDeleteTest extends BookKeeperClusterTestCase {
         Thread.sleep(2000);
 
         // Verify that the first entry log (0.log) has been deleted from all of the Bookie Servers.
-        for (File ledgerDirectory : bookieLedgerDirs()) {
+        for (File ledgerDirectory : ledgerDirectories) {
             assertFalse("Found the entry log file (0.log) that should have been deleted in ledgerDirectory: "
                 + ledgerDirectory, TestUtils.hasLogFiles(ledgerDirectory, true, 0));
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java
index 705098a..e1270d2 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java
@@ -241,7 +241,7 @@ public class ReadOnlyBookieTest extends BookKeeperClusterTestCase {
 
         File[] ledgerDirs = new File[numOfLedgerDirs];
         for (int i = 0; i < numOfLedgerDirs; i++) {
-            File dir = createTempDir("bookie", "test");
+            File dir = tmpDirs.createNew("bookie", "test");
             ledgerDirs[i] = dir;
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TmpDirs.java
similarity index 52%
copy from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java
copy to bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TmpDirs.java
index 751f99b..7b8d23e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestBookieImpl.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TmpDirs.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -18,20 +18,33 @@
  * under the License.
  *
  */
-package org.apache.bookkeeper.bookie;
+package org.apache.bookkeeper.test;
 
-import io.netty.buffer.PooledByteBufAllocator;
-import java.io.IOException;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.proto.SimpleBookieServiceInfoProvider;
-import org.apache.bookkeeper.stats.NullStatsLogger;
+import java.io.File;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.io.FileUtils;
 
 /**
- * Test wrapper for BookieImpl that chooses defaults for dependencies.
+ * Utility class for managing tmp directories in tests.
  */
-public class TestBookieImpl extends BookieImpl {
-    public TestBookieImpl(ServerConfiguration conf) throws IOException, InterruptedException, BookieException {
-        super(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
-                new SimpleBookieServiceInfoProvider(conf));
+public class TmpDirs {
+    private final List<File> tmpDirs = new LinkedList<>(); // retained to delete files
+
+    public File createNew(String prefix, String suffix) throws Exception {
+        File dir = IOUtils.createTempDir(prefix, suffix);
+        tmpDirs.add(dir);
+        return dir;
+    }
+
+    public void cleanup() throws Exception {
+        for (File f : tmpDirs) {
+            FileUtils.deleteDirectory(f);
+        }
+    }
+
+    public List<File> getDirs() {
+        return tmpDirs;
     }
-}
+}
\ No newline at end of file
diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java
index 947ee4c..c808df5 100644
--- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java
+++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java
@@ -53,7 +53,7 @@ class EtcdBookieRegister implements AutoCloseable, Runnable, Supplier<Long> {
     private final Lease leaseClient;
     private final long ttlSeconds;
     private final ScheduledExecutorService executor;
-    private final RegistrationListener regListener;
+    private RegistrationListener regListener;
     private volatile CompletableFuture<Long> leaseFuture = new CompletableFuture<>();
     private volatile CompletableFuture<Void> keepAliveFuture = new CompletableFuture<>();
 
@@ -65,9 +65,7 @@ class EtcdBookieRegister implements AutoCloseable, Runnable, Supplier<Long> {
     private Future<?> runFuture = null;
 
     EtcdBookieRegister(Lease leaseClient,
-                       long ttlSeconds,
-                       RegistrationListener regListener) {
-        this.regListener = regListener;
+                       long ttlSeconds) {
         this.leaseClient = leaseClient;
         this.ttlSeconds = ttlSeconds;
         this.executor = Executors.newSingleThreadScheduledExecutor(
@@ -76,6 +74,11 @@ class EtcdBookieRegister implements AutoCloseable, Runnable, Supplier<Long> {
                 .build());
     }
 
+    public EtcdBookieRegister addRegistrationListener(RegistrationListener regListener) {
+        this.regListener = regListener;
+        return this;
+    }
+
     long getTtlSeconds() {
         return ttlSeconds;
     }
diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java
index 764018b..4cbb352 100644
--- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java
+++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java
@@ -18,7 +18,6 @@ import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
@@ -40,27 +39,23 @@ public class EtcdMetadataBookieDriver extends EtcdMetadataDriverBase implements
     ServerConfiguration conf;
     EtcdBookieRegister bkRegister;
     RegistrationManager regMgr;
-    RegistrationListener listener;
 
     @Override
     public synchronized MetadataBookieDriver initialize(ServerConfiguration conf,
-                                                        RegistrationListener listener,
                                                         StatsLogger statsLogger)
             throws MetadataException {
         super.initialize(conf, statsLogger);
         this.conf = conf;
-        this.listener = listener;
         this.statsLogger = statsLogger;
         return null;
     }
 
     @Override
-    public synchronized RegistrationManager getRegistrationManager() {
+    public synchronized RegistrationManager createRegistrationManager() {
         if (null == bkRegister) {
             bkRegister = new EtcdBookieRegister(
                 client.getLeaseClient(),
-                TimeUnit.MILLISECONDS.toSeconds(conf.getZkTimeout()),
-                listener
+                TimeUnit.MILLISECONDS.toSeconds(conf.getZkTimeout())
             ).start();
         }
         if (null == regMgr) {
diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java
index 55f1121..f133ecd 100644
--- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java
+++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java
@@ -114,9 +114,8 @@ class EtcdRegistrationManager implements RegistrationManager {
             scope,
             new EtcdBookieRegister(
                 client.getLeaseClient(),
-                ttlSeconds,
-                listener
-            ).start(),
+                ttlSeconds
+            ).addRegistrationListener(listener).start(),
             true);
     }
 
@@ -523,6 +522,11 @@ class EtcdRegistrationManager implements RegistrationManager {
         return nukeExistingCluster(kvClient, scope);
     }
 
+    @Override
+    public void addRegistrationListener(RegistrationListener listener) {
+        bkRegister.addRegistrationListener(listener);
+    }
+
     static boolean nukeExistingCluster(KV kvClient, String scope) throws Exception {
         ByteSequence rootScopeKey = ByteSequence.from(scope, UTF_8);
         GetResponse resp = msResult(kvClient.get(rootScopeKey));
diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
index 0c3cebf..ee4cf65 100644
--- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
+++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
@@ -16,18 +16,20 @@ package org.apache.bookkeeper.metadata.etcd.testing;
 
 import static org.junit.Assert.assertTrue;
 
+import io.netty.buffer.ByteBufAllocator;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.api.BookKeeper;
 import org.apache.bookkeeper.common.net.ServiceURI;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
-import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.metadata.etcd.EtcdMetadataBookieDriver;
 import org.apache.bookkeeper.metadata.etcd.EtcdMetadataClientDriver;
@@ -121,8 +123,9 @@ public abstract class EtcdBKClusterTestBase extends EtcdTestBase {
     private static BookieServer startBookie(ServerConfiguration conf) throws Exception {
         conf.setAutoRecoveryDaemonEnabled(true);
         TestStatsProvider provider = new TestStatsProvider();
-        BookieServer server = new BookieServer(conf, provider.getStatsLogger(""),
-                                               BookieServiceInfo.NO_INFO);
+        Bookie bookie = new TestBookieImpl(conf);
+        BookieServer server = new BookieServer(conf, bookie, provider.getStatsLogger(""),
+                                               ByteBufAllocator.DEFAULT);
         server.start();
         return server;
     }
diff --git a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java
index 9354440..1d86e49 100644
--- a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java
+++ b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java
@@ -38,6 +38,7 @@ import java.util.stream.Collectors;
 import lombok.Cleanup;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.net.BookieId;
@@ -60,6 +61,7 @@ public class BKRegistrationNameResolverTest extends BookKeeperClusterTestCase {
     private final BKRegistrationNameResolverProvider resolverProvider;
 
     private MetadataBookieDriver bookieDriver;
+    private RegistrationManager regManager;
     private URI serviceUri;
 
     public BKRegistrationNameResolverTest() {
@@ -84,12 +86,14 @@ public class BKRegistrationNameResolverTest extends BookKeeperClusterTestCase {
         ServerConfiguration serverConf = new ServerConfiguration();
         serverConf.setMetadataServiceUri(serviceUri.toString());
         bookieDriver = MetadataDrivers.getBookieDriver(serviceUri);
-        bookieDriver.initialize(serverConf, () -> {}, NullStatsLogger.INSTANCE);
+        bookieDriver.initialize(serverConf, NullStatsLogger.INSTANCE);
+        regManager = bookieDriver.createRegistrationManager();
     }
 
     @After
     @Override
     public void tearDown() throws Exception {
+        regManager.close();
         bookieDriver.close();
 
         super.tearDown();
@@ -103,7 +107,7 @@ public class BKRegistrationNameResolverTest extends BookKeeperClusterTestCase {
         for (int i = 0; i < numServers; i++) {
             InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
             addressSet.add(address);
-            bookieDriver.getRegistrationManager().registerBookie(
+            bookieDriver.createRegistrationManager().registerBookie(
                 BookieId.parse("127.0.0.1:" + (3181 + i)), false, BookieServiceInfo.EMPTY
             );
         }
@@ -138,7 +142,7 @@ public class BKRegistrationNameResolverTest extends BookKeeperClusterTestCase {
         for (int i = numServers; i < 2 * numServers; i++) {
             InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
             addressSet.add(address);
-            bookieDriver.getRegistrationManager().registerBookie(
+            regManager.registerBookie(
                 BookieId.parse("127.0.0.1:" + (3181 + i)), false, BookieServiceInfo.EMPTY
             );
         }
diff --git a/stream/distributedlog/core/build.gradle b/stream/distributedlog/core/build.gradle
index a3ee41b..ee9a79f 100644
--- a/stream/distributedlog/core/build.gradle
+++ b/stream/distributedlog/core/build.gradle
@@ -47,6 +47,7 @@ dependencies {
     testRuntimeOnly depLibs.snappy
 
     testImplementation project(path: ':bookkeeper-common', configuration: 'testArtifacts')
+    testImplementation project(path: ':bookkeeper-server', configuration: 'testArtifacts')
     testImplementation project(path: ':stream:distributedlog:common', configuration: 'testArtifacts')
     testImplementation depLibs.junit
     testImplementation depLibs.mockito
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
index 196caeb..e33f02b 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
@@ -67,6 +67,7 @@ public class LocalDLMEmulator {
     private final String zkHost;
     private final int zkPort;
     private final int numBookies;
+    private final LocalBookKeeper lb;
 
     /**
      * Builder to build LocalDLMEmulator.
@@ -76,7 +77,6 @@ public class LocalDLMEmulator {
         private int numBookies = DEFAULT_NUM_BOOKIES;
         private String zkHost = DEFAULT_ZK_HOST;
         private int zkPort = DEFAULT_ZK_PORT;
-        private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT;
         private boolean shouldStartZK = true;
         private Optional<ServerConfiguration> serverConf = Optional.empty();
 
@@ -96,10 +96,6 @@ public class LocalDLMEmulator {
             this.zkTimeoutSec = zkTimeoutSec;
             return this;
         }
-        public Builder initialBookiePort(int initialBookiePort) {
-            this.initialBookiePort = initialBookiePort;
-            return this;
-        }
         public Builder shouldStartZK(boolean shouldStartZK) {
             this.shouldStartZK = shouldStartZK;
             return this;
@@ -122,7 +118,7 @@ public class LocalDLMEmulator {
             newConf.setAllowLoopback(true);
 
             return new LocalDLMEmulator(numBookies, shouldStartZK, zkHost, zkPort,
-                initialBookiePort, zkTimeoutSec, newConf);
+                zkTimeoutSec, newConf);
         }
     }
 
@@ -131,7 +127,7 @@ public class LocalDLMEmulator {
     }
 
     private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK,
-                             final String zkHost, final int zkPort, final int initialBookiePort,
+                             final String zkHost, final int zkPort,
                              final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
         this.numBookies = numBookies;
         this.zkHost = zkHost;
@@ -139,13 +135,21 @@ public class LocalDLMEmulator {
         this.zkEnsemble = zkHost + ":" + zkPort;
         this.uri = URI.create("distributedlog://" + zkEnsemble + DLOG_NAMESPACE);
         this.zkTimeoutSec = zkTimeoutSec;
+        this.lb = LocalBookKeeper.getLocalBookies(zkHost, zkPort,
+                numBookies, shouldStartZK, serverConf);
         this.bkStartupThread = new Thread() {
             public void run() {
                 try {
-                    LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
-                    LocalBookKeeper.startLocalBookies(zkHost, zkPort,
-                            numBookies, shouldStartZK, initialBookiePort, serverConf);
-                    LOG.info("{} bookies are started.", numBookies);
+                    try {
+                        LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
+                        lb.start();
+                        LOG.info("{} bookies are started.", numBookies);
+                        while (true) {
+                            Thread.sleep(1000);
+                        }
+                    } finally {
+                        lb.close();
+                    }
                 } catch (InterruptedException e) {
                     Thread.currentThread().interrupt();
                     // go away quietly
@@ -229,6 +233,14 @@ public class LocalDLMEmulator {
         }
     }
 
+    public void addBookie() throws Exception {
+        lb.addBookie();
+    }
+
+    public void removeBookie() throws Exception {
+        lb.removeBookie();
+    }
+
     public static String getBkLedgerPath() {
         return "/ledgers";
     }
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java
index 5e0a514..464afc2 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java
@@ -17,11 +17,13 @@
  */
 package org.apache.distributedlog;
 
+import io.netty.buffer.UnpooledByteBufAllocator;
+
 import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.stats.NullStatsProvider;
 import org.junit.Assert;
@@ -46,46 +48,41 @@ public class TestTxnId extends TestDistributedLogBase {
             .setLogSegmentRollingConcurrency(-1)
             .setMaxLogSegmentBytes(400000);
 
-        long entryId = 0;
-        List<BookieServer> extraBookies = new ArrayList<>();
-        try {
-            extraBookies.add(startExtraBookie());
-            extraBookies.add(startExtraBookie());
-
-            try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-                 BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) {
-                writer.write(DLMTestUtil.getLogRecordInstance(1, 100000)).join();
-                writer.write(DLMTestUtil.getLogRecordInstance(2, 100000)).join();
+        bkutil.addBookie();
+        bkutil.addBookie();
 
-                extraBookies.forEach(b -> b.shutdown());
+        try (BKDistributedLogManager dlm = createNewDLM(conf, name);
+             BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) {
+            writer.write(DLMTestUtil.getLogRecordInstance(1, 100000)).join();
+            writer.write(DLMTestUtil.getLogRecordInstance(2, 100000)).join();
 
-                try {
-                    writer.write(DLMTestUtil.getLogRecordInstance(3, 100000)).join();
-                    Assert.fail("Shouldn't have succeeded");
-                } catch (Exception e) {
-                    // expected
-                }
+            bkutil.removeBookie();
+            bkutil.removeBookie();
 
-                writer.write(DLMTestUtil.getLogRecordInstance(4, 100000)).join();
-                Assert.fail("Shouldn't be able to write");
+            try {
+                writer.write(DLMTestUtil.getLogRecordInstance(3, 100000)).join();
+                Assert.fail("Shouldn't have succeeded");
             } catch (Exception e) {
                 // expected
             }
 
-            extraBookies.add(startExtraBookie());
-            extraBookies.add(startExtraBookie());
-
-            try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-                 BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) {
-                long firstTxid = dlm.getLastTxId() + 1;
-                for (int i = 0; i < 20; i++) {
-                    logger.info("Writing entry {}", i);
-                    writer.write(DLMTestUtil.getLogRecordInstance(firstTxid + i, 100000)).join();
-                    Thread.sleep(100);
-                }
+            writer.write(DLMTestUtil.getLogRecordInstance(4, 100000)).join();
+            Assert.fail("Shouldn't be able to write");
+        } catch (Exception e) {
+            // expected
+        }
+
+        bkutil.addBookie();
+        bkutil.addBookie();
+
+        try (BKDistributedLogManager dlm = createNewDLM(conf, name);
+             BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) {
+            long firstTxid = dlm.getLastTxId() + 1;
+            for (int i = 0; i < 20; i++) {
+                logger.info("Writing entry {}", i);
+                writer.write(DLMTestUtil.getLogRecordInstance(firstTxid + i, 100000)).join();
+                Thread.sleep(100);
             }
-        } finally {
-            extraBookies.forEach(b -> b.shutdown());
         }
     }
 
@@ -108,8 +105,10 @@ public class TestTxnId extends TestDistributedLogBase {
         conf.setJournalDirName(journalDir.getPath());
         conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
 
-        BookieServer server = new BookieServer(conf, new NullStatsProvider().getStatsLogger(""),
-                                               BookieServiceInfo.NO_INFO);
+        Bookie bookie = new TestBookieImpl(conf);
+        BookieServer server = new BookieServer(conf, bookie,
+                                               new NullStatsProvider().getStatsLogger(""),
+                                               UnpooledByteBufAllocator.DEFAULT);
         server.start();
 
         while (!server.isRunning()) {
diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
index 5490dee..f1f28d4 100644
--- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
+++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
@@ -39,6 +39,7 @@ import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.common.component.LifecycleComponent;
 import org.apache.bookkeeper.common.net.ServiceURI;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -143,18 +144,18 @@ public class StreamCluster
 
         // format the bookkeeper cluster
         MetadataDrivers.runFunctionWithMetadataBookieDriver(newBookieConfiguration(metadataServiceUri), driver -> {
-            try {
-                boolean initialized = driver.getRegistrationManager().initNewCluster();
-                if (initialized) {
-                    log.info("Successfully initialized the segment storage");
-                } else {
-                    log.info("The segment storage was already initialized");
+                try (RegistrationManager rm = driver.createRegistrationManager()) {
+                    boolean initialized = rm.initNewCluster();
+                    if (initialized) {
+                        log.info("Successfully initialized the segment storage");
+                    } else {
+                        log.info("The segment storage was already initialized");
+                    }
+                } catch (Exception e) {
+                    throw new StorageRuntimeException("Failed to initialize the segment storage", e);
                 }
-            } catch (Exception e) {
-                throw new StorageRuntimeException("Failed to initialize the segment storage", e);
-            }
-            return null;
-        });
+                return null;
+            });
     }
 
     private LifecycleComponent startServer() throws Exception {
diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
index f9b655c..88e5392 100644
--- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
+++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
@@ -13,6 +13,12 @@
  */
 package org.apache.bookkeeper.stream.server.service;
 
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+
+import io.netty.buffer.ByteBufAllocator;
+
 import java.io.IOException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -22,14 +28,27 @@ import java.util.function.Supplier;
 import lombok.Getter;
 import lombok.experimental.Accessors;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
 import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.server.Main;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.stream.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.util.DiskChecker;
 
 /**
  * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs
@@ -42,12 +61,18 @@ public class BookieService extends AbstractLifecycleComponent<BookieConfiguratio
     @Getter
     private final ServerConfiguration serverConf;
     private BookieServer bs;
+    private MetadataBookieDriver metadataDriver;
+    private RegistrationManager rm;
+    private LedgerManagerFactory lmFactory;
+    private LedgerManager ledgerManager;
+    private Supplier<BookieServiceInfo> bookieServiceInfoProvider;
 
     public BookieService(BookieConfiguration conf, StatsLogger statsLogger,
                          Supplier<BookieServiceInfo> bookieServiceInfoProvider) throws Exception {
         super("bookie-server", conf, statsLogger);
         this.serverConf = new ServerConfiguration();
         this.serverConf.loadConf(conf.getUnderlyingConf());
+        this.bookieServiceInfoProvider = bookieServiceInfoProvider;
         String hello = String.format(
             "Hello, I'm your bookie, bookieId is %1$s, listening on port %2$s. Metadata service uri is %3$s."
                 + " Journals are in %4$s. Ledgers are stored in %5$s.",
@@ -56,8 +81,44 @@ public class BookieService extends AbstractLifecycleComponent<BookieConfiguratio
             serverConf.getMetadataServiceUriUnchecked(),
             Arrays.asList(serverConf.getJournalDirNames()),
             Arrays.asList(serverConf.getLedgerDirNames()));
+
+        ByteBufAllocator allocator = BookieResources.createAllocator(serverConf);
+
+        this.metadataDriver = BookieResources.createMetadataDriver(
+                serverConf, statsLogger);
+        StatsLogger bookieStats = statsLogger.scope(BOOKIE_SCOPE);
+        this.rm = this.metadataDriver.createRegistrationManager();
+        this.lmFactory = this.metadataDriver.getLedgerManagerFactory();
+        this.ledgerManager = this.lmFactory.newLedgerManager();
+
+        DiskChecker diskChecker = BookieResources.createDiskChecker(serverConf);
+        LedgerDirsManager ledgerDirsManager = BookieResources.createLedgerDirsManager(
+                serverConf, diskChecker, bookieStats.scope(LD_LEDGER_SCOPE));
+        LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
+                serverConf, diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
+        LedgerStorage storage = BookieResources.createLedgerStorage(
+                serverConf, ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+
+        LegacyCookieValidation cookieValidation = new LegacyCookieValidation(serverConf, rm);
+        cookieValidation.checkCookies(Main.storageDirectoriesFromConf(serverConf));
+
+        Bookie bookie;
+        if (serverConf.isForceReadOnlyBookie()) {
+            bookie = new ReadOnlyBookie(serverConf, rm, storage, diskChecker,
+                    ledgerDirsManager, indexDirsManager,
+                    statsLogger.scope(BOOKIE_SCOPE),
+                    allocator, bookieServiceInfoProvider);
+        } else {
+            bookie = new BookieImpl(serverConf, rm, storage, diskChecker,
+                    ledgerDirsManager, indexDirsManager,
+                    statsLogger.scope(BOOKIE_SCOPE),
+                    allocator, bookieServiceInfoProvider);
+        }
+
+        this.bs = new BookieServer(serverConf, bookie,
+                statsLogger, allocator);
+
         log.info(hello);
-        this.bs = new BookieServer(serverConf, statsLogger, bookieServiceInfoProvider);
     }
 
     @Override
@@ -75,6 +136,26 @@ public class BookieService extends AbstractLifecycleComponent<BookieConfiguratio
         if (null != bs) {
             bs.shutdown();
         }
+        if (rm != null) {
+            rm.close();
+        }
+        if (ledgerManager != null) {
+            try {
+                ledgerManager.close();
+            } catch (Exception e) {
+                log.error("Error shutting down ledger manager", e);
+            }
+        }
+        if (lmFactory != null) {
+            try {
+                lmFactory.close();
+            } catch (Exception e) {
+                log.error("Error shutting down ledger manager factory", e);
+            }
+        }
+        if (null != metadataDriver) {
+            metadataDriver.close();
+        }
     }
 
     @Override
diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java
index 8a47b8c..afe33ac 100644
--- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java
+++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java
@@ -65,8 +65,8 @@ public class RegistrationStateService
             regManager = new ZKRegistrationManager(
                 bkServerConf,
                 regServiceProvider.getZkClient(),
-                regServiceProvider.getRegistrationPath(),
-                () -> {
+                regServiceProvider.getRegistrationPath());
+            regManager.addRegistrationListener(() -> {
                     if (null == stateManager) {
                         log.warn("Registration state manager is not initialized yet");
                         return;
@@ -79,7 +79,7 @@ public class RegistrationStateService
                 stateManager = new BookieStateManager(
                     bkServerConf,
                     statsLogger.scope("state"),
-                    () -> regManager,
+                    regManager,
                     Collections.emptyList(),
                     () -> BookieId.parse(NetUtils.endpointToString(myEndpoint)),
                     BookieServiceInfo.NO_INFO);
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java
index 6849ed1..8adaf92 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java
@@ -39,8 +39,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Vector;
 import java.util.stream.LongStream;
-import org.apache.bookkeeper.bookie.CheckpointSource;
-import org.apache.bookkeeper.bookie.Checkpointer;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerCache;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -145,12 +143,10 @@ public class ConvertToInterleavedStorageCommandTest extends BookieCommandTestBas
             verifyNew(InterleavedLedgerStorage.class, times(1)).withNoArguments();
 
             verify(dbStorage, times(1)).initialize(eq(conf), eq(null), any(LedgerDirsManager.class),
-                any(LedgerDirsManager.class), eq(null), any(CheckpointSource.class),
-                any(Checkpointer.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT));
+                any(LedgerDirsManager.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT));
             verify(interleavedLedgerStorage, times(1))
                 .initialize(eq(conf), eq(null), any(LedgerDirsManager.class),
-                    any(LedgerDirsManager.class), eq(null), any(CheckpointSource.class),
-                    any(Checkpointer.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT));
+                    any(LedgerDirsManager.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT));
             verify(dbStorage, times(1)).getActiveLedgersInRange(anyLong(), anyLong());
             verify(dbStorage, times(10)).readMasterKey(anyLong());
             verify(interleavedLedgerStorage, times(10)).setMasterKey(anyLong(), any());
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java
index a530ff5..9444f48 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java
@@ -47,6 +47,7 @@ import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
@@ -97,7 +98,6 @@ public class AdminCommandTest extends BookieCommandTestBase {
         PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class)
                     .withArguments(eq(conf)).thenReturn(serverConfiguration);
         PowerMockito.mockStatic(Cookie.class);
-        PowerMockito.mockStatic(MetadataDrivers.class);
         PowerMockito.mockStatic(Bookie.class);
         PowerMockito.mockStatic(BookieImpl.class);
 
@@ -161,12 +161,15 @@ public class AdminCommandTest extends BookieCommandTestBase {
 
     private void mockExpandStorage() throws Exception {
         MetadataBookieDriver metadataBookieDriver = mock(MetadataBookieDriver.class);
+        RegistrationManager registrationManager = mock(RegistrationManager.class);
         PowerMockito.doAnswer(invocationOnMock -> {
             Function<MetadataBookieDriver, ?> f = invocationOnMock.getArgument(1);
             f.apply(metadataBookieDriver);
             return true;
         }).when(MetadataDrivers.class, "runFunctionWithMetadataBookieDriver", any(ServerConfiguration.class),
                 any(Function.class));
+
+        when(metadataBookieDriver.createRegistrationManager()).thenReturn(registrationManager);
     }
 
     private void mockListOrDeleteCookies() throws UnknownHostException {
@@ -190,6 +193,7 @@ public class AdminCommandTest extends BookieCommandTestBase {
         verify(cookie, times(2)).verify(any(Cookie.class));
     }
 
+    @Ignore
     @Test
     public void testWithExpand() {
         testCommand("-e");