Turn Bookie into an interface


### Motivation

Bookie was previously a concrete class that was used and abused all
over the place, especially in tests. A classic example of the God
object antipattern. The extensive use in tests, resulted in test cases
which spin up many instances of the whole system, which is very heavy
and very slow, especially when trying to unit tests a particular
feature.

This change is the first step to resolving this situation. Bookie is
now an interface, implemented by BookieImpl. Subsequent changes will
break out parts of the interface, cleanup calls and add dependency
injection.

Reviewers: Matteo Merli <mmerli@apache.org>, Andrey Yegorov, Henry Saputra <hsaputra@apache.org>, Enrico Olivelli <eolivelli@gmail.com>

This closes #2717 from pkumar-singh/merge_back_to_oss and squashes the following commits:

3edee4958 [Prashant] Replace SettableFuture  with CompletableFuture
db690262a [Ivan Kelly] Turn Bookie into an interface
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 28c76f6..39794c2 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
@@ -1,5 +1,4 @@
-/**
- *
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,158 +15,86 @@
  * 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.JOURNAL_MEMORY_MAX;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_MEMORY_USED;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.SettableFuture;
-
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
-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.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;
-import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
-import org.apache.bookkeeper.bookie.stats.BookieStats;
-import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
+import java.util.PrimitiveIterator;
+import java.util.concurrent.CompletableFuture;
 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;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
-import org.apache.bookkeeper.proto.SimpleBookieServiceInfoProvider;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.annotations.StatsDoc;
-import org.apache.bookkeeper.util.BookKeeperConstants;
-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;
 
 /**
- * Implements a bookie.
+ * Interface for the bookie.
  */
-public class Bookie extends BookieCriticalThread {
+public interface Bookie {
 
-    private static final Logger LOG = LoggerFactory.getLogger(Bookie.class);
+    void start();
+    void join() throws InterruptedException;
+    boolean isRunning();
+    int getExitCode();
+    int shutdown();
 
-    final List<File> journalDirectories;
-    final ServerConfiguration conf;
+    boolean isAvailableForHighPriorityWrites();
+    boolean isReadOnly();
 
-    final SyncThread syncThread;
-    final LedgerManagerFactory ledgerManagerFactory;
-    final LedgerManager ledgerManager;
-    final LedgerStorage ledgerStorage;
-    final List<Journal> journals;
+    // TODO: replace callback with futures
+    // TODO: replace ackBeforeSync with flags
+    void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
+            throws IOException, BookieException, InterruptedException;
+    void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
+            throws IOException, BookieException, InterruptedException;
+    void forceLedger(long ledgerId, WriteCallback cb, Object ctx);
+    void setExplicitLac(ByteBuf entry, WriteCallback writeCallback, Object ctx, byte[] masterKey)
+            throws IOException, InterruptedException, BookieException;
+    ByteBuf getExplicitLac(long ledgerId) throws IOException, NoLedgerException;
 
-    final HandleFactory handles;
-    final boolean entryLogPerLedgerEnabled;
+    // these can probably be moved out and called directly on ledgerdirmanager
+    long getTotalDiskSpace() throws IOException;
+    long getTotalFreeSpace() throws IOException;
 
-    public static final long METAENTRY_ID_LEDGER_KEY = -0x1000;
-    public static final long METAENTRY_ID_FENCE_KEY  = -0x2000;
-    public static final long METAENTRY_ID_FORCE_LEDGER  = -0x4000;
-    static final long METAENTRY_ID_LEDGER_EXPLICITLAC  = -0x8000;
+    // TODO: Shouldn't this be async?
+    ByteBuf readEntry(long ledgerId, long entryId)
+            throws IOException, NoLedgerException;
+    long readLastAddConfirmed(long ledgerId) throws IOException;
+    PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException, NoLedgerException;
 
-    private final LedgerDirsManager ledgerDirsManager;
-    protected final Supplier<BookieServiceInfo> bookieServiceInfoProvider;
-    private final LedgerDirsManager indexDirsManager;
-    LedgerDirsMonitor dirsMonitor;
+    /**
+     * Fences a ledger. From this point on, clients will be unable to
+     * write to this ledger. Only recoveryAddEntry will be
+     * able to add entries to the ledger.
+     * This method is idempotent. Once a ledger is fenced, it can
+     * never be unfenced. Fencing a fenced ledger has no effect.
+     * @return
+     */
+    CompletableFuture<Boolean> fenceLedger(long ledgerId, byte[] masterKey)
+            throws IOException, BookieException;
 
-    // Registration Manager for managing registration
-    protected final MetadataBookieDriver metadataDriver;
+    // TODO: Replace Watcher with a completableFuture (cancellable)
+    boolean waitForLastAddConfirmedUpdate(long ledgerId,
+                                          long previousLAC,
+                                          Watcher<LastAddConfirmedUpdateNotification> watcher)
+            throws IOException;
+    void cancelWaitForLastAddConfirmedUpdate(long ledgerId,
+                                             Watcher<LastAddConfirmedUpdateNotification> watcher)
+            throws IOException;
 
-    private int exitCode = ExitCode.OK;
+    // TODO: StateManager should be passed as a parameter to Bookie
+    StateManager getStateManager();
 
-    private final ConcurrentLongHashMap<byte[]> masterKeyCache = new ConcurrentLongHashMap<>();
+    // TODO: Should be constructed and passed in as a parameter
+    LedgerStorage getLedgerStorage();
 
-    protected StateManager stateManager;
+    // TODO: LedgerManagerFactory, should be constructed elsewhere, passed in as parameter
+    LedgerManagerFactory getLedgerManagerFactory();
 
-    // Expose Stats
-    final StatsLogger statsLogger;
-    private final BookieStats bookieStats;
-
-    private final ByteBufAllocator allocator;
-
-    private final boolean writeDataToJournal;
-
-    @StatsDoc(
-            name = JOURNAL_MEMORY_MAX,
-            help = "The max amount of memory in bytes that can be used by the bookie journal"
-    )
-    private final Gauge<Long> journalMemoryMaxStats;
-
-    @StatsDoc(
-            name = JOURNAL_MEMORY_USED,
-            help = "The actual amount of memory in bytes currently used by the bookie journal"
-    )
-    private final Gauge<Long> journalMemoryUsedStats;
-
+    // TODO: Move this exceptions somewhere else
     /**
      * Exception is thrown when no such a ledger is found in this bookie.
      */
-    public static class NoLedgerException extends IOException {
+    class NoLedgerException extends IOException {
         private static final long serialVersionUID = 1L;
         private final long ledgerId;
         public NoLedgerException(long ledgerId) {
@@ -182,7 +109,7 @@
     /**
      * Exception is thrown when no such an entry is found in this bookie.
      */
-    public static class NoEntryException extends IOException {
+    class NoEntryException extends IOException {
         private static final long serialVersionUID = 1L;
         private final long ledgerId;
         private final long entryId;
@@ -204,1505 +131,4 @@
         }
     }
 
-    // Write Callback do nothing
-    static class NopWriteCallback implements WriteCallback {
-        @Override
-        public void writeComplete(int rc, long ledgerId, long entryId,
-                                  BookieId addr, Object ctx) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Finished writing entry {} @ ledger {} for {} : {}",
-                        entryId, ledgerId, addr, rc);
-            }
-        }
-    }
-
-    public static void checkDirectoryStructure(File dir) throws IOException {
-        if (!dir.exists()) {
-            File parent = dir.getParentFile();
-            File preV3versionFile = new File(dir.getParent(),
-                    BookKeeperConstants.VERSION_FILENAME);
-
-            final AtomicBoolean oldDataExists = new AtomicBoolean(false);
-            parent.list(new FilenameFilter() {
-                    @Override
-                    public boolean accept(File dir, String name) {
-                        if (name.endsWith(".txn") || name.endsWith(".idx") || name.endsWith(".log")) {
-                            oldDataExists.set(true);
-                        }
-                        return true;
-                    }
-                });
-            if (preV3versionFile.exists() || oldDataExists.get()) {
-                String err = "Directory layout version is less than 3, upgrade needed";
-                LOG.error(err);
-                throw new IOException(err);
-            }
-            if (!dir.mkdirs()) {
-                String err = "Unable to create directory " + dir;
-                LOG.error(err);
-                throw new IOException(err);
-            }
-        }
-    }
-
-    /**
-     * Check that the environment for the bookie is correct.
-     * 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)
-            throws BookieException, IOException {
-        List<File> allLedgerDirs = new ArrayList<File>(ledgerDirsManager.getAllLedgerDirs().size()
-                + indexDirsManager.getAllLedgerDirs().size());
-        allLedgerDirs.addAll(ledgerDirsManager.getAllLedgerDirs());
-        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;
-        }
-
-        checkEnvironmentWithStorageExpansion(conf, metadataDriver, journalDirectories, allLedgerDirs);
-
-        checkIfDirsOnSameDiskPartition(allLedgerDirs);
-        checkIfDirsOnSameDiskPartition(journalDirectories);
-    }
-
-    /**
-     * Checks if multiple directories are in same diskpartition/filesystem/device.
-     * If ALLOW_MULTIPLEDIRS_UNDER_SAME_DISKPARTITION config parameter is not enabled, and
-     * if it is found that there are multiple directories in the same DiskPartition then
-     * it will throw DiskPartitionDuplicationException.
-     *
-     * @param dirs dirs to validate
-     *
-     * @throws IOException
-     */
-    private void checkIfDirsOnSameDiskPartition(List<File> dirs) throws DiskPartitionDuplicationException {
-        boolean allowDiskPartitionDuplication = conf.isAllowMultipleDirsUnderSameDiskPartition();
-        final MutableBoolean isDuplicationFoundAndNotAllowed = new MutableBoolean(false);
-        Map<FileStore, List<File>> fileStoreDirsMap = new HashMap<FileStore, List<File>>();
-        for (File dir : dirs) {
-            FileStore fileStore;
-            try {
-                fileStore = Files.getFileStore(dir.toPath());
-            } catch (IOException e) {
-                LOG.error("Got IOException while trying to FileStore of {}", dir);
-                throw new BookieException.DiskPartitionDuplicationException(e);
-            }
-            if (fileStoreDirsMap.containsKey(fileStore)) {
-                fileStoreDirsMap.get(fileStore).add(dir);
-            } else {
-                List<File> dirsList = new ArrayList<File>();
-                dirsList.add(dir);
-                fileStoreDirsMap.put(fileStore, dirsList);
-            }
-        }
-
-        fileStoreDirsMap.forEach((fileStore, dirsList) -> {
-            if (dirsList.size() > 1) {
-                if (allowDiskPartitionDuplication) {
-                    LOG.warn("Dirs: {} are in same DiskPartition/FileSystem: {}", dirsList, fileStore);
-                } else {
-                    LOG.error("Dirs: {} are in same DiskPartition/FileSystem: {}", dirsList, fileStore);
-                    isDuplicationFoundAndNotAllowed.setValue(true);
-                }
-            }
-        });
-        if (isDuplicationFoundAndNotAllowed.getValue()) {
-            throw new BookieException.DiskPartitionDuplicationException();
-        }
-    }
-
-    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(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());
-            }
-            if (null != conf.getBookieId()) {
-                addresses.add(BookieId.parse(conf.getBookieId()));
-            }
-        } 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) {
-            return BookieId.parse(customBookieId);
-        }
-        return getBookieAddress(conf).toBookieId();
-    }
-
-    /**
-     * Return the configured address of the bookie.
-     */
-    public static BookieSocketAddress getBookieAddress(ServerConfiguration conf)
-            throws UnknownHostException {
-        // Advertised address takes precedence over the listening interface and the
-        // useHostNameAsBookieID settings
-        if (conf.getAdvertisedAddress() != null && conf.getAdvertisedAddress().trim().length() > 0) {
-            String hostAddress = conf.getAdvertisedAddress().trim();
-            return new BookieSocketAddress(hostAddress, conf.getBookiePort());
-        }
-
-        String iface = conf.getListeningInterface();
-        if (iface == null) {
-            iface = "default";
-        }
-
-        String hostName = DNS.getDefaultHost(iface);
-        InetSocketAddress inetAddr = new InetSocketAddress(hostName, conf.getBookiePort());
-        if (inetAddr.isUnresolved()) {
-            throw new UnknownHostException("Unable to resolve default hostname: "
-                    + hostName + " for interface: " + iface);
-        }
-        String hostAddress = null;
-        InetAddress iAddress = inetAddr.getAddress();
-        if (conf.getUseHostNameAsBookieID()) {
-            hostAddress = iAddress.getCanonicalHostName();
-            if (conf.getUseShortHostName()) {
-                /*
-                 * if short hostname is used, then FQDN is not used. Short
-                 * hostname is the hostname cut at the first dot.
-                 */
-                hostAddress = hostAddress.split("\\.", 2)[0];
-            }
-        } else {
-            hostAddress = iAddress.getHostAddress();
-        }
-
-        BookieSocketAddress addr =
-                new BookieSocketAddress(hostAddress, conf.getBookiePort());
-        if (addr.getSocketAddress().getAddress().isLoopbackAddress()
-            && !conf.getAllowLoopback()) {
-            throw new UnknownHostException("Trying to listen on loopback address, "
-                    + addr + " but this is forbidden by default "
-                    + "(see ServerConfiguration#getAllowLoopback()).\n"
-                    + "If this happen, you can consider specifying the network interface"
-                    + " to listen on (e.g. listeningInterface=eth0) or specifying the"
-                    + " advertised address (e.g. advertisedAddress=172.x.y.z)");
-        }
-        return addr;
-    }
-
-    public LedgerDirsManager getLedgerDirsManager() {
-        return ledgerDirsManager;
-    }
-
-    LedgerDirsManager getIndexDirsManager() {
-        return indexDirsManager;
-    }
-
-    public long getTotalDiskSpace() throws IOException {
-        return getLedgerDirsManager().getTotalDiskSpace(ledgerDirsManager.getAllLedgerDirs());
-    }
-
-    public long getTotalFreeSpace() throws IOException {
-        return getLedgerDirsManager().getTotalFreeSpace(ledgerDirsManager.getAllLedgerDirs());
-    }
-
-    public static File getCurrentDirectory(File dir) {
-        return new File(dir, BookKeeperConstants.CURRENT_DIR);
-    }
-
-    public static File[] getCurrentDirectories(File[] dirs) {
-        File[] currentDirs = new File[dirs.length];
-        for (int i = 0; i < dirs.length; i++) {
-            currentDirs[i] = getCurrentDirectory(dirs[i]);
-        }
-        return currentDirs;
-    }
-
-    public Bookie(ServerConfiguration conf)
-            throws IOException, InterruptedException, BookieException {
-        this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT, new SimpleBookieServiceInfoProvider(conf));
-    }
-
-    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.
-     *
-     * <p>The caller is responsible for disposing of the ledgerStorage object.
-     *
-     * @param conf Bookie config.
-     * @param ledgerStorage Instance to initialize.
-     * @return Passed ledgerStorage instance
-     * @throws IOException
-     */
-    public static LedgerStorage mountLedgerStorageOffline(ServerConfiguration conf, LedgerStorage ledgerStorage)
-            throws IOException {
-        StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
-
-        LedgerDirsManager ledgerDirsManager = createLedgerDirsManager(
-                conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE));
-        LedgerDirsManager indexDirsManager = 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(
-                conf,
-                null,
-                ledgerDirsManager,
-                indexDirsManager,
-                null,
-                checkpointSource,
-                checkpointer,
-                statsLogger,
-                UnpooledByteBufAllocator.DEFAULT);
-
-        return ledgerStorage;
-    }
-
-    public Bookie(ServerConfiguration conf, StatsLogger statsLogger,
-            ByteBufAllocator allocator, Supplier<BookieServiceInfo> bookieServiceInfoProvider)
-            throws IOException, InterruptedException, BookieException {
-        super("Bookie-" + conf.getBookiePort());
-        this.bookieServiceInfoProvider = bookieServiceInfoProvider;
-        this.statsLogger = statsLogger;
-        this.conf = conf;
-        this.journalDirectories = Lists.newArrayList();
-        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.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);
-        }
-        stateManager = initializeStateManager();
-        // register shutdown handler using trigger mode
-        stateManager.setShutdownHandler(exitCode -> triggerBookieShutdown(exitCode));
-        // Initialise dirsMonitor. This would look through all the
-        // configured directories. When disk errors or all the ledger
-        // directories are full, would throws exception and fail bookie startup.
-        List<LedgerDirsManager> dirsManagers = new ArrayList<>();
-        dirsManagers.add(ledgerDirsManager);
-        if (indexDirsManager != ledgerDirsManager) {
-            dirsManagers.add(indexDirsManager);
-        }
-        this.dirsMonitor = new LedgerDirsMonitor(conf, diskChecker, dirsManagers);
-        try {
-            this.dirsMonitor.init();
-        } catch (NoWritableLedgerDirException nle) {
-            // start in read-only mode if no writable dirs and read-only allowed
-            if (!conf.isReadOnlyModeEnabled()) {
-                throw nle;
-            } else {
-                this.stateManager.transitionToReadOnlyMode();
-            }
-        }
-
-        // instantiate the journals
-        journals = Lists.newArrayList();
-        for (int i = 0; i < journalDirectories.size(); i++) {
-            journals.add(new Journal(i, journalDirectories.get(i),
-                    conf, ledgerDirsManager, statsLogger.scope(JOURNAL_SCOPE), allocator));
-        }
-
-        this.entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled();
-        CheckpointSource checkpointSource = new CheckpointSourceList(journals);
-
-        ledgerStorage = buildLedgerStorage(conf);
-
-        boolean isDbLedgerStorage = ledgerStorage instanceof DbLedgerStorage;
-
-        /*
-         * with this change https://github.com/apache/bookkeeper/pull/677,
-         * LedgerStorage drives the checkpoint logic.
-         *
-         * <p>There are two exceptions:
-         *
-         * 1) with multiple entry logs, checkpoint logic based on a entry log is
-         *    not possible, hence it needs to be timebased recurring thing and
-         *    it is driven by SyncThread. SyncThread.start does that and it is
-         *    started in Bookie.start method.
-         *
-         * 2) DbLedgerStorage
-         */
-        if (entryLogPerLedgerEnabled || isDbLedgerStorage) {
-            syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource) {
-                @Override
-                public void startCheckpoint(Checkpoint checkpoint) {
-                    /*
-                     * in the case of entryLogPerLedgerEnabled, LedgerStorage
-                     * dont drive checkpoint logic, but instead it is done
-                     * periodically by SyncThread. So startCheckpoint which
-                     * will be called by LedgerStorage will be no-op.
-                     */
-                }
-
-                @Override
-                public void start() {
-                    executor.scheduleAtFixedRate(() -> {
-                        doCheckpoint(checkpointSource.newCheckpoint());
-                    }, conf.getFlushInterval(), conf.getFlushInterval(), TimeUnit.MILLISECONDS);
-                }
-            };
-        } else {
-            syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource);
-        }
-
-        ledgerStorage.initialize(
-            conf,
-            ledgerManager,
-            ledgerDirsManager,
-            indexDirsManager,
-            stateManager,
-            checkpointSource,
-            syncThread,
-            statsLogger,
-            allocator);
-
-
-        handles = new HandleFactoryImpl(ledgerStorage);
-
-        // Expose Stats
-        this.bookieStats = new BookieStats(statsLogger);
-        journalMemoryMaxStats = new Gauge<Long>() {
-            final long journalMaxMemory = conf.getJournalMaxMemorySizeMb() * 1024 * 1024;
-
-            @Override
-            public Long getDefaultValue() {
-                return journalMaxMemory;
-            }
-
-            @Override
-            public Long getSample() {
-                return journalMaxMemory;
-            }
-        };
-        statsLogger.scope(JOURNAL_SCOPE).registerGauge(JOURNAL_MEMORY_MAX, journalMemoryMaxStats);
-
-        journalMemoryUsedStats = new Gauge<Long>() {
-            @Override
-            public Long getDefaultValue() {
-                return -1L;
-            }
-
-            @Override
-            public Long getSample() {
-                long totalMemory = 0L;
-                for (int i = 0; i < journals.size(); i++) {
-                    totalMemory += journals.get(i).getMemoryUsage();
-                }
-                return totalMemory;
-            }
-        };
-        statsLogger.scope(JOURNAL_SCOPE).registerGauge(JOURNAL_MEMORY_USED, journalMemoryUsedStats);
-    }
-
-    StateManager initializeStateManager() throws IOException {
-        return new BookieStateManager(conf, statsLogger, metadataDriver,
-                ledgerDirsManager, bookieServiceInfoProvider);
-    }
-
-    void readJournal() throws IOException, BookieException {
-        long startTs = System.currentTimeMillis();
-        JournalScanner scanner = new JournalScanner() {
-            @Override
-            public void process(int journalVersion, long offset, ByteBuffer recBuff) throws IOException {
-                long ledgerId = recBuff.getLong();
-                long entryId = recBuff.getLong();
-                try {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("Replay journal - ledger id : {}, entry id : {}.", ledgerId, entryId);
-                    }
-                    if (entryId == METAENTRY_ID_LEDGER_KEY) {
-                        if (journalVersion >= JournalChannel.V3) {
-                            int masterKeyLen = recBuff.getInt();
-                            byte[] masterKey = new byte[masterKeyLen];
-
-                            recBuff.get(masterKey);
-                            masterKeyCache.put(ledgerId, masterKey);
-
-                            // Force to re-insert the master key in ledger storage
-                            handles.getHandle(ledgerId, masterKey);
-                        } else {
-                            throw new IOException("Invalid journal. Contains journalKey "
-                                    + " but layout version (" + journalVersion
-                                    + ") is too old to hold this");
-                        }
-                    } else if (entryId == METAENTRY_ID_FENCE_KEY) {
-                        if (journalVersion >= JournalChannel.V4) {
-                            byte[] key = masterKeyCache.get(ledgerId);
-                            if (key == null) {
-                                key = ledgerStorage.readMasterKey(ledgerId);
-                            }
-                            LedgerDescriptor handle = handles.getHandle(ledgerId, key);
-                            handle.setFenced();
-                        } else {
-                            throw new IOException("Invalid journal. Contains fenceKey "
-                                    + " but layout version (" + journalVersion
-                                    + ") is too old to hold this");
-                        }
-                    } else if (entryId == METAENTRY_ID_LEDGER_EXPLICITLAC) {
-                        if (journalVersion >= JournalChannel.V6) {
-                            int explicitLacBufLength = recBuff.getInt();
-                            ByteBuf explicitLacBuf = Unpooled.buffer(explicitLacBufLength);
-                            byte[] explicitLacBufArray = new byte[explicitLacBufLength];
-                            recBuff.get(explicitLacBufArray);
-                            explicitLacBuf.writeBytes(explicitLacBufArray);
-                            byte[] key = masterKeyCache.get(ledgerId);
-                            if (key == null) {
-                                key = ledgerStorage.readMasterKey(ledgerId);
-                            }
-                            LedgerDescriptor handle = handles.getHandle(ledgerId, key);
-                            handle.setExplicitLac(explicitLacBuf);
-                        } else {
-                            throw new IOException("Invalid journal. Contains explicitLAC " + " but layout version ("
-                                    + journalVersion + ") is too old to hold this");
-                        }
-                    } else if (entryId < 0) {
-                        /*
-                         * this is possible if bookie code binary is rolledback
-                         * to older version but when it is trying to read
-                         * Journal which was created previously using newer
-                         * code/journalversion, which introduced new special
-                         * entry. So in anycase, if we see unrecognizable
-                         * special entry while replaying journal we should skip
-                         * (ignore) it.
-                         */
-                        LOG.warn("Read unrecognizable entryId: {} for ledger: {} while replaying Journal. Skipping it",
-                                entryId, ledgerId);
-                    } else {
-                        byte[] key = masterKeyCache.get(ledgerId);
-                        if (key == null) {
-                            key = ledgerStorage.readMasterKey(ledgerId);
-                        }
-                        LedgerDescriptor handle = handles.getHandle(ledgerId, key);
-
-                        recBuff.rewind();
-                        handle.addEntry(Unpooled.wrappedBuffer(recBuff));
-                    }
-                } catch (NoLedgerException nsle) {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("Skip replaying entries of ledger {} since it was deleted.", ledgerId);
-                    }
-                } catch (BookieException be) {
-                    throw new IOException(be);
-                }
-            }
-        };
-
-        for (Journal journal : journals) {
-            replay(journal, scanner);
-        }
-        long elapsedTs = System.currentTimeMillis() - startTs;
-        LOG.info("Finished replaying journal in {} ms.", elapsedTs);
-    }
-
-    /**
-     * Replay journal files and updates journal's in-memory lastLogMark object.
-     *
-     * @param journal Journal object corresponding to a journalDir
-     * @param scanner Scanner to process replayed entries.
-     * @throws IOException
-     */
-    private void replay(Journal journal, JournalScanner scanner) throws IOException {
-        final LogMark markedLog = journal.getLastLogMark().getCurMark();
-        List<Long> logs = Journal.listJournalIds(journal.getJournalDirectory(), journalId ->
-            journalId >= markedLog.getLogFileId());
-        // last log mark may be missed due to no sync up before
-        // validate filtered log ids only when we have markedLogId
-        if (markedLog.getLogFileId() > 0) {
-            if (logs.size() == 0 || logs.get(0) != markedLog.getLogFileId()) {
-                throw new IOException("Recovery log " + markedLog.getLogFileId() + " is missing");
-            }
-        }
-
-        // TODO: When reading in the journal logs that need to be synced, we
-        // should use BufferedChannels instead to minimize the amount of
-        // system calls done.
-        for (Long id : logs) {
-            long logPosition = 0L;
-            if (id == markedLog.getLogFileId()) {
-                logPosition = markedLog.getLogFileOffset();
-            }
-            LOG.info("Replaying journal {} from position {}", id, logPosition);
-            long scanOffset = journal.scanJournal(id, logPosition, scanner);
-            // Update LastLogMark after completely replaying journal
-            // scanOffset will point to EOF position
-            // After LedgerStorage flush, SyncThread should persist this to disk
-            journal.setLastLogMark(id, scanOffset);
-        }
-    }
-
-    @Override
-    public synchronized void start() {
-        setDaemon(true);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("I'm starting a bookie with journal directories {}",
-                    journalDirectories.stream().map(File::getName).collect(Collectors.joining(", ")));
-        }
-        //Start DiskChecker thread
-        dirsMonitor.start();
-
-        // replay journals
-        try {
-            readJournal();
-        } catch (IOException | BookieException ioe) {
-            LOG.error("Exception while replaying journals, shutting down", ioe);
-            shutdown(ExitCode.BOOKIE_EXCEPTION);
-            return;
-        }
-
-        // Do a fully flush after journal replay
-        try {
-            syncThread.requestFlush().get();
-        } catch (InterruptedException e) {
-            LOG.warn("Interrupting the fully flush after replaying journals : ", e);
-            Thread.currentThread().interrupt();
-        } catch (ExecutionException e) {
-            LOG.error("Error on executing a fully flush after replaying journals.");
-            shutdown(ExitCode.BOOKIE_EXCEPTION);
-            return;
-        }
-
-        if (conf.isLocalConsistencyCheckOnStartup()) {
-            LOG.info("Running local consistency check on startup prior to accepting IO.");
-            List<LedgerStorage.DetectedInconsistency> errors = null;
-            try {
-                errors = ledgerStorage.localConsistencyCheck(Optional.empty());
-            } catch (IOException e) {
-                LOG.error("Got a fatal exception while checking store", e);
-                shutdown(ExitCode.BOOKIE_EXCEPTION);
-                return;
-            }
-            if (errors != null && errors.size() > 0) {
-                LOG.error("Bookie failed local consistency check:");
-                for (LedgerStorage.DetectedInconsistency error : errors) {
-                    LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException());
-                }
-                shutdown(ExitCode.BOOKIE_EXCEPTION);
-                return;
-            }
-        }
-
-        LOG.info("Finished reading journal, starting bookie");
-
-
-        /*
-         * start sync thread first, so during replaying journals, we could do
-         * checkpoint which reduce the chance that we need to replay journals
-         * again if bookie restarted again before finished journal replays.
-         */
-        syncThread.start();
-
-        // start bookie thread
-        super.start();
-
-        // After successful bookie startup, register listener for disk
-        // error/full notifications.
-        ledgerDirsManager.addLedgerDirsListener(getLedgerDirsListener());
-        if (indexDirsManager != ledgerDirsManager) {
-            indexDirsManager.addLedgerDirsListener(getLedgerDirsListener());
-        }
-
-        ledgerStorage.start();
-
-        // check the bookie status to start with, and set running.
-        // since bookie server use running as a flag to tell bookie server whether it is alive
-        // if setting it in bookie thread, the watcher might run before bookie thread.
-        stateManager.initState();
-
-        try {
-            stateManager.registerBookie(true).get();
-        } catch (Exception e) {
-            LOG.error("Couldn't register bookie with zookeeper, shutting down : ", e);
-            shutdown(ExitCode.ZK_REG_FAIL);
-        }
-    }
-
-    /*
-     * Get the DiskFailure listener for the bookie
-     */
-    private LedgerDirsListener getLedgerDirsListener() {
-
-        return new LedgerDirsListener() {
-
-            @Override
-            public void diskFailed(File disk) {
-                // Shutdown the bookie on disk failure.
-                triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
-            }
-
-            @Override
-            public void allDisksFull(boolean highPriorityWritesAllowed) {
-                // Transition to readOnly mode on all disks full
-                stateManager.setHighPriorityWritesAvailability(highPriorityWritesAllowed);
-                stateManager.transitionToReadOnlyMode();
-            }
-
-            @Override
-            public void fatalError() {
-                LOG.error("Fatal error reported by ledgerDirsManager");
-                triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
-            }
-
-            @Override
-            public void diskWritable(File disk) {
-                // Transition to writable mode when a disk becomes writable again.
-                stateManager.setHighPriorityWritesAvailability(true);
-                stateManager.transitionToWritableMode();
-            }
-
-            @Override
-            public void diskJustWritable(File disk) {
-                // Transition to writable mode when a disk becomes writable again.
-                stateManager.setHighPriorityWritesAvailability(true);
-                stateManager.transitionToWritableMode();
-            }
-        };
-    }
-
-    /**
-     * 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.
-     */
-    public boolean isReadOnly() {
-        return stateManager.isReadOnly();
-    }
-
-    /**
-     * Check whether Bookie is available for high priority writes.
-     *
-     * @return true if the bookie is able to take high priority writes.
-     */
-    public boolean isAvailableForHighPriorityWrites() {
-        return stateManager.isAvailableForHighPriorityWrites();
-    }
-
-    public boolean isRunning() {
-        return stateManager.isRunning();
-    }
-
-    @Override
-    public void run() {
-        // bookie thread wait for journal thread
-        try {
-            // start journals
-            for (Journal journal: journals) {
-                journal.start();
-            }
-
-            // wait until journal quits
-            for (Journal journal: journals) {
-
-                journal.joinThread();
-            }
-            LOG.info("Journal thread(s) quit.");
-        } catch (InterruptedException ie) {
-            Thread.currentThread().interrupt();
-            LOG.warn("Interrupted on running journal thread : ", ie);
-        }
-        // if the journal thread quits due to shutting down, it is ok
-        if (!stateManager.isShuttingDown()) {
-            // some error found in journal thread and it quits
-            // following add operations to it would hang unit client timeout
-            // so we should let bookie server exists
-            LOG.error("Journal manager quits unexpectedly.");
-            triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
-        }
-    }
-
-    // Triggering the Bookie shutdown in its own thread,
-    // because shutdown can be called from sync thread which would be
-    // interrupted by shutdown call.
-    AtomicBoolean shutdownTriggered = new AtomicBoolean(false);
-    void triggerBookieShutdown(final int exitCode) {
-        if (!shutdownTriggered.compareAndSet(false, true)) {
-            return;
-        }
-        LOG.info("Triggering shutdown of Bookie-{} with exitCode {}",
-                 conf.getBookiePort(), exitCode);
-        BookieThread th = new BookieThread("BookieShutdownTrigger") {
-            @Override
-            public void run() {
-                Bookie.this.shutdown(exitCode);
-            }
-        };
-        th.start();
-    }
-
-    // provided a public shutdown method for other caller
-    // to shut down bookie gracefully
-    public int shutdown() {
-        return shutdown(ExitCode.OK);
-    }
-
-    // internal shutdown method to let shutdown bookie gracefully
-    // when encountering exception
-    synchronized int shutdown(int exitCode) {
-        try {
-            if (isRunning()) { // avoid shutdown twice
-                // the exitCode only set when first shutdown usually due to exception found
-                LOG.info("Shutting down Bookie-{} with exitCode {}",
-                         conf.getBookiePort(), exitCode);
-                if (this.exitCode == ExitCode.OK) {
-                    this.exitCode = exitCode;
-                }
-
-                stateManager.forceToShuttingDown();
-
-                // turn bookie to read only during shutting down process
-                LOG.info("Turning bookie to read only during shut down");
-                stateManager.forceToReadOnly();
-
-                // Shutdown Sync thread
-                syncThread.shutdown();
-
-                // Shutdown journals
-                for (Journal journal : journals) {
-                    journal.shutdown();
-                }
-                this.join();
-
-                // Shutdown the EntryLogger which has the GarbageCollector Thread running
-                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);
-        } catch (Exception e) {
-            LOG.error("Got Exception while trying to shutdown Bookie", e);
-            throw e;
-        } finally {
-            // setting running to false here, so watch thread
-            // in bookie server know it only after bookie shut down
-            stateManager.close();
-        }
-        return this.exitCode;
-    }
-
-    /**
-     * Retrieve the ledger descriptor for the ledger which entry should be added to.
-     * The LedgerDescriptor returned from this method should be eventually freed with
-     * #putHandle().
-     *
-     * @throws BookieException if masterKey does not match the master key of the ledger
-     */
-    @VisibleForTesting
-    LedgerDescriptor getLedgerForEntry(ByteBuf entry, final byte[] masterKey)
-            throws IOException, BookieException {
-        final long ledgerId = entry.getLong(entry.readerIndex());
-
-        return handles.getHandle(ledgerId, masterKey);
-    }
-
-    private Journal getJournal(long ledgerId) {
-        return journals.get(MathUtils.signSafeMod(ledgerId, journals.size()));
-    }
-
-    /**
-     * Add an entry to a ledger as specified by handle.
-     */
-    private void addEntryInternal(LedgerDescriptor handle, ByteBuf entry,
-                                  boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
-            throws IOException, BookieException, InterruptedException {
-        long ledgerId = handle.getLedgerId();
-        long entryId = handle.addEntry(entry);
-
-        bookieStats.getWriteBytes().add(entry.readableBytes());
-
-        // journal `addEntry` should happen after the entry is added to ledger storage.
-        // otherwise the journal entry can potentially be rolled before the ledger is created in ledger storage.
-        if (masterKeyCache.get(ledgerId) == null) {
-            // Force the load into masterKey cache
-            byte[] oldValue = masterKeyCache.putIfAbsent(ledgerId, masterKey);
-            if (oldValue == null) {
-                // new handle, we should add the key to journal ensure we can rebuild
-                ByteBuffer bb = ByteBuffer.allocate(8 + 8 + 4 + masterKey.length);
-                bb.putLong(ledgerId);
-                bb.putLong(METAENTRY_ID_LEDGER_KEY);
-                bb.putInt(masterKey.length);
-                bb.put(masterKey);
-                bb.flip();
-
-                getJournal(ledgerId).logAddEntry(bb, false /* ackBeforeSync */, new NopWriteCallback(), null);
-            }
-        }
-
-        if (!writeDataToJournal) {
-            cb.writeComplete(0, ledgerId, entryId, null, ctx);
-            return;
-        }
-
-        if (LOG.isTraceEnabled()) {
-            LOG.trace("Adding {}@{}", entryId, ledgerId);
-        }
-        getJournal(ledgerId).logAddEntry(entry, ackBeforeSync, cb, ctx);
-    }
-
-    /**
-     * Add entry to a ledger, even if the ledger has previous been fenced. This should only
-     * happen in bookie recovery or ledger recovery cases, where entries are being replicates
-     * so that they exist on a quorum of bookies. The corresponding client side call for this
-     * is not exposed to users.
-     */
-    public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
-            throws IOException, BookieException, InterruptedException {
-        long requestNanos = MathUtils.nowInNano();
-        boolean success = false;
-        int entrySize = 0;
-        try {
-            LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
-            synchronized (handle) {
-                entrySize = entry.readableBytes();
-                addEntryInternal(handle, entry, false /* ackBeforeSync */, cb, ctx, masterKey);
-            }
-            success = true;
-        } catch (NoWritableLedgerDirException e) {
-            stateManager.transitionToReadOnlyMode();
-            throw new IOException(e);
-        } finally {
-            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
-            if (success) {
-                bookieStats.getRecoveryAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize);
-            } else {
-                bookieStats.getRecoveryAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getAddBytesStats().registerFailedValue(entrySize);
-            }
-
-            entry.release();
-        }
-    }
-
-    private ByteBuf createExplicitLACEntry(long ledgerId, ByteBuf explicitLac) {
-        ByteBuf bb = allocator.directBuffer(8 + 8 + 4 + explicitLac.capacity());
-        bb.writeLong(ledgerId);
-        bb.writeLong(METAENTRY_ID_LEDGER_EXPLICITLAC);
-        bb.writeInt(explicitLac.capacity());
-        bb.writeBytes(explicitLac);
-        return bb;
-    }
-
-    public void setExplicitLac(ByteBuf entry, WriteCallback writeCallback, Object ctx, byte[] masterKey)
-            throws IOException, InterruptedException, BookieException {
-        try {
-            long ledgerId = entry.getLong(entry.readerIndex());
-            LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey);
-            synchronized (handle) {
-                entry.markReaderIndex();
-                handle.setExplicitLac(entry);
-                entry.resetReaderIndex();
-                ByteBuf explicitLACEntry = createExplicitLACEntry(ledgerId, entry);
-                getJournal(ledgerId).logAddEntry(explicitLACEntry, false /* ackBeforeSync */, writeCallback, ctx);
-            }
-        } catch (NoWritableLedgerDirException e) {
-            stateManager.transitionToReadOnlyMode();
-            throw new IOException(e);
-        }
-    }
-
-    public ByteBuf getExplicitLac(long ledgerId) throws IOException, Bookie.NoLedgerException {
-        ByteBuf lac;
-        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-        synchronized (handle) {
-            lac = handle.getExplicitLac();
-        }
-        return lac;
-    }
-
-    /**
-     * Force sync given 'ledgerId' entries on the journal to the disk.
-     * It works like a regular addEntry with ackBeforeSync=false.
-     * This is useful for ledgers with DEFERRED_SYNC write flag.
-     */
-    public void forceLedger(long ledgerId, WriteCallback cb,
-                            Object ctx) {
-        if (LOG.isTraceEnabled()) {
-            LOG.trace("Forcing ledger {}", ledgerId);
-        }
-        Journal journal = getJournal(ledgerId);
-        journal.forceLedger(ledgerId, cb, ctx);
-        bookieStats.getForceLedgerOps().inc();
-    }
-
-    /**
-     * Add entry to a ledger.
-     */
-    public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
-            throws IOException, BookieException, InterruptedException {
-        long requestNanos = MathUtils.nowInNano();
-        boolean success = false;
-        int entrySize = 0;
-        try {
-            LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
-            synchronized (handle) {
-                if (handle.isFenced()) {
-                    throw BookieException
-                            .create(BookieException.Code.LedgerFencedException);
-                }
-                entrySize = entry.readableBytes();
-                addEntryInternal(handle, entry, ackBeforeSync, cb, ctx, masterKey);
-            }
-            success = true;
-        } catch (NoWritableLedgerDirException e) {
-            stateManager.transitionToReadOnlyMode();
-            throw new IOException(e);
-        } finally {
-            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
-            if (success) {
-                bookieStats.getAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize);
-            } else {
-                bookieStats.getAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getAddBytesStats().registerFailedValue(entrySize);
-            }
-
-            entry.release();
-        }
-    }
-
-    /**
-     * Fences a ledger. From this point on, clients will be unable to
-     * write to this ledger. Only recoveryAddEntry will be
-     * able to add entries to the ledger.
-     * This method is idempotent. Once a ledger is fenced, it can
-     * never be unfenced. Fencing a fenced ledger has no effect.
-     */
-    public SettableFuture<Boolean> fenceLedger(long ledgerId, byte[] masterKey)
-            throws IOException, BookieException {
-        LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey);
-        return handle.fenceAndLogInJournal(getJournal(ledgerId));
-    }
-
-    public ByteBuf readEntry(long ledgerId, long entryId)
-            throws IOException, NoLedgerException {
-        long requestNanos = MathUtils.nowInNano();
-        boolean success = false;
-        int entrySize = 0;
-        try {
-            LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("Reading {}@{}", entryId, ledgerId);
-            }
-            ByteBuf entry = handle.readEntry(entryId);
-            bookieStats.getReadBytes().add(entry.readableBytes());
-            success = true;
-            return entry;
-        } finally {
-            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
-            if (success) {
-                bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getReadBytesStats().registerSuccessfulValue(entrySize);
-            } else {
-                bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-                bookieStats.getReadEntryStats().registerFailedValue(entrySize);
-            }
-        }
-    }
-
-    public long readLastAddConfirmed(long ledgerId) throws IOException {
-        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-        return handle.getLastAddConfirmed();
-    }
-
-    public boolean waitForLastAddConfirmedUpdate(long ledgerId,
-                                                 long previousLAC,
-                                                 Watcher<LastAddConfirmedUpdateNotification> watcher)
-            throws IOException {
-        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-        return handle.waitForLastAddConfirmedUpdate(previousLAC, watcher);
-    }
-
-    public void cancelWaitForLastAddConfirmedUpdate(long ledgerId,
-                                                    Watcher<LastAddConfirmedUpdateNotification> watcher)
-            throws IOException {
-        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-        handle.cancelWaitForLastAddConfirmedUpdate(watcher);
-    }
-
-    @VisibleForTesting
-    public LedgerStorage getLedgerStorage() {
-        return ledgerStorage;
-    }
-
-    @VisibleForTesting
-    public BookieStateManager getStateManager() {
-        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;
-    }
-
-    /**
-     * Format the bookie server data.
-     *
-     * @param conf ServerConfiguration
-     * @param isInteractive Whether format should ask prompt for confirmation if old data exists or not.
-     * @param force If non interactive and force is true, then old data will be removed without confirm prompt.
-     * @return Returns true if the format is success else returns false
-     */
-    public static boolean format(ServerConfiguration conf,
-            boolean isInteractive, boolean force) {
-        for (File journalDir : conf.getJournalDirs()) {
-            String[] journalDirFiles =
-                    journalDir.exists() && journalDir.isDirectory() ? journalDir.list() : null;
-            if (journalDirFiles != null && journalDirFiles.length != 0) {
-                try {
-                    boolean confirm = false;
-                    if (!isInteractive) {
-                        // If non interactive and force is set, then delete old
-                        // data.
-                        confirm = force;
-                    } else {
-                        confirm = IOUtils
-                                .confirmPrompt("Are you sure to format Bookie data..?");
-                    }
-
-                    if (!confirm) {
-                        LOG.error("Bookie format aborted!!");
-                        return false;
-                    }
-                } catch (IOException e) {
-                    LOG.error("Error during bookie format", e);
-                    return false;
-                }
-            }
-            if (!cleanDir(journalDir)) {
-                LOG.error("Formatting journal directory failed");
-                return false;
-            }
-        }
-
-        File[] ledgerDirs = conf.getLedgerDirs();
-        for (File dir : ledgerDirs) {
-            if (!cleanDir(dir)) {
-                LOG.error("Formatting ledger directory " + dir + " failed");
-                return false;
-            }
-        }
-
-        // Clean up index directories if they are separate from the ledger dirs
-        File[] indexDirs = conf.getIndexDirs();
-        if (null != indexDirs) {
-            for (File dir : indexDirs) {
-                if (!cleanDir(dir)) {
-                    LOG.error("Formatting index directory " + dir + " failed");
-                    return false;
-                }
-            }
-        }
-
-        LOG.info("Bookie format completed successfully");
-        return true;
-    }
-
-    private static boolean cleanDir(File dir) {
-        if (dir.exists()) {
-            File[] files = dir.listFiles();
-            if (files != null) {
-                for (File child : files) {
-                    boolean delete = FileUtils.deleteQuietly(child);
-                    if (!delete) {
-                        LOG.error("Not able to delete " + child);
-                        return false;
-                    }
-                }
-            }
-        } else if (!dir.mkdirs()) {
-            LOG.error("Not able to create the directory " + dir);
-            return false;
-        }
-        return true;
-    }
-
-    /**
-     * Returns exit code - cause of failure.
-     *
-     * @return {@link ExitCode}
-     */
-    public int getExitCode() {
-        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;
-        try {
-            LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("GetEntriesOfLedger {}", ledgerId);
-            }
-            OfLong entriesOfLedger = handle.getListOfEntriesOfLedger(ledgerId);
-            success = true;
-            return entriesOfLedger;
-        } finally {
-            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
-            if (success) {
-                bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-            } else {
-                bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
-            }
-        }
-    }
 }
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
new file mode 100644
index 0000000..2dfa1a1
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
@@ -0,0 +1,1665 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.JOURNAL_MEMORY_MAX;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_MEMORY_USED;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+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;
+import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
+import org.apache.bookkeeper.bookie.stats.BookieStats;
+import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
+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;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.proto.SimpleBookieServiceInfoProvider;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+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;
+
+/**
+ * Implements a bookie.
+ */
+public class BookieImpl extends BookieCriticalThread implements Bookie {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Bookie.class);
+
+    final List<File> journalDirectories;
+    final ServerConfiguration conf;
+
+    final SyncThread syncThread;
+    final LedgerManagerFactory ledgerManagerFactory;
+    final LedgerManager ledgerManager;
+    final LedgerStorage ledgerStorage;
+    final List<Journal> journals;
+
+    final HandleFactory handles;
+    final boolean entryLogPerLedgerEnabled;
+
+    public static final long METAENTRY_ID_LEDGER_KEY = -0x1000;
+    public static final long METAENTRY_ID_FENCE_KEY  = -0x2000;
+    public static final long METAENTRY_ID_FORCE_LEDGER  = -0x4000;
+    static final long METAENTRY_ID_LEDGER_EXPLICITLAC  = -0x8000;
+
+    private final LedgerDirsManager ledgerDirsManager;
+    protected final Supplier<BookieServiceInfo> bookieServiceInfoProvider;
+    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<>();
+
+    protected StateManager stateManager;
+
+    // Expose Stats
+    final StatsLogger statsLogger;
+    private final BookieStats bookieStats;
+
+    private final ByteBufAllocator allocator;
+
+    private final boolean writeDataToJournal;
+
+    @StatsDoc(
+            name = JOURNAL_MEMORY_MAX,
+            help = "The max amount of memory in bytes that can be used by the bookie journal"
+    )
+    private final Gauge<Long> journalMemoryMaxStats;
+
+    @StatsDoc(
+            name = JOURNAL_MEMORY_USED,
+            help = "The actual amount of memory in bytes currently used by the bookie journal"
+    )
+    private final Gauge<Long> journalMemoryUsedStats;
+
+    // Write Callback do nothing
+    static class NopWriteCallback implements WriteCallback {
+        @Override
+        public void writeComplete(int rc, long ledgerId, long entryId,
+                                  BookieId addr, Object ctx) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Finished writing entry {} @ ledger {} for {} : {}",
+                        entryId, ledgerId, addr, rc);
+            }
+        }
+    }
+
+    public static void checkDirectoryStructure(File dir) throws IOException {
+        if (!dir.exists()) {
+            File parent = dir.getParentFile();
+            File preV3versionFile = new File(dir.getParent(),
+                    BookKeeperConstants.VERSION_FILENAME);
+
+            final AtomicBoolean oldDataExists = new AtomicBoolean(false);
+            parent.list(new FilenameFilter() {
+                    @Override
+                    public boolean accept(File dir, String name) {
+                        if (name.endsWith(".txn") || name.endsWith(".idx") || name.endsWith(".log")) {
+                            oldDataExists.set(true);
+                        }
+                        return true;
+                    }
+                });
+            if (preV3versionFile.exists() || oldDataExists.get()) {
+                String err = "Directory layout version is less than 3, upgrade needed";
+                LOG.error(err);
+                throw new IOException(err);
+            }
+            if (!dir.mkdirs()) {
+                String err = "Unable to create directory " + dir;
+                LOG.error(err);
+                throw new IOException(err);
+            }
+        }
+    }
+
+    /**
+     * Check that the environment for the bookie is correct.
+     * 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)
+            throws BookieException, IOException {
+        List<File> allLedgerDirs = new ArrayList<File>(ledgerDirsManager.getAllLedgerDirs().size()
+                + indexDirsManager.getAllLedgerDirs().size());
+        allLedgerDirs.addAll(ledgerDirsManager.getAllLedgerDirs());
+        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;
+        }
+
+        checkEnvironmentWithStorageExpansion(conf, metadataDriver, journalDirectories, allLedgerDirs);
+
+        checkIfDirsOnSameDiskPartition(allLedgerDirs);
+        checkIfDirsOnSameDiskPartition(journalDirectories);
+    }
+
+    /**
+     * Checks if multiple directories are in same diskpartition/filesystem/device.
+     * If ALLOW_MULTIPLEDIRS_UNDER_SAME_DISKPARTITION config parameter is not enabled, and
+     * if it is found that there are multiple directories in the same DiskPartition then
+     * it will throw DiskPartitionDuplicationException.
+     *
+     * @param dirs dirs to validate
+     *
+     * @throws IOException
+     */
+    private void checkIfDirsOnSameDiskPartition(List<File> dirs) throws DiskPartitionDuplicationException {
+        boolean allowDiskPartitionDuplication = conf.isAllowMultipleDirsUnderSameDiskPartition();
+        final MutableBoolean isDuplicationFoundAndNotAllowed = new MutableBoolean(false);
+        Map<FileStore, List<File>> fileStoreDirsMap = new HashMap<FileStore, List<File>>();
+        for (File dir : dirs) {
+            FileStore fileStore;
+            try {
+                fileStore = Files.getFileStore(dir.toPath());
+            } catch (IOException e) {
+                LOG.error("Got IOException while trying to FileStore of {}", dir);
+                throw new BookieException.DiskPartitionDuplicationException(e);
+            }
+            if (fileStoreDirsMap.containsKey(fileStore)) {
+                fileStoreDirsMap.get(fileStore).add(dir);
+            } else {
+                List<File> dirsList = new ArrayList<File>();
+                dirsList.add(dir);
+                fileStoreDirsMap.put(fileStore, dirsList);
+            }
+        }
+
+        fileStoreDirsMap.forEach((fileStore, dirsList) -> {
+            if (dirsList.size() > 1) {
+                if (allowDiskPartitionDuplication) {
+                    LOG.warn("Dirs: {} are in same DiskPartition/FileSystem: {}", dirsList, fileStore);
+                } else {
+                    LOG.error("Dirs: {} are in same DiskPartition/FileSystem: {}", dirsList, fileStore);
+                    isDuplicationFoundAndNotAllowed.setValue(true);
+                }
+            }
+        });
+        if (isDuplicationFoundAndNotAllowed.getValue()) {
+            throw new BookieException.DiskPartitionDuplicationException();
+        }
+    }
+
+    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(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());
+            }
+            if (null != conf.getBookieId()) {
+                addresses.add(BookieId.parse(conf.getBookieId()));
+            }
+        } 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) {
+            return BookieId.parse(customBookieId);
+        }
+        return getBookieAddress(conf).toBookieId();
+    }
+
+    /**
+     * Return the configured address of the bookie.
+     */
+    public static BookieSocketAddress getBookieAddress(ServerConfiguration conf)
+            throws UnknownHostException {
+        // Advertised address takes precedence over the listening interface and the
+        // useHostNameAsBookieID settings
+        if (conf.getAdvertisedAddress() != null && conf.getAdvertisedAddress().trim().length() > 0) {
+            String hostAddress = conf.getAdvertisedAddress().trim();
+            return new BookieSocketAddress(hostAddress, conf.getBookiePort());
+        }
+
+        String iface = conf.getListeningInterface();
+        if (iface == null) {
+            iface = "default";
+        }
+
+        String hostName = DNS.getDefaultHost(iface);
+        InetSocketAddress inetAddr = new InetSocketAddress(hostName, conf.getBookiePort());
+        if (inetAddr.isUnresolved()) {
+            throw new UnknownHostException("Unable to resolve default hostname: "
+                    + hostName + " for interface: " + iface);
+        }
+        String hostAddress = null;
+        InetAddress iAddress = inetAddr.getAddress();
+        if (conf.getUseHostNameAsBookieID()) {
+            hostAddress = iAddress.getCanonicalHostName();
+            if (conf.getUseShortHostName()) {
+                /*
+                 * if short hostname is used, then FQDN is not used. Short
+                 * hostname is the hostname cut at the first dot.
+                 */
+                hostAddress = hostAddress.split("\\.", 2)[0];
+            }
+        } else {
+            hostAddress = iAddress.getHostAddress();
+        }
+
+        BookieSocketAddress addr =
+                new BookieSocketAddress(hostAddress, conf.getBookiePort());
+        if (addr.getSocketAddress().getAddress().isLoopbackAddress()
+            && !conf.getAllowLoopback()) {
+            throw new UnknownHostException("Trying to listen on loopback address, "
+                    + addr + " but this is forbidden by default "
+                    + "(see ServerConfiguration#getAllowLoopback()).\n"
+                    + "If this happen, you can consider specifying the network interface"
+                    + " to listen on (e.g. listeningInterface=eth0) or specifying the"
+                    + " advertised address (e.g. advertisedAddress=172.x.y.z)");
+        }
+        return addr;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public long getTotalDiskSpace() throws IOException {
+        return getLedgerDirsManager().getTotalDiskSpace(ledgerDirsManager.getAllLedgerDirs());
+    }
+
+    public long getTotalFreeSpace() throws IOException {
+        return getLedgerDirsManager().getTotalFreeSpace(ledgerDirsManager.getAllLedgerDirs());
+    }
+
+    public static File getCurrentDirectory(File dir) {
+        return new File(dir, BookKeeperConstants.CURRENT_DIR);
+    }
+
+    public static File[] getCurrentDirectories(File[] dirs) {
+        File[] currentDirs = new File[dirs.length];
+        for (int i = 0; i < dirs.length; i++) {
+            currentDirs[i] = getCurrentDirectory(dirs[i]);
+        }
+        return currentDirs;
+    }
+
+    public BookieImpl(ServerConfiguration conf)
+            throws IOException, InterruptedException, BookieException {
+        this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT, new SimpleBookieServiceInfoProvider(conf));
+    }
+
+    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.
+     *
+     * <p>The caller is responsible for disposing of the ledgerStorage object.
+     *
+     * @param conf Bookie config.
+     * @param ledgerStorage Instance to initialize.
+     * @return Passed ledgerStorage instance
+     * @throws IOException
+     */
+    public static LedgerStorage mountLedgerStorageOffline(ServerConfiguration conf, LedgerStorage ledgerStorage)
+            throws IOException {
+        StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+
+        LedgerDirsManager ledgerDirsManager = createLedgerDirsManager(
+                conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE));
+        LedgerDirsManager indexDirsManager = 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(
+                conf,
+                null,
+                ledgerDirsManager,
+                indexDirsManager,
+                null,
+                checkpointSource,
+                checkpointer,
+                statsLogger,
+                UnpooledByteBufAllocator.DEFAULT);
+
+        return ledgerStorage;
+    }
+
+    public BookieImpl(ServerConfiguration conf, StatsLogger statsLogger,
+                      ByteBufAllocator allocator, Supplier<BookieServiceInfo> bookieServiceInfoProvider)
+            throws IOException, InterruptedException, BookieException {
+        super("Bookie-" + conf.getBookiePort());
+        this.bookieServiceInfoProvider = bookieServiceInfoProvider;
+        this.statsLogger = statsLogger;
+        this.conf = conf;
+        this.journalDirectories = Lists.newArrayList();
+        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.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);
+        }
+        stateManager = initializeStateManager();
+        // register shutdown handler using trigger mode
+        stateManager.setShutdownHandler(exitCode -> triggerBookieShutdown(exitCode));
+        // Initialise dirsMonitor. This would look through all the
+        // configured directories. When disk errors or all the ledger
+        // directories are full, would throws exception and fail bookie startup.
+        List<LedgerDirsManager> dirsManagers = new ArrayList<>();
+        dirsManagers.add(ledgerDirsManager);
+        if (indexDirsManager != ledgerDirsManager) {
+            dirsManagers.add(indexDirsManager);
+        }
+        this.dirsMonitor = new LedgerDirsMonitor(conf, diskChecker, dirsManagers);
+        try {
+            this.dirsMonitor.init();
+        } catch (NoWritableLedgerDirException nle) {
+            // start in read-only mode if no writable dirs and read-only allowed
+            if (!conf.isReadOnlyModeEnabled()) {
+                throw nle;
+            } else {
+                this.stateManager.transitionToReadOnlyMode();
+            }
+        }
+
+        // instantiate the journals
+        journals = Lists.newArrayList();
+        for (int i = 0; i < journalDirectories.size(); i++) {
+            journals.add(new Journal(i, journalDirectories.get(i),
+                    conf, ledgerDirsManager, statsLogger.scope(JOURNAL_SCOPE), allocator));
+        }
+
+        this.entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled();
+        CheckpointSource checkpointSource = new CheckpointSourceList(journals);
+
+        ledgerStorage = buildLedgerStorage(conf);
+
+        boolean isDbLedgerStorage = ledgerStorage instanceof DbLedgerStorage;
+
+        /*
+         * with this change https://github.com/apache/bookkeeper/pull/677,
+         * LedgerStorage drives the checkpoint logic.
+         *
+         * <p>There are two exceptions:
+         *
+         * 1) with multiple entry logs, checkpoint logic based on a entry log is
+         *    not possible, hence it needs to be timebased recurring thing and
+         *    it is driven by SyncThread. SyncThread.start does that and it is
+         *    started in Bookie.start method.
+         *
+         * 2) DbLedgerStorage
+         */
+        if (entryLogPerLedgerEnabled || isDbLedgerStorage) {
+            syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource) {
+                @Override
+                public void startCheckpoint(Checkpoint checkpoint) {
+                    /*
+                     * in the case of entryLogPerLedgerEnabled, LedgerStorage
+                     * dont drive checkpoint logic, but instead it is done
+                     * periodically by SyncThread. So startCheckpoint which
+                     * will be called by LedgerStorage will be no-op.
+                     */
+                }
+
+                @Override
+                public void start() {
+                    executor.scheduleAtFixedRate(() -> {
+                        doCheckpoint(checkpointSource.newCheckpoint());
+                    }, conf.getFlushInterval(), conf.getFlushInterval(), TimeUnit.MILLISECONDS);
+                }
+            };
+        } else {
+            syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource);
+        }
+
+        ledgerStorage.initialize(
+            conf,
+            ledgerManager,
+            ledgerDirsManager,
+            indexDirsManager,
+            stateManager,
+            checkpointSource,
+            syncThread,
+            statsLogger,
+            allocator);
+
+
+        handles = new HandleFactoryImpl(ledgerStorage);
+
+        // Expose Stats
+        this.bookieStats = new BookieStats(statsLogger);
+        journalMemoryMaxStats = new Gauge<Long>() {
+            final long journalMaxMemory = conf.getJournalMaxMemorySizeMb() * 1024 * 1024;
+
+            @Override
+            public Long getDefaultValue() {
+                return journalMaxMemory;
+            }
+
+            @Override
+            public Long getSample() {
+                return journalMaxMemory;
+            }
+        };
+        statsLogger.scope(JOURNAL_SCOPE).registerGauge(JOURNAL_MEMORY_MAX, journalMemoryMaxStats);
+
+        journalMemoryUsedStats = new Gauge<Long>() {
+            @Override
+            public Long getDefaultValue() {
+                return -1L;
+            }
+
+            @Override
+            public Long getSample() {
+                long totalMemory = 0L;
+                for (int i = 0; i < journals.size(); i++) {
+                    totalMemory += journals.get(i).getMemoryUsage();
+                }
+                return totalMemory;
+            }
+        };
+        statsLogger.scope(JOURNAL_SCOPE).registerGauge(JOURNAL_MEMORY_USED, journalMemoryUsedStats);
+    }
+
+    StateManager initializeStateManager() throws IOException {
+        return new BookieStateManager(conf, statsLogger, metadataDriver,
+                ledgerDirsManager, bookieServiceInfoProvider);
+    }
+
+    void readJournal() throws IOException, BookieException {
+        long startTs = System.currentTimeMillis();
+        JournalScanner scanner = new JournalScanner() {
+            @Override
+            public void process(int journalVersion, long offset, ByteBuffer recBuff) throws IOException {
+                long ledgerId = recBuff.getLong();
+                long entryId = recBuff.getLong();
+                try {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Replay journal - ledger id : {}, entry id : {}.", ledgerId, entryId);
+                    }
+                    if (entryId == METAENTRY_ID_LEDGER_KEY) {
+                        if (journalVersion >= JournalChannel.V3) {
+                            int masterKeyLen = recBuff.getInt();
+                            byte[] masterKey = new byte[masterKeyLen];
+
+                            recBuff.get(masterKey);
+                            masterKeyCache.put(ledgerId, masterKey);
+
+                            // Force to re-insert the master key in ledger storage
+                            handles.getHandle(ledgerId, masterKey);
+                        } else {
+                            throw new IOException("Invalid journal. Contains journalKey "
+                                    + " but layout version (" + journalVersion
+                                    + ") is too old to hold this");
+                        }
+                    } else if (entryId == METAENTRY_ID_FENCE_KEY) {
+                        if (journalVersion >= JournalChannel.V4) {
+                            byte[] key = masterKeyCache.get(ledgerId);
+                            if (key == null) {
+                                key = ledgerStorage.readMasterKey(ledgerId);
+                            }
+                            LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+                            handle.setFenced();
+                        } else {
+                            throw new IOException("Invalid journal. Contains fenceKey "
+                                    + " but layout version (" + journalVersion
+                                    + ") is too old to hold this");
+                        }
+                    } else if (entryId == METAENTRY_ID_LEDGER_EXPLICITLAC) {
+                        if (journalVersion >= JournalChannel.V6) {
+                            int explicitLacBufLength = recBuff.getInt();
+                            ByteBuf explicitLacBuf = Unpooled.buffer(explicitLacBufLength);
+                            byte[] explicitLacBufArray = new byte[explicitLacBufLength];
+                            recBuff.get(explicitLacBufArray);
+                            explicitLacBuf.writeBytes(explicitLacBufArray);
+                            byte[] key = masterKeyCache.get(ledgerId);
+                            if (key == null) {
+                                key = ledgerStorage.readMasterKey(ledgerId);
+                            }
+                            LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+                            handle.setExplicitLac(explicitLacBuf);
+                        } else {
+                            throw new IOException("Invalid journal. Contains explicitLAC " + " but layout version ("
+                                    + journalVersion + ") is too old to hold this");
+                        }
+                    } else if (entryId < 0) {
+                        /*
+                         * this is possible if bookie code binary is rolledback
+                         * to older version but when it is trying to read
+                         * Journal which was created previously using newer
+                         * code/journalversion, which introduced new special
+                         * entry. So in anycase, if we see unrecognizable
+                         * special entry while replaying journal we should skip
+                         * (ignore) it.
+                         */
+                        LOG.warn("Read unrecognizable entryId: {} for ledger: {} while replaying Journal. Skipping it",
+                                entryId, ledgerId);
+                    } else {
+                        byte[] key = masterKeyCache.get(ledgerId);
+                        if (key == null) {
+                            key = ledgerStorage.readMasterKey(ledgerId);
+                        }
+                        LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+
+                        recBuff.rewind();
+                        handle.addEntry(Unpooled.wrappedBuffer(recBuff));
+                    }
+                } catch (NoLedgerException nsle) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Skip replaying entries of ledger {} since it was deleted.", ledgerId);
+                    }
+                } catch (BookieException be) {
+                    throw new IOException(be);
+                }
+            }
+        };
+
+        for (Journal journal : journals) {
+            replay(journal, scanner);
+        }
+        long elapsedTs = System.currentTimeMillis() - startTs;
+        LOG.info("Finished replaying journal in {} ms.", elapsedTs);
+    }
+
+    /**
+     * Replay journal files and updates journal's in-memory lastLogMark object.
+     *
+     * @param journal Journal object corresponding to a journalDir
+     * @param scanner Scanner to process replayed entries.
+     * @throws IOException
+     */
+    private void replay(Journal journal, JournalScanner scanner) throws IOException {
+        final LogMark markedLog = journal.getLastLogMark().getCurMark();
+        List<Long> logs = Journal.listJournalIds(journal.getJournalDirectory(), journalId ->
+            journalId >= markedLog.getLogFileId());
+        // last log mark may be missed due to no sync up before
+        // validate filtered log ids only when we have markedLogId
+        if (markedLog.getLogFileId() > 0) {
+            if (logs.size() == 0 || logs.get(0) != markedLog.getLogFileId()) {
+                throw new IOException("Recovery log " + markedLog.getLogFileId() + " is missing");
+            }
+        }
+
+        // TODO: When reading in the journal logs that need to be synced, we
+        // should use BufferedChannels instead to minimize the amount of
+        // system calls done.
+        for (Long id : logs) {
+            long logPosition = 0L;
+            if (id == markedLog.getLogFileId()) {
+                logPosition = markedLog.getLogFileOffset();
+            }
+            LOG.info("Replaying journal {} from position {}", id, logPosition);
+            long scanOffset = journal.scanJournal(id, logPosition, scanner);
+            // Update LastLogMark after completely replaying journal
+            // scanOffset will point to EOF position
+            // After LedgerStorage flush, SyncThread should persist this to disk
+            journal.setLastLogMark(id, scanOffset);
+        }
+    }
+
+    @Override
+    public synchronized void start() {
+        setDaemon(true);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("I'm starting a bookie with journal directories {}",
+                    journalDirectories.stream().map(File::getName).collect(Collectors.joining(", ")));
+        }
+        //Start DiskChecker thread
+        dirsMonitor.start();
+
+        // replay journals
+        try {
+            readJournal();
+        } catch (IOException | BookieException ioe) {
+            LOG.error("Exception while replaying journals, shutting down", ioe);
+            shutdown(ExitCode.BOOKIE_EXCEPTION);
+            return;
+        }
+
+        // Do a fully flush after journal replay
+        try {
+            syncThread.requestFlush().get();
+        } catch (InterruptedException e) {
+            LOG.warn("Interrupting the fully flush after replaying journals : ", e);
+            Thread.currentThread().interrupt();
+        } catch (ExecutionException e) {
+            LOG.error("Error on executing a fully flush after replaying journals.");
+            shutdown(ExitCode.BOOKIE_EXCEPTION);
+            return;
+        }
+
+        if (conf.isLocalConsistencyCheckOnStartup()) {
+            LOG.info("Running local consistency check on startup prior to accepting IO.");
+            List<LedgerStorage.DetectedInconsistency> errors = null;
+            try {
+                errors = ledgerStorage.localConsistencyCheck(Optional.empty());
+            } catch (IOException e) {
+                LOG.error("Got a fatal exception while checking store", e);
+                shutdown(ExitCode.BOOKIE_EXCEPTION);
+                return;
+            }
+            if (errors != null && errors.size() > 0) {
+                LOG.error("Bookie failed local consistency check:");
+                for (LedgerStorage.DetectedInconsistency error : errors) {
+                    LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException());
+                }
+                shutdown(ExitCode.BOOKIE_EXCEPTION);
+                return;
+            }
+        }
+
+        LOG.info("Finished reading journal, starting bookie");
+
+
+        /*
+         * start sync thread first, so during replaying journals, we could do
+         * checkpoint which reduce the chance that we need to replay journals
+         * again if bookie restarted again before finished journal replays.
+         */
+        syncThread.start();
+
+        // start bookie thread
+        super.start();
+
+        // After successful bookie startup, register listener for disk
+        // error/full notifications.
+        ledgerDirsManager.addLedgerDirsListener(getLedgerDirsListener());
+        if (indexDirsManager != ledgerDirsManager) {
+            indexDirsManager.addLedgerDirsListener(getLedgerDirsListener());
+        }
+
+        ledgerStorage.start();
+
+        // check the bookie status to start with, and set running.
+        // since bookie server use running as a flag to tell bookie server whether it is alive
+        // if setting it in bookie thread, the watcher might run before bookie thread.
+        stateManager.initState();
+
+        try {
+            stateManager.registerBookie(true).get();
+        } catch (Exception e) {
+            LOG.error("Couldn't register bookie with zookeeper, shutting down : ", e);
+            shutdown(ExitCode.ZK_REG_FAIL);
+        }
+    }
+
+    /*
+     * Get the DiskFailure listener for the bookie
+     */
+    private LedgerDirsListener getLedgerDirsListener() {
+
+        return new LedgerDirsListener() {
+
+            @Override
+            public void diskFailed(File disk) {
+                // Shutdown the bookie on disk failure.
+                triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
+            }
+
+            @Override
+            public void allDisksFull(boolean highPriorityWritesAllowed) {
+                // Transition to readOnly mode on all disks full
+                stateManager.setHighPriorityWritesAvailability(highPriorityWritesAllowed);
+                stateManager.transitionToReadOnlyMode();
+            }
+
+            @Override
+            public void fatalError() {
+                LOG.error("Fatal error reported by ledgerDirsManager");
+                triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
+            }
+
+            @Override
+            public void diskWritable(File disk) {
+                // Transition to writable mode when a disk becomes writable again.
+                stateManager.setHighPriorityWritesAvailability(true);
+                stateManager.transitionToWritableMode();
+            }
+
+            @Override
+            public void diskJustWritable(File disk) {
+                // Transition to writable mode when a disk becomes writable again.
+                stateManager.setHighPriorityWritesAvailability(true);
+                stateManager.transitionToWritableMode();
+            }
+        };
+    }
+
+    /**
+     * 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.
+     */
+    public boolean isReadOnly() {
+        return stateManager.isReadOnly();
+    }
+
+    /**
+     * Check whether Bookie is available for high priority writes.
+     *
+     * @return true if the bookie is able to take high priority writes.
+     */
+    public boolean isAvailableForHighPriorityWrites() {
+        return stateManager.isAvailableForHighPriorityWrites();
+    }
+
+    public boolean isRunning() {
+        return stateManager.isRunning();
+    }
+
+    @Override
+    public void run() {
+        // bookie thread wait for journal thread
+        try {
+            // start journals
+            for (Journal journal: journals) {
+                journal.start();
+            }
+
+            // wait until journal quits
+            for (Journal journal: journals) {
+
+                journal.joinThread();
+            }
+            LOG.info("Journal thread(s) quit.");
+        } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+            LOG.warn("Interrupted on running journal thread : ", ie);
+        }
+        // if the journal thread quits due to shutting down, it is ok
+        if (!stateManager.isShuttingDown()) {
+            // some error found in journal thread and it quits
+            // following add operations to it would hang unit client timeout
+            // so we should let bookie server exists
+            LOG.error("Journal manager quits unexpectedly.");
+            triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
+        }
+    }
+
+    // Triggering the Bookie shutdown in its own thread,
+    // because shutdown can be called from sync thread which would be
+    // interrupted by shutdown call.
+    AtomicBoolean shutdownTriggered = new AtomicBoolean(false);
+    void triggerBookieShutdown(final int exitCode) {
+        if (!shutdownTriggered.compareAndSet(false, true)) {
+            return;
+        }
+        LOG.info("Triggering shutdown of Bookie-{} with exitCode {}",
+                 conf.getBookiePort(), exitCode);
+        BookieThread th = new BookieThread("BookieShutdownTrigger") {
+            @Override
+            public void run() {
+                BookieImpl.this.shutdown(exitCode);
+            }
+        };
+        th.start();
+    }
+
+    // provided a public shutdown method for other caller
+    // to shut down bookie gracefully
+    public int shutdown() {
+        return shutdown(ExitCode.OK);
+    }
+
+    // internal shutdown method to let shutdown bookie gracefully
+    // when encountering exception
+    synchronized int shutdown(int exitCode) {
+        try {
+            if (isRunning()) { // avoid shutdown twice
+                // the exitCode only set when first shutdown usually due to exception found
+                LOG.info("Shutting down Bookie-{} with exitCode {}",
+                         conf.getBookiePort(), exitCode);
+                if (this.exitCode == ExitCode.OK) {
+                    this.exitCode = exitCode;
+                }
+
+                stateManager.forceToShuttingDown();
+
+                // turn bookie to read only during shutting down process
+                LOG.info("Turning bookie to read only during shut down");
+                stateManager.forceToReadOnly();
+
+                // Shutdown Sync thread
+                syncThread.shutdown();
+
+                // Shutdown journals
+                for (Journal journal : journals) {
+                    journal.shutdown();
+                }
+                this.join();
+
+                // Shutdown the EntryLogger which has the GarbageCollector Thread running
+                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);
+        } catch (Exception e) {
+            LOG.error("Got Exception while trying to shutdown Bookie", e);
+            throw e;
+        } finally {
+            // setting running to false here, so watch thread
+            // in bookie server know it only after bookie shut down
+            stateManager.close();
+        }
+        return this.exitCode;
+    }
+
+    /**
+     * Retrieve the ledger descriptor for the ledger which entry should be added to.
+     * The LedgerDescriptor returned from this method should be eventually freed with
+     * #putHandle().
+     *
+     * @throws BookieException if masterKey does not match the master key of the ledger
+     */
+    @VisibleForTesting
+    LedgerDescriptor getLedgerForEntry(ByteBuf entry, final byte[] masterKey)
+            throws IOException, BookieException {
+        final long ledgerId = entry.getLong(entry.readerIndex());
+
+        return handles.getHandle(ledgerId, masterKey);
+    }
+
+    private Journal getJournal(long ledgerId) {
+        return journals.get(MathUtils.signSafeMod(ledgerId, journals.size()));
+    }
+
+    /**
+     * Add an entry to a ledger as specified by handle.
+     */
+    private void addEntryInternal(LedgerDescriptor handle, ByteBuf entry,
+                                  boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
+            throws IOException, BookieException, InterruptedException {
+        long ledgerId = handle.getLedgerId();
+        long entryId = handle.addEntry(entry);
+
+        bookieStats.getWriteBytes().add(entry.readableBytes());
+
+        // journal `addEntry` should happen after the entry is added to ledger storage.
+        // otherwise the journal entry can potentially be rolled before the ledger is created in ledger storage.
+        if (masterKeyCache.get(ledgerId) == null) {
+            // Force the load into masterKey cache
+            byte[] oldValue = masterKeyCache.putIfAbsent(ledgerId, masterKey);
+            if (oldValue == null) {
+                // new handle, we should add the key to journal ensure we can rebuild
+                ByteBuffer bb = ByteBuffer.allocate(8 + 8 + 4 + masterKey.length);
+                bb.putLong(ledgerId);
+                bb.putLong(METAENTRY_ID_LEDGER_KEY);
+                bb.putInt(masterKey.length);
+                bb.put(masterKey);
+                bb.flip();
+
+                getJournal(ledgerId).logAddEntry(bb, false /* ackBeforeSync */, new NopWriteCallback(), null);
+            }
+        }
+
+        if (!writeDataToJournal) {
+            cb.writeComplete(0, ledgerId, entryId, null, ctx);
+            return;
+        }
+
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Adding {}@{}", entryId, ledgerId);
+        }
+        getJournal(ledgerId).logAddEntry(entry, ackBeforeSync, cb, ctx);
+    }
+
+    /**
+     * Add entry to a ledger, even if the ledger has previous been fenced. This should only
+     * happen in bookie recovery or ledger recovery cases, where entries are being replicates
+     * so that they exist on a quorum of bookies. The corresponding client side call for this
+     * is not exposed to users.
+     */
+    public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
+            throws IOException, BookieException, InterruptedException {
+        long requestNanos = MathUtils.nowInNano();
+        boolean success = false;
+        int entrySize = 0;
+        try {
+            LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
+            synchronized (handle) {
+                entrySize = entry.readableBytes();
+                addEntryInternal(handle, entry, false /* ackBeforeSync */, cb, ctx, masterKey);
+            }
+            success = true;
+        } catch (NoWritableLedgerDirException e) {
+            stateManager.transitionToReadOnlyMode();
+            throw new IOException(e);
+        } finally {
+            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
+            if (success) {
+                bookieStats.getRecoveryAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize);
+            } else {
+                bookieStats.getRecoveryAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getAddBytesStats().registerFailedValue(entrySize);
+            }
+
+            entry.release();
+        }
+    }
+
+    private ByteBuf createExplicitLACEntry(long ledgerId, ByteBuf explicitLac) {
+        ByteBuf bb = allocator.directBuffer(8 + 8 + 4 + explicitLac.capacity());
+        bb.writeLong(ledgerId);
+        bb.writeLong(METAENTRY_ID_LEDGER_EXPLICITLAC);
+        bb.writeInt(explicitLac.capacity());
+        bb.writeBytes(explicitLac);
+        return bb;
+    }
+
+    public void setExplicitLac(ByteBuf entry, WriteCallback writeCallback, Object ctx, byte[] masterKey)
+            throws IOException, InterruptedException, BookieException {
+        try {
+            long ledgerId = entry.getLong(entry.readerIndex());
+            LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey);
+            synchronized (handle) {
+                entry.markReaderIndex();
+                handle.setExplicitLac(entry);
+                entry.resetReaderIndex();
+                ByteBuf explicitLACEntry = createExplicitLACEntry(ledgerId, entry);
+                getJournal(ledgerId).logAddEntry(explicitLACEntry, false /* ackBeforeSync */, writeCallback, ctx);
+            }
+        } catch (NoWritableLedgerDirException e) {
+            stateManager.transitionToReadOnlyMode();
+            throw new IOException(e);
+        }
+    }
+
+    public ByteBuf getExplicitLac(long ledgerId) throws IOException, Bookie.NoLedgerException {
+        ByteBuf lac;
+        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+        synchronized (handle) {
+            lac = handle.getExplicitLac();
+        }
+        return lac;
+    }
+
+    /**
+     * Force sync given 'ledgerId' entries on the journal to the disk.
+     * It works like a regular addEntry with ackBeforeSync=false.
+     * This is useful for ledgers with DEFERRED_SYNC write flag.
+     */
+    public void forceLedger(long ledgerId, WriteCallback cb,
+                            Object ctx) {
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Forcing ledger {}", ledgerId);
+        }
+        Journal journal = getJournal(ledgerId);
+        journal.forceLedger(ledgerId, cb, ctx);
+        bookieStats.getForceLedgerOps().inc();
+    }
+
+    /**
+     * Add entry to a ledger.
+     */
+    public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
+            throws IOException, BookieException, InterruptedException {
+        long requestNanos = MathUtils.nowInNano();
+        boolean success = false;
+        int entrySize = 0;
+        try {
+            LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
+            synchronized (handle) {
+                if (handle.isFenced()) {
+                    throw BookieException
+                            .create(BookieException.Code.LedgerFencedException);
+                }
+                entrySize = entry.readableBytes();
+                addEntryInternal(handle, entry, ackBeforeSync, cb, ctx, masterKey);
+            }
+            success = true;
+        } catch (NoWritableLedgerDirException e) {
+            stateManager.transitionToReadOnlyMode();
+            throw new IOException(e);
+        } finally {
+            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
+            if (success) {
+                bookieStats.getAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize);
+            } else {
+                bookieStats.getAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getAddBytesStats().registerFailedValue(entrySize);
+            }
+
+            entry.release();
+        }
+    }
+
+    /**
+     * Fences a ledger. From this point on, clients will be unable to
+     * write to this ledger. Only recoveryAddEntry will be
+     * able to add entries to the ledger.
+     * This method is idempotent. Once a ledger is fenced, it can
+     * never be unfenced. Fencing a fenced ledger has no effect.
+     * @return
+     */
+    public CompletableFuture<Boolean> fenceLedger(long ledgerId, byte[] masterKey)
+            throws IOException, BookieException {
+        LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey);
+        return handle.fenceAndLogInJournal(getJournal(ledgerId));
+    }
+
+    public ByteBuf readEntry(long ledgerId, long entryId)
+            throws IOException, NoLedgerException {
+        long requestNanos = MathUtils.nowInNano();
+        boolean success = false;
+        int entrySize = 0;
+        try {
+            LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Reading {}@{}", entryId, ledgerId);
+            }
+            ByteBuf entry = handle.readEntry(entryId);
+            bookieStats.getReadBytes().add(entry.readableBytes());
+            success = true;
+            return entry;
+        } finally {
+            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
+            if (success) {
+                bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getReadBytesStats().registerSuccessfulValue(entrySize);
+            } else {
+                bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+                bookieStats.getReadEntryStats().registerFailedValue(entrySize);
+            }
+        }
+    }
+
+    public long readLastAddConfirmed(long ledgerId) throws IOException {
+        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+        return handle.getLastAddConfirmed();
+    }
+
+    public boolean waitForLastAddConfirmedUpdate(long ledgerId,
+                                                 long previousLAC,
+                                                 Watcher<LastAddConfirmedUpdateNotification> watcher)
+            throws IOException {
+        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+        return handle.waitForLastAddConfirmedUpdate(previousLAC, watcher);
+    }
+
+    public void cancelWaitForLastAddConfirmedUpdate(long ledgerId,
+                                                    Watcher<LastAddConfirmedUpdateNotification> watcher)
+            throws IOException {
+        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+        handle.cancelWaitForLastAddConfirmedUpdate(watcher);
+    }
+
+    @VisibleForTesting
+    public LedgerStorage getLedgerStorage() {
+        return ledgerStorage;
+    }
+
+    @VisibleForTesting
+    public BookieStateManager getStateManager() {
+        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;
+    }
+
+    /**
+     * Format the bookie server data.
+     *
+     * @param conf ServerConfiguration
+     * @param isInteractive Whether format should ask prompt for confirmation if old data exists or not.
+     * @param force If non interactive and force is true, then old data will be removed without confirm prompt.
+     * @return Returns true if the format is success else returns false
+     */
+    public static boolean format(ServerConfiguration conf,
+            boolean isInteractive, boolean force) {
+        for (File journalDir : conf.getJournalDirs()) {
+            String[] journalDirFiles =
+                    journalDir.exists() && journalDir.isDirectory() ? journalDir.list() : null;
+            if (journalDirFiles != null && journalDirFiles.length != 0) {
+                try {
+                    boolean confirm = false;
+                    if (!isInteractive) {
+                        // If non interactive and force is set, then delete old
+                        // data.
+                        confirm = force;
+                    } else {
+                        confirm = IOUtils
+                                .confirmPrompt("Are you sure to format Bookie data..?");
+                    }
+
+                    if (!confirm) {
+                        LOG.error("Bookie format aborted!!");
+                        return false;
+                    }
+                } catch (IOException e) {
+                    LOG.error("Error during bookie format", e);
+                    return false;
+                }
+            }
+            if (!cleanDir(journalDir)) {
+                LOG.error("Formatting journal directory failed");
+                return false;
+            }
+        }
+
+        File[] ledgerDirs = conf.getLedgerDirs();
+        for (File dir : ledgerDirs) {
+            if (!cleanDir(dir)) {
+                LOG.error("Formatting ledger directory " + dir + " failed");
+                return false;
+            }
+        }
+
+        // Clean up index directories if they are separate from the ledger dirs
+        File[] indexDirs = conf.getIndexDirs();
+        if (null != indexDirs) {
+            for (File dir : indexDirs) {
+                if (!cleanDir(dir)) {
+                    LOG.error("Formatting index directory " + dir + " failed");
+                    return false;
+                }
+            }
+        }
+
+        LOG.info("Bookie format completed successfully");
+        return true;
+    }
+
+    private static boolean cleanDir(File dir) {
+        if (dir.exists()) {
+            File[] files = dir.listFiles();
+            if (files != null) {
+                for (File child : files) {
+                    boolean delete = FileUtils.deleteQuietly(child);
+                    if (!delete) {
+                        LOG.error("Not able to delete " + child);
+                        return false;
+                    }
+                }
+            }
+        } else if (!dir.mkdirs()) {
+            LOG.error("Not able to create the directory " + dir);
+            return false;
+        }
+        return true;
+    }
+
+    /**
+     * Returns exit code - cause of failure.
+     *
+     * @return {@link ExitCode}
+     */
+    public int getExitCode() {
+        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;
+        try {
+            LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("GetEntriesOfLedger {}", ledgerId);
+            }
+            OfLong entriesOfLedger = handle.getListOfEntriesOfLedger(ledgerId);
+            success = true;
+            return entriesOfLedger;
+        } finally {
+            long elapsedNanos = MathUtils.elapsedNanos(requestNanos);
+            if (success) {
+                bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+            } else {
+                bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS);
+            }
+        }
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
index 0da44a7..c16efca 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
@@ -2186,12 +2186,12 @@
     @Override
     public void setConf(CompositeConfiguration conf) throws Exception {
         bkConf.loadConf(conf);
-        journalDirectories = Bookie.getCurrentDirectories(bkConf.getJournalDirs());
-        ledgerDirectories = Bookie.getCurrentDirectories(bkConf.getLedgerDirs());
+        journalDirectories = BookieImpl.getCurrentDirectories(bkConf.getJournalDirs());
+        ledgerDirectories = BookieImpl.getCurrentDirectories(bkConf.getLedgerDirs());
         if (null == bkConf.getIndexDirs()) {
             indexDirectories = ledgerDirectories;
         } else {
-            indexDirectories = Bookie.getCurrentDirectories(bkConf.getIndexDirs());
+            indexDirectories = BookieImpl.getCurrentDirectories(bkConf.getIndexDirs());
         }
         pageSize = bkConf.getPageSize();
         entriesPerPage = pageSize / 8;
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 0965124..d27209c 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
@@ -102,7 +102,7 @@
             ledgerDirsManager.getAllLedgerDirs(),
             () -> {
                 try {
-                    return Bookie.getBookieId(conf);
+                    return BookieImpl.getBookieId(conf);
                 } catch (UnknownHostException e) {
                     throw new UncheckedIOException("Failed to resolve bookie id", e);
                 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
index 70e3bb6..e548cbd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
@@ -236,7 +236,7 @@
             throws BookieException {
         BookieId address = null;
         try {
-            address = Bookie.getBookieId(conf);
+            address = BookieImpl.getBookieId(conf);
         } catch (UnknownHostException e) {
             throw new UnknownBookieIdException(e);
         }
@@ -257,7 +257,7 @@
                                               Version version) throws BookieException {
         BookieId address = null;
         try {
-            address = Bookie.getBookieId(conf);
+            address = BookieImpl.getBookieId(conf);
         } catch (UnknownHostException e) {
             throw new UnknownBookieIdException(e);
         }
@@ -293,7 +293,7 @@
             throws UnknownHostException {
         Builder builder = Cookie.newBuilder();
         builder.setLayoutVersion(CURRENT_COOKIE_LAYOUT_VERSION);
-        builder.setBookieId(Bookie.getBookieId(conf).toString());
+        builder.setBookieId(BookieImpl.getBookieId(conf).toString());
         builder.setJournalDirs(Joiner.on(',').join(conf.getJournalDirNames()));
         builder.setLedgerDirs(encodeDirPaths(conf.getLedgerDirNames()));
         return builder;
@@ -310,7 +310,7 @@
     public static Versioned<Cookie> readFromRegistrationManager(RegistrationManager rm, ServerConfiguration conf)
             throws BookieException {
         try {
-            return readFromRegistrationManager(rm, Bookie.getBookieId(conf));
+            return readFromRegistrationManager(rm, BookieImpl.getBookieId(conf));
         } catch (UnknownHostException e) {
             throw new UnknownBookieIdException(e);
         }
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 90f5afc..e0c86e8 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 void initiate(boolean dryRun) throws IOException {
         LOG.info("Starting index rebuilding");
 
-        DiskChecker diskChecker = Bookie.createDiskChecker(conf);
-        LedgerDirsManager ledgerDirsManager = Bookie.createLedgerDirsManager(
+        DiskChecker diskChecker = BookieImpl.createDiskChecker(conf);
+        LedgerDirsManager ledgerDirsManager = BookieImpl.createLedgerDirsManager(
                 conf, diskChecker, NullStatsLogger.INSTANCE);
-        LedgerDirsManager indexDirsManager = Bookie.createIndexDirsManager(
+        LedgerDirsManager indexDirsManager = BookieImpl.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/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
index 02e5de5..1d35b32 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
@@ -882,7 +882,7 @@
     void forceLedger(long ledgerId, WriteCallback cb, Object ctx) {
         queue.add(QueueEntry.create(
                 null, false /* ackBeforeSync */, ledgerId,
-                Bookie.METAENTRY_ID_FORCE_LEDGER, cb, ctx, MathUtils.nowInNano(),
+                BookieImpl.METAENTRY_ID_FORCE_LEDGER, cb, ctx, MathUtils.nowInNano(),
                 journalStats.getJournalForceLedgerStats(),
                 journalStats.getJournalCbQueueSize()));
         // Increment afterwards because the add operation could fail.
@@ -1109,7 +1109,7 @@
                 if (qe == null) { // no more queue entry
                     continue;
                 }
-                if ((qe.entryId == Bookie.METAENTRY_ID_LEDGER_EXPLICITLAC)
+                if ((qe.entryId == BookieImpl.METAENTRY_ID_LEDGER_EXPLICITLAC)
                         && (journalFormatVersionToWrite < JournalChannel.V6)) {
                     /*
                      * this means we are using new code which supports
@@ -1120,7 +1120,7 @@
                      */
                     memoryLimitController.releaseMemory(qe.entry.readableBytes());
                     qe.entry.release();
-                } else if (qe.entryId != Bookie.METAENTRY_ID_FORCE_LEDGER) {
+                } else if (qe.entryId != BookieImpl.METAENTRY_ID_FORCE_LEDGER) {
                     int entrySize = qe.entry.readableBytes();
                     journalStats.getJournalWriteBytes().add(entrySize);
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
index bb5f350..a862b7b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
@@ -21,14 +21,12 @@
 
 package org.apache.bookkeeper.bookie;
 
-import static org.apache.bookkeeper.bookie.Bookie.METAENTRY_ID_FENCE_KEY;
-
-import com.google.common.util.concurrent.SettableFuture;
+import static org.apache.bookkeeper.bookie.BookieImpl.METAENTRY_ID_FENCE_KEY;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import java.io.IOException;
 import java.util.PrimitiveIterator.OfLong;
-
+import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.common.util.Watcher;
 
 /**
@@ -72,8 +70,9 @@
      * then log the fence entry in Journal so that we can rebuild the state.
      *
      * <p>We should satisfy the future only after we complete logging fence entry in Journal
+     * @return
      */
-    abstract SettableFuture<Boolean> fenceAndLogInJournal(Journal journal) throws IOException;
+    abstract CompletableFuture<Boolean> fenceAndLogInJournal(Journal journal) throws IOException;
 
     abstract long addEntry(ByteBuf entry) throws IOException, BookieException;
     abstract ByteBuf readEntry(long entryId) throws IOException;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
index cf54b8e..2496d69 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
@@ -21,13 +21,14 @@
 
 package org.apache.bookkeeper.bookie;
 
-import com.google.common.util.concurrent.SettableFuture;
 import io.netty.buffer.ByteBuf;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.PrimitiveIterator.OfLong;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.bookkeeper.client.api.BKException;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.common.util.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +44,7 @@
     final byte[] masterKey;
 
     private AtomicBoolean fenceEntryPersisted = new AtomicBoolean();
-    private SettableFuture<Boolean> logFenceResult = null;
+    private CompletableFuture<Boolean> logFenceResult = null;
 
     LedgerDescriptorImpl(byte[] masterKey,
                          long ledgerId,
@@ -88,7 +89,7 @@
     }
 
     @Override
-    synchronized SettableFuture<Boolean> fenceAndLogInJournal(Journal journal) throws IOException {
+    synchronized CompletableFuture<Boolean> fenceAndLogInJournal(Journal journal) throws IOException {
         boolean success = this.setFenced();
         if (success) {
             // fenced for first time, we should add the key to journal ensure we can rebuild.
@@ -101,8 +102,8 @@
             if (logFenceResult == null || fenceEntryPersisted.get()){
                 // Either ledger's fenced state is recovered from Journal
                 // Or Log fence entry in Journal succeed
-                SettableFuture<Boolean> result = SettableFuture.create();
-                result.set(true);
+                CompletableFuture<Boolean> result = FutureUtils.createFuture();
+                result.complete(true);
                 return result;
             } else if (logFenceResult.isDone()) {
                 // We failed to log fence entry in Journal, try again.
@@ -118,10 +119,10 @@
      * @param journal log the fence entry in the Journal
      * @return A future which will be satisfied when add entry to journal complete
      */
-    private SettableFuture<Boolean> logFenceEntryInJournal(Journal journal) {
-        SettableFuture<Boolean> result;
+    private CompletableFuture<Boolean> logFenceEntryInJournal(Journal journal) {
+        CompletableFuture<Boolean> result;
         synchronized (this) {
-            result = logFenceResult = SettableFuture.create();
+            result = logFenceResult = FutureUtils.createFuture();
         }
         ByteBuf entry = createLedgerFenceEntry(ledgerId);
         try {
@@ -132,14 +133,14 @@
                 }
                 if (rc == 0) {
                     fenceEntryPersisted.compareAndSet(false, true);
-                    result.set(true);
+                    result.complete(true);
                 } else {
-                    result.set(false);
+                    result.complete(false);
                 }
             }, null);
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
-            result.setException(e);
+            result.completeExceptionally(e);
         }
         return result;
     }
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 5fb8215..2768aff 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
@@ -63,8 +63,7 @@
     }
 
     public LedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker, StatsLogger statsLogger) {
-        this.ledgerDirectories = Arrays.asList(Bookie
-                .getCurrentDirectories(dirs));
+        this.ledgerDirectories = Arrays.asList(BookieImpl.getCurrentDirectories(dirs));
         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/LocalBookieEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
index d44e576..5440497 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
@@ -62,7 +62,7 @@
         serverConf.addConfiguration(conf);
 
         try {
-            bookieAddress = Bookie.getBookieId(serverConf);
+            bookieAddress = BookieImpl.getBookieId(serverConf);
         } catch (UnknownHostException e) {
             LOG.warn("Unable to get bookie address", e);
             throw new RuntimeException(e);
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 3c84a06..e6cdf64 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
@@ -38,7 +38,7 @@
  * ReadOnlyBookie is force started as readonly, and will not change to writable.
  * </p>
  */
-public class ReadOnlyBookie extends Bookie {
+public class ReadOnlyBookie extends BookieImpl {
 
     private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyBookie.class);
 
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 cff0250..9dd6a83 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
@@ -94,7 +94,8 @@
         this.ledgerManager = ledgerManager;
         this.ledgerStorage = ledgerStorage;
         this.conf = conf;
-        this.selfBookieAddress = Bookie.getBookieId(conf);
+        this.selfBookieAddress = BookieImpl.getBookieId(conf);
+
         this.gcOverReplicatedLedgerIntervalMillis = conf.getGcOverreplicatedLedgerWaitTimeMillis();
         this.lastOverReplicatedLedgerGcTimeMillis = System.currentTimeMillis();
         if (gcOverReplicatedLedgerIntervalMillis > 0) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
index f23275a..6c9ca2c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
@@ -33,7 +33,7 @@
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.EntryLogger;
 import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -59,7 +59,7 @@
         LOG.info("Starting index rebuilding");
 
         // Move locations index to a backup directory
-        String basePath = Bookie.getCurrentDirectory(conf.getLedgerDirs()[0]).toString();
+        String basePath = BookieImpl.getCurrentDirectory(conf.getLedgerDirs()[0]).toString();
         Path currentPath = FileSystems.getDefault().getPath(basePath, "locations");
         String timestamp = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ").format(new Date());
         Path backupPath = FileSystems.getDefault().getPath(basePath, "locations.BACKUP-" + timestamp);
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 7dbf4df..4f73374 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
@@ -23,12 +23,10 @@
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver;
 import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager;
-
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.UncheckedExecutionException;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -52,9 +50,8 @@
 import java.util.function.BiConsumer;
 import java.util.function.Predicate;
 import lombok.SneakyThrows;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
 import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
 import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence;
@@ -1298,7 +1295,7 @@
                  * make sure that there is no bookie registered with the same
                  * bookieid and the cookie for the same bookieid is not existing.
                  */
-                BookieId bookieId = Bookie.getBookieId(conf);
+                BookieId bookieId = BookieImpl.getBookieId(conf);
                 if (rm.isBookieRegistered(bookieId)) {
                     LOG.error("Bookie with bookieId: {} is still registered, "
                         + "If this node is running bookie process, try stopping it first.", bookieId);
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 93a93ba..17f89de 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
@@ -22,7 +22,6 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ExtensionRegistry;
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.PooledByteBufAllocator;
@@ -53,7 +52,6 @@
 import io.netty.handler.codec.LengthFieldPrepender;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.util.concurrent.DefaultThreadFactory;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
@@ -67,14 +65,12 @@
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import javax.net.ssl.SSLPeerUnverifiedException;
-
 import org.apache.bookkeeper.auth.AuthProviderFactoryFactory;
 import org.apache.bookkeeper.auth.BookKeeperPrincipal;
 import org.apache.bookkeeper.auth.BookieAuthProvider;
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.common.collections.BlockingMpscQueue;
 import org.apache.bookkeeper.common.util.affinity.CpuAffinity;
 import org.apache.bookkeeper.conf.ServerConfiguration;
@@ -163,8 +159,8 @@
         } else {
             jvmEventLoopGroup = null;
         }
-        bookieId = Bookie.getBookieId(conf);
-        bookieAddress = Bookie.getBookieAddress(conf);
+        bookieId = BookieImpl.getBookieId(conf);
+        bookieAddress = BookieImpl.getBookieAddress(conf);
         if (conf.getListeningInterface() == null) {
             bindAddress = new InetSocketAddress(conf.getBookiePort());
         } else {
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 ee83327..4e6a782 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
@@ -23,10 +23,8 @@
 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;
 import io.netty.buffer.ByteBufAllocator;
-
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.UnknownHostException;
@@ -34,10 +32,10 @@
 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;
@@ -128,7 +126,7 @@
         shFactory = SecurityProviderFactoryFactory
                 .getSecurityProviderFactory(conf.getTLSProviderFactoryClass());
         this.requestProcessor = new BookieRequestProcessor(conf, bookie,
-                statsLogger.scope(SERVER_SCOPE), shFactory, bookie.getAllocator());
+                statsLogger.scope(SERVER_SCOPE), shFactory, allocator);
         this.nettyServer.setRequestProcessor(this.requestProcessor);
     }
 
@@ -149,7 +147,7 @@
         throws IOException, KeeperException, InterruptedException, BookieException {
         return conf.isForceReadOnlyBookie()
             ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider)
-            : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider);
+            : new BookieImpl(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider);
     }
 
     public void start() throws InterruptedException {
@@ -176,12 +174,12 @@
 
     @VisibleForTesting
     public BookieSocketAddress getLocalAddress() throws UnknownHostException {
-        return Bookie.getBookieAddress(conf);
+        return BookieImpl.getBookieAddress(conf);
     }
 
     @VisibleForTesting
     public BookieId getBookieId() throws UnknownHostException {
-        return Bookie.getBookieId(conf);
+        return BookieImpl.getBookieId(conf);
     }
 
     @VisibleForTesting
@@ -339,7 +337,7 @@
         String addr = "UNKNOWN";
         String id = "?";
         try {
-            addr = Bookie.getBookieAddress(conf).toString();
+            addr = BookieImpl.getBookieAddress(conf).toString();
             id = getBookieId().toString();
         } catch (UnknownHostException e) {
             //Ignored...
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java
index 55aa97b..2ac8988 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java
@@ -24,7 +24,7 @@
 
 import io.netty.channel.Channel;
 import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerResponse;
@@ -60,7 +60,7 @@
         BookkeeperInternalCallbacks.WriteCallback wcb =
                 (int rc, long ledgerId1, long entryId, BookieId addr, Object ctx) -> {
 
-            checkArgument(entryId == Bookie.METAENTRY_ID_FORCE_LEDGER,
+            checkArgument(entryId == BookieImpl.METAENTRY_ID_FORCE_LEDGER,
                     "entryId must be METAENTRY_ID_FORCE_LEDGER but was {}", entryId);
 
             checkArgument(ledgerId1 == ledgerId,
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
index a760383..c8313bb 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
@@ -17,23 +17,19 @@
  */
 package org.apache.bookkeeper.proto;
 
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
 import io.netty.util.Recycler;
 import io.netty.util.ReferenceCountUtil;
-
 import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.util.MathUtils;
@@ -68,7 +64,7 @@
         long startTimeNanos = MathUtils.nowInNano();
         ByteBuf data = null;
         try {
-            SettableFuture<Boolean> fenceResult = null;
+            CompletableFuture<Boolean> fenceResult = null;
             if (request.isFencing()) {
                 LOG.warn("Ledger: {}  fenced by: {}", request.getLedgerId(), channel.remoteAddress());
 
@@ -146,11 +142,11 @@
         sendResponse(data, retCode, startTimeNanos);
     }
 
-    private void handleReadResultForFenceRead(ListenableFuture<Boolean> fenceResult,
+    private void handleReadResultForFenceRead(CompletableFuture<Boolean> fenceResult,
                                               ByteBuf data,
                                               long startTimeNanos) {
         if (null != fenceThreadPool) {
-            Futures.addCallback(fenceResult, new FutureCallback<Boolean>() {
+            fenceResult.whenCompleteAsync(new FutureEventListener<Boolean>() {
                 @Override
                 public void onSuccess(Boolean result) {
                     sendFenceResponse(result, data, startTimeNanos);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
index a8ecc11..a563cc0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
@@ -18,21 +18,17 @@
 package org.apache.bookkeeper.proto;
 
 import com.google.common.base.Stopwatch;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.SettableFuture;
 import com.google.protobuf.ByteString;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
 import io.netty.util.ReferenceCountUtil;
-
 import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadResponse;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.Request;
@@ -50,7 +46,7 @@
     protected Stopwatch lastPhaseStartTime;
     private final ExecutorService fenceThreadPool;
 
-    private SettableFuture<Boolean> fenceResult = null;
+    private CompletableFuture<Boolean> fenceResult = null;
 
     protected final ReadRequest readRequest;
     protected final long ledgerId;
@@ -113,7 +109,7 @@
         // reset last phase start time to measure fence result waiting time
         lastPhaseStartTime.reset().start();
         if (null != fenceThreadPool) {
-            Futures.addCallback(fenceResult, new FutureCallback<Boolean>() {
+            fenceResult.whenCompleteAsync(new FutureEventListener<Boolean>() {
                 @Override
                 public void onSuccess(Boolean result) {
                     sendFenceResponse(readResponseBuilder, entryBody, result, startTimeSw);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java
index 2231b95..3c75049 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java
@@ -22,7 +22,7 @@
 
 import java.net.UnknownHostException;
 import java.util.function.Supplier;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
 import org.apache.bookkeeper.discover.BookieServiceInfoUtils;
@@ -36,7 +36,7 @@
 
     public SimpleBookieServiceInfoProvider(ServerConfiguration serverConfiguration) {
         try {
-            this.bookieSocketAddress = Bookie.getBookieAddress(serverConfiguration);
+            this.bookieSocketAddress = BookieImpl.getBookieAddress(serverConfiguration);
         } catch (UnknownHostException err) {
             throw new RuntimeException(err);
         }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
index 1ccb322..f475de0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
@@ -22,17 +22,14 @@
 
 import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE;
 import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_WORKER_SCOPE;
-
 import com.google.common.annotations.VisibleForTesting;
-
 import java.io.File;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.MalformedURLException;
 import java.util.concurrent.ExecutionException;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieCriticalThread;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.ExitCode;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -100,7 +97,7 @@
         });
 
         auditorElector = new AuditorElector(
-            Bookie.getBookieId(conf).toString(),
+            BookieImpl.getBookieId(conf).toString(),
             conf,
             bkc,
             statsLogger.scope(AUDITOR_SCOPE),
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 2d59661..3109bce 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
@@ -19,14 +19,13 @@
 package org.apache.bookkeeper.server.http.service;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-
 import com.google.common.collect.Lists;
 import java.io.File;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.http.HttpServer;
 import org.apache.bookkeeper.http.service.HttpEndpointService;
@@ -64,13 +63,13 @@
         HttpServiceResponse response = new HttpServiceResponse();
 
         if (HttpServer.Method.PUT == request.getMethod()) {
-            File[] ledgerDirectories = Bookie.getCurrentDirectories(conf.getLedgerDirs());
-            File[] journalDirectories = Bookie.getCurrentDirectories(conf.getJournalDirs());
+            File[] ledgerDirectories = BookieImpl.getCurrentDirectories(conf.getLedgerDirs());
+            File[] journalDirectories = BookieImpl.getCurrentDirectories(conf.getJournalDirs());
             File[] indexDirectories;
             if (null == conf.getIndexDirs()) {
                 indexDirectories = ledgerDirectories;
             } else {
-                indexDirectories = Bookie.getCurrentDirectories(conf.getIndexDirs());
+                indexDirectories = BookieImpl.getCurrentDirectories(conf.getIndexDirs());
             }
 
             List<File> allLedgerDirs = Lists.newArrayList();
@@ -83,7 +82,7 @@
                 URI.create(conf.getMetadataServiceUri())
             )) {
                 driver.initialize(conf, () -> { }, NullStatsLogger.INSTANCE);
-                Bookie.checkEnvironmentWithStorageExpansion(conf, driver,
+                BookieImpl.checkEnvironmentWithStorageExpansion(conf, driver,
                   Lists.newArrayList(journalDirectories), allLedgerDirs);
             } catch (BookieException e) {
                 LOG.error("Exception occurred while updating cookie for storage expansion", e);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java
index dc48b6c..602caf0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java
@@ -22,7 +22,7 @@
 import com.google.common.util.concurrent.UncheckedExecutionException;
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerCache;
 import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
@@ -79,10 +79,10 @@
         ServerConfiguration bkConf = new ServerConfiguration(conf);
 
         InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage();
-        Bookie.mountLedgerStorageOffline(bkConf, interleavedStorage);
+        BookieImpl.mountLedgerStorageOffline(bkConf, interleavedStorage);
 
         DbLedgerStorage dbStorage = new DbLedgerStorage();
-        Bookie.mountLedgerStorageOffline(bkConf, dbStorage);
+        BookieImpl.mountLedgerStorageOffline(bkConf, dbStorage);
 
         int convertedLedgers = 0;
         for (long ledgerId : interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java
index adcf2fd..8570452 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java
@@ -24,7 +24,7 @@
 import java.util.concurrent.TimeUnit;
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieShell;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -144,9 +144,9 @@
         final BookKeeperAdmin admin = new BookKeeperAdmin(bk);
         final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin);
         final ServerConfiguration serverConfiguration = new ServerConfiguration(conf);
-        final BookieId newBookieId = Bookie.getBookieId(serverConfiguration);
+        final BookieId newBookieId = BookieImpl.getBookieId(serverConfiguration);
         serverConfiguration.setUseHostNameAsBookieID(!flags.hostname);
-        final BookieId oldBookieId = Bookie.getBookieId(serverConfiguration);
+        final BookieId oldBookieId = BookieImpl.getBookieId(serverConfiguration);
 
         BookieShell.UpdateLedgerNotifier progressable = new BookieShell.UpdateLedgerNotifier() {
             long lastReport = System.nanoTime();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java
index f213c17..ecef51e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java
@@ -25,7 +25,7 @@
 import java.util.concurrent.ExecutionException;
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.Cookie;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
@@ -84,7 +84,7 @@
     public boolean apply(ServerConfiguration conf, Flags cmdFlags) {
 
         ServerConfiguration bfconf = new ServerConfiguration(conf);
-        boolean result = Bookie.format(bfconf, cmdFlags.nonInteractive, cmdFlags.force);
+        boolean result = BookieImpl.format(bfconf, cmdFlags.nonInteractive, cmdFlags.force);
 
         // delete cookie
         if (cmdFlags.deleteCookie) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java
index 834ee6d..46ed287 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java
@@ -20,7 +20,7 @@
 
 import io.netty.buffer.ByteBuf;
 import java.util.Formatter;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.util.EntryFormatter;
 import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.slf4j.Logger;
@@ -53,7 +53,7 @@
         LOG.info(
             "--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId + ", ByteOffset=" + pos
                 + ", EntrySize=" + entrySize + " ---------");
-        if (entryId == Bookie.METAENTRY_ID_LEDGER_KEY) {
+        if (entryId == BookieImpl.METAENTRY_ID_LEDGER_KEY) {
             int masterKeyLen = recBuff.readInt();
             byte[] masterKey = new byte[masterKeyLen];
             recBuff.readBytes(masterKey);
@@ -62,7 +62,7 @@
             LOG.info("");
             return;
         }
-        if (entryId == Bookie.METAENTRY_ID_FENCE_KEY) {
+        if (entryId == BookieImpl.METAENTRY_ID_FENCE_KEY) {
             LOG.info("Type:           META");
             LOG.info("Fenced");
             LOG.info("");
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java
index d158c5a..d9cf8d7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java
@@ -25,7 +25,7 @@
 
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerCache;
 import org.apache.bookkeeper.bookie.LedgerEntryPage;
@@ -100,7 +100,7 @@
             ServerConfiguration tConf = new ServerConfiguration(conf);
             InterleavedLedgerStorage interleavedLedgerStorage = new InterleavedLedgerStorage();
             try {
-                Bookie.mountLedgerStorageOffline(tConf, interleavedLedgerStorage);
+                BookieImpl.mountLedgerStorageOffline(tConf, interleavedLedgerStorage);
             } catch (IOException e) {
                 throw new UncheckedExecutionException(e.getMessage(), e);
             }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java
index 60a94bc..181d1e6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java
@@ -21,7 +21,7 @@
 import com.google.common.util.concurrent.UncheckedExecutionException;
 import java.io.IOException;
 import java.util.List;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
@@ -60,7 +60,7 @@
     private boolean check(ServerConfiguration conf) throws IOException {
         LOG.info("=== Performing local consistency check ===");
         ServerConfiguration serverConfiguration = new ServerConfiguration(conf);
-        LedgerStorage ledgerStorage = Bookie.mountLedgerStorageOffline(serverConfiguration, null);
+        LedgerStorage ledgerStorage = BookieImpl.mountLedgerStorageOffline(serverConfiguration, null);
         List<LedgerStorage.DetectedInconsistency> errors = ledgerStorage.localConsistencyCheck(
             java.util.Optional.empty());
         if (errors.size() > 0) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java
index 3466efe..90da459 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java
@@ -19,14 +19,13 @@
 package org.apache.bookkeeper.tools.cli.commands.bookies;
 
 import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager;
-
 import com.beust.jcommander.Parameter;
 import com.google.common.util.concurrent.UncheckedExecutionException;
 import java.io.IOException;
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.Cookie;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
@@ -92,7 +91,7 @@
         try {
             final String remoteBookieidToDecommission = flags.remoteBookieIdToDecommission;
             final BookieId bookieAddressToDecommission = (StringUtils.isBlank(remoteBookieidToDecommission)
-                                                                  ? Bookie.getBookieId(conf)
+                                                                  ? BookieImpl.getBookieId(conf)
                                                                   : BookieId.parse(remoteBookieidToDecommission));
             admin.decommissionBookie(bookieAddressToDecommission);
             LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated",
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 7429afc..ca039a4 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
@@ -20,7 +20,6 @@
 
 import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver;
 import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager;
-
 import com.beust.jcommander.Parameter;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.UncheckedExecutionException;
@@ -31,8 +30,8 @@
 import java.util.List;
 import lombok.Setter;
 import lombok.experimental.Accessors;
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.Cookie;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieId;
@@ -110,12 +109,12 @@
     }
 
     private void initDirectory(ServerConfiguration bkConf) {
-        this.journalDirectories = Bookie.getCurrentDirectories(bkConf.getJournalDirs());
-        this.ledgerDirectories = Bookie.getCurrentDirectories(bkConf.getLedgerDirs());
+        this.journalDirectories = BookieImpl.getCurrentDirectories(bkConf.getJournalDirs());
+        this.ledgerDirectories = BookieImpl.getCurrentDirectories(bkConf.getLedgerDirs());
         if (null == bkConf.getIndexDirs()) {
             this.indexDirectories = this.ledgerDirectories;
         } else {
-            this.indexDirectories = Bookie.getCurrentDirectories(bkConf.getIndexDirs());
+            this.indexDirectories = BookieImpl.getCurrentDirectories(bkConf.getIndexDirs());
         }
     }
 
@@ -149,7 +148,7 @@
         return runFunctionWithRegistrationManager(bkConf, rm -> {
             try {
                 ServerConfiguration conf = new ServerConfiguration(bkConf);
-                String newBookieId = Bookie.getBookieId(conf).toString();
+                String newBookieId = BookieImpl.getBookieId(conf).toString();
                 // read oldcookie
                 Versioned<Cookie> oldCookie = null;
                 try {
@@ -236,8 +235,8 @@
             }
 
             try {
-                Bookie.checkEnvironmentWithStorageExpansion(bkConf, driver, Arrays.asList(journalDirectories),
-                                                            allLedgerDirs);
+                BookieImpl.checkEnvironmentWithStorageExpansion(bkConf, driver, Arrays.asList(journalDirectories),
+                                                                allLedgerDirs);
                 return true;
             } catch (BookieException e) {
                 LOG.error("Exception while updating cookie for storage expansion", e);
@@ -247,7 +246,7 @@
     }
 
     private boolean listOrDeleteCookies(ServerConfiguration bkConf, boolean delete, boolean force) throws Exception {
-        BookieId bookieAddress = Bookie.getBookieId(bkConf);
+        BookieId bookieAddress = BookieImpl.getBookieId(bkConf);
         File[] journalDirs = bkConf.getJournalDirs();
         File[] ledgerDirs = bkConf.getLedgerDirs();
         File[] indexDirs = bkConf.getIndexDirs();
@@ -256,7 +255,7 @@
             allDirs = ArrayUtils.addAll(allDirs, indexDirs);
         }
 
-        File[] allCurDirs = Bookie.getCurrentDirectories(allDirs);
+        File[] allCurDirs = BookieImpl.getCurrentDirectories(allDirs);
         List<File> allVersionFiles = new LinkedList<File>();
         File versionFile;
         for (File curDir : allCurDirs) {
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 c264e6b..0e87e01 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
@@ -36,7 +36,7 @@
 
 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.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
@@ -254,7 +254,7 @@
             bsConfs[i].setLedgerDirNames(ledgerDirs);
 
             // write config into file before start so we can know what's wrong if start failed
-            String fileName = Bookie.getBookieId(bsConfs[i]).toString() + ".conf";
+            String fileName = BookieImpl.getBookieId(bsConfs[i]).toString() + ".conf";
             serializeLocalBookieConfig(bsConfs[i], fileName);
 
             // Mimic BookKeeper Main
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 2b02883..70619fc 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
@@ -72,10 +72,10 @@
         assertEquals("10.0.0.1", conf.getAdvertisedAddress());
 
         BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort);
-        assertEquals(bkAddress, Bookie.getBookieAddress(conf));
-        assertEquals(bkAddress.toBookieId(), Bookie.getBookieId(conf));
+        assertEquals(bkAddress, BookieImpl.getBookieAddress(conf));
+        assertEquals(bkAddress.toBookieId(), BookieImpl.getBookieId(conf));
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         b.start();
 
         BookKeeperAdmin bka = new BookKeeperAdmin(baseClientConf);
@@ -102,8 +102,8 @@
         assertEquals("10.0.0.1", conf.getAdvertisedAddress());
 
         BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort);
-        assertEquals(bkAddress, Bookie.getBookieAddress(conf));
-        assertEquals(bkAddress.toBookieId(), Bookie.getBookieId(conf));
+        assertEquals(bkAddress, BookieImpl.getBookieAddress(conf));
+        assertEquals(bkAddress.toBookieId(), BookieImpl.getBookieId(conf));
     }
 
     /**
@@ -124,10 +124,10 @@
         assertEquals(uuid, conf.getBookieId());
 
         BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort);
-        assertEquals(bkAddress, Bookie.getBookieAddress(conf));
-        assertEquals(uuid, Bookie.getBookieId(conf).getId());
+        assertEquals(bkAddress, BookieImpl.getBookieAddress(conf));
+        assertEquals(uuid, BookieImpl.getBookieId(conf).getId());
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         b.start();
 
         BookKeeperAdmin bka = new BookKeeperAdmin(baseClientConf);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java
index 4c8ad46..a182e68 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java
@@ -31,7 +31,7 @@
     /**
      * Force a bookie to flush its ledger storage.
      */
-    public static void forceFlush(Bookie b) throws IOException {
+    public static void forceFlush(BookieImpl b) throws IOException {
         CheckpointSourceList source = new CheckpointSourceList(b.journals);
         Checkpoint cp = source.newCheckpoint();
         b.ledgerStorage.flush();
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 e633f8b..7f4257b 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
@@ -198,7 +198,7 @@
         List<LastLogMark> lastLogMarkList = new ArrayList<>(journalDirs.length);
 
         for (int i = 0; i < journalDirs.length; i++) {
-            Journal journal = server.getBookie().journals.get(i);
+            Journal journal = ((BookieImpl) server.getBookie()).journals.get(i);
             // LastLogMark should be (0, 0) at the bookie clean start
             journal.getLastLogMark().readLog();
             lastLogMarkList.add(journal.getLastLogMark().markLog());
@@ -223,7 +223,7 @@
         }
 
         for (int i = 0; i < journalDirs.length; i++) {
-            Journal journal = server.getBookie().journals.get(i);
+            Journal journal = ((BookieImpl) server.getBookie()).journals.get(i);
             // In-memory LastLogMark should be updated with every write to journal
             assertTrue(journal.getLastLogMark().getCurMark().compare(lastLogMarkList.get(i).getCurMark()) > 0);
             lastLogMarkList.set(i, journal.getLastLogMark().markLog());
@@ -239,7 +239,7 @@
         server = new BookieServer(conf);
 
         for (int i = 0; i < journalDirs.length; i++) {
-            Journal journal = server.getBookie().journals.get(i);
+            Journal journal = ((BookieImpl) server.getBookie()).journals.get(i);
             // LastLogMark should be (0, 0) before bookie restart since bookie crashed before persisting lastMark
             assertEquals(0, journal.getLastLogMark().getCurMark().compare(new LogMark(0, 0)));
         }
@@ -255,7 +255,7 @@
             server.start();
 
             for (int j = 0; j < journalDirs.length; j++) {
-                Journal journal = server.getBookie().journals.get(j);
+                Journal journal = ((BookieImpl) server.getBookie()).journals.get(j);
                 assertTrue(journal.getLastLogMark().getCurMark().compare(lastLogMarkList.get(j).getCurMark()) > 0);
                 lastLogMarkList.set(j, journal.getLastLogMark().markLog());
             }
@@ -307,7 +307,7 @@
                      Supplier<BookieServiceInfo> bookieServiceInfoProvider)
                     throws IOException, KeeperException, InterruptedException,
                     BookieException {
-                Bookie bookie = new Bookie(conf);
+                Bookie bookie = new BookieImpl(conf);
                 MetadataBookieDriver driver = Whitebox.getInternalState(bookie, "metadataDriver");
                 ((ZKMetadataBookieDriver) driver).setRegManager(rm);
                 return bookie;
@@ -326,7 +326,7 @@
         conf.setMetadataServiceUri(metadataServiceUri)
             .setListeningInterface(null);
 
-        BookieId bookieId = Bookie.getBookieId(conf);
+        BookieId bookieId = BookieImpl.getBookieId(conf);
 
         driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
         try (StateManager manager = new BookieStateManager(conf, driver)) {
@@ -354,7 +354,7 @@
         conf.setMetadataServiceUri(metadataServiceUri)
             .setListeningInterface(null);
 
-        String bookieId = Bookie.getBookieAddress(conf).toString();
+        String bookieId = BookieImpl.getBookieAddress(conf).toString();
         final String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf)
             + "/" + AVAILABLE_NODE + "/" + bookieId;
 
@@ -727,16 +727,16 @@
     @Test
     public void testBookieStartException() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         /*
          * add few entries to journal file.
          */
         int numOfEntries = 100;
-        writeV5Journal(Bookie.getCurrentDirectory(journalDir), numOfEntries,
+        writeV5Journal(BookieImpl.getCurrentDirectory(journalDir), numOfEntries,
                 "testV5Journal".getBytes());
 
         /*
@@ -764,7 +764,7 @@
         Versioned<byte[]> newCookie = new Versioned<>(
                 cookie.toString().getBytes(UTF_8), Version.NEW
         );
-        driver.getRegistrationManager().writeCookie(Bookie.getBookieId(conf), newCookie);
+        driver.getRegistrationManager().writeCookie(BookieImpl.getBookieId(conf), newCookie);
 
         /*
          * Create LifecycleComponent for BookieServer and start it.
@@ -800,12 +800,12 @@
     @Test
     public void testNegativeLengthEntryBookieShutdown() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeV5Journal(Bookie.getCurrentDirectory(journalDir), 5,
+        writeV5Journal(BookieImpl.getCurrentDirectory(journalDir), 5,
                 "testV5Journal".getBytes(), true);
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -815,7 +815,7 @@
 
         Bookie b = null;
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             b.start();
             assertFalse("Bookie should shutdown normally after catching IOException"
                     + " due to corrupt entry with negative length", b.isRunning());
@@ -891,7 +891,7 @@
         conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()).setZkTimeout(5000);
 
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Should throw ConnectionLossException as ZKServer is not running!");
         } catch (BookieException.MetadataStoreException e) {
             // expected behaviour
@@ -913,7 +913,7 @@
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri(zkRoot))
             .setZkTimeout(5000);
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Should throw NoNodeException");
         } catch (Exception e) {
             // shouldn't be able to start
@@ -922,7 +922,7 @@
         adminConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri(zkRoot));
         BookKeeperAdmin.format(adminConf, false, false);
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         b.shutdown();
     }
 
@@ -950,7 +950,7 @@
         conf.setMinUsableSizeForEntryLogCreation(Long.MAX_VALUE)
             .setReadOnlyModeEnabled(false);
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("NoWritableLedgerDirException expected");
         } catch (NoWritableLedgerDirException e) {
             // expected
@@ -959,7 +959,7 @@
         conf.setMinUsableSizeForEntryLogCreation(Long.MIN_VALUE)
             .setReadOnlyModeEnabled(false);
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("NoWritableLedgerDirException expected");
         } catch (NoWritableLedgerDirException e) {
             // expected
@@ -971,7 +971,7 @@
         try {
             // bookie is okay to start up when readonly mode is enabled because entry log file creation
             // is deferred.
-            bookie = new Bookie(conf);
+            bookie = new BookieImpl(conf);
         } catch (NoWritableLedgerDirException e) {
             fail("NoWritableLedgerDirException unexpected");
         } finally {
@@ -1003,7 +1003,7 @@
         // while replaying the journal)
         conf.setReadOnlyModeEnabled(true)
             .setIsForceGCAllowWhenNoSpace(true);
-        final Bookie bk = new Bookie(conf);
+        final Bookie bk = new BookieImpl(conf);
         bk.start();
         Thread.sleep((conf.getDiskCheckInterval() * 2) + 100);
 
@@ -1028,7 +1028,7 @@
         }
     }
 
-    class MockBookieWithNoopShutdown extends Bookie {
+    class MockBookieWithNoopShutdown extends BookieImpl {
         public MockBookieWithNoopShutdown(ServerConfiguration conf, StatsLogger statsLogger)
                 throws IOException, KeeperException, InterruptedException, BookieException {
             super(conf, statsLogger, UnpooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO);
@@ -1268,7 +1268,7 @@
         Bookie bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
         // transition to readonly mode, bookie status should be persisted in ledger disks
-        bookie.getStateManager().doTransitionToReadOnlyMode();
+        bookie.getStateManager().transitionToReadOnlyMode().get();
         assertTrue(bookie.isReadOnly());
 
         // restart bookie should start in read only mode
@@ -1278,7 +1278,7 @@
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
         // transition to writable mode
-        bookie.getStateManager().doTransitionToWritableMode();
+        bookie.getStateManager().transitionToWritableMode().get();
         // restart bookie should start in writable mode
         bookieServer.shutdown();
         bookieServer = new BookieServer(conf);
@@ -1305,8 +1305,8 @@
         bookieServer.start();
         Bookie bookie = bookieServer.getBookie();
         // persist bookie status
-        bookie.getStateManager().doTransitionToReadOnlyMode();
-        bookie.getStateManager().doTransitionToWritableMode();
+        bookie.getStateManager().transitionToReadOnlyMode().get();
+        bookie.getStateManager().transitionToWritableMode().get();
         assertFalse(bookie.isReadOnly());
         bookieServer.shutdown();
         // start read only bookie
@@ -1318,7 +1318,7 @@
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
         // transition to writable should fail
-        bookie.getStateManager().doTransitionToWritableMode();
+        bookie.getStateManager().transitionToWritableMode().get();
         assertTrue(bookie.isReadOnly());
         bookieServer.shutdown();
     }
@@ -1345,12 +1345,12 @@
         BookieServer bookieServer = new BookieServer(conf);
         bookieServer.start();
         // transition in to read only and persist the status on disk
-        Bookie bookie = bookieServer.getBookie();
+        Bookie bookie = (BookieImpl) bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
-        bookie.getStateManager().doTransitionToReadOnlyMode();
+        bookie.getStateManager().transitionToReadOnlyMode().get();
         assertTrue(bookie.isReadOnly());
         // corrupt status file
-        List<File> ledgerDirs = bookie.getLedgerDirsManager().getAllLedgerDirs();
+        List<File> ledgerDirs = ((BookieImpl) bookie).getLedgerDirsManager().getAllLedgerDirs();
         corruptFile(new File(ledgerDirs.get(0), BOOKIE_STATUS_FILENAME));
         corruptFile(new File(ledgerDirs.get(1), BOOKIE_STATUS_FILENAME));
         // restart the bookie should be in read only mode
@@ -1384,15 +1384,15 @@
         BookieServer bookieServer = new BookieServer(conf);
         bookieServer.start();
         // transition in to read only and persist the status on disk
-        Bookie bookie = bookieServer.getBookie();
+        Bookie bookie = (BookieImpl) bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
-        bookie.getStateManager().doTransitionToReadOnlyMode();
+        bookie.getStateManager().transitionToReadOnlyMode().get();
         assertTrue(bookie.isReadOnly());
         // Manually update a status file, so it becomes the latest
         Thread.sleep(1);
         BookieStatus status = new BookieStatus();
         List<File> dirs = new ArrayList<File>();
-        dirs.add(bookie.getLedgerDirsManager().getAllLedgerDirs().get(0));
+        dirs.add(((BookieImpl) bookie).getLedgerDirsManager().getAllLedgerDirs().get(0));
         status.writeToDirectories(dirs);
         // restart the bookie should start in writable state
         bookieServer.shutdown();
@@ -1556,9 +1556,9 @@
         conf.setJournalDirsName(journalDirs);
         conf.setLedgerDirNames(new String[] { tmpLedgerDir.getPath() });
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
 
-        final BookieId bookieAddress = Bookie.getBookieId(conf);
+        final BookieId bookieAddress = BookieImpl.getBookieId(conf);
 
         // Read cookie from registation manager
         Versioned<Cookie> rmCookie = Cookie.readFromRegistrationManager(rm, bookieAddress);
@@ -1570,7 +1570,7 @@
         rmCookie.getValue().deleteFromRegistrationManager(rm, conf, rmCookie.getVersion());
 
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             Assert.fail("Bookie should not have come up. Cookie no present in metadata store.");
         } catch (Exception e) {
             LOG.info("As expected Bookie fails to come up without a cookie in metadata store.");
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalBypassTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalBypassTest.java
index 2d14a0f..fe20322 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalBypassTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalBypassTest.java
@@ -56,10 +56,12 @@
     public void testJournalBypass() throws Exception {
         ClientConfiguration conf = new ClientConfiguration(baseClientConf);
 
-        Journal journal0 = bs.get(0).getBookie().journals.get(0);
+        BookieImpl bookieImpl = (BookieImpl) bs.get(0).getBookie();
+        Journal journal0 = bookieImpl.journals.get(0);
         LedgerStorage ls0 = bs.get(0).getBookie().getLedgerStorage();
 
-        Journal journal1 = bs.get(1).getBookie().journals.get(0);
+        bookieImpl = (BookieImpl) bs.get(1).getBookie();
+        Journal journal1 = bookieImpl.journals.get(0);
         LedgerStorage ls1 = bs.get(1).getBookie().getLedgerStorage();
 
         ls0.flush();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java
index ee55fb2..984b085 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java
@@ -75,7 +75,7 @@
     @Test
     public void testAckAfterSync() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
             .setJournalDirName(journalDir.getPath())
@@ -138,7 +138,7 @@
     @Test
     public void testAckBeforeSync() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -189,7 +189,7 @@
     @Test
     public void testAckBeforeSyncWithJournalBufferedEntriesThreshold() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         final int journalBufferedEntriesThreshold = 10;
         // sending a burst of entries, more than journalBufferedEntriesThreshold
@@ -255,7 +255,7 @@
     @Test
     public void testInterleavedRequests() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -322,7 +322,7 @@
     @Test
     public void testForceLedger() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalMaxMemoryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalMaxMemoryTest.java
index 02299ad..804a7f3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalMaxMemoryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalMaxMemoryTest.java
@@ -58,7 +58,7 @@
     @Test
     public void testAckAfterSyncPageCacheFlush() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(journalDir.getPath())
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
index 538139d..05488b7 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
@@ -92,7 +92,7 @@
     @Test
     public void testAckAfterSyncPageCacheFlush() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(journalDir.getPath())
@@ -159,7 +159,7 @@
     @Test
     public void testAckBeforeSyncPageCacheFlush() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(journalDir.getPath())
@@ -222,7 +222,7 @@
     @Test
     public void testAckBeforeUnSyncPageCacheFlush() throws Exception {
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setJournalDirName(journalDir.getPath())
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 88f2db7..def20f6 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
@@ -121,7 +121,7 @@
     private static ByteBuf generateFenceEntry(long ledgerId) {
         ByteBuf bb = Unpooled.buffer();
         bb.writeLong(ledgerId);
-        bb.writeLong(Bookie.METAENTRY_ID_FENCE_KEY);
+        bb.writeLong(BookieImpl.METAENTRY_ID_FENCE_KEY);
         return bb;
     }
 
@@ -131,7 +131,7 @@
     private static ByteBuf generateMetaEntry(long ledgerId, byte[] masterKey) {
         ByteBuf bb = Unpooled.buffer();
         bb.writeLong(ledgerId);
-        bb.writeLong(Bookie.METAENTRY_ID_LEDGER_KEY);
+        bb.writeLong(BookieImpl.METAENTRY_ID_LEDGER_KEY);
         bb.writeInt(masterKey.length);
         bb.writeBytes(masterKey);
         return bb;
@@ -355,13 +355,13 @@
     @Test
     public void testPreV2Journal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 100);
-        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir), 1, "testPasswd".getBytes());
+        writePreV2Journal(BookieImpl.getCurrentDirectory(journalDir), 100);
+        writeIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir), 1, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -384,19 +384,19 @@
     @Test
     public void testV4Journal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeV4Journal(Bookie.getCurrentDirectory(journalDir), 100, "testPasswd".getBytes());
+        writeV4Journal(BookieImpl.getCurrentDirectory(journalDir), 100, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
             .setLedgerDirNames(new String[] { ledgerDir.getPath() })
             .setMetadataServiceUri(null);
 
-        Bookie b = createBookieAndReadJournal(conf);
+        BookieImpl b = createBookieAndReadJournal(conf);
 
         b.readEntry(1, 100);
         try {
@@ -413,12 +413,12 @@
     @Test
     public void testV5Journal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeV5Journal(Bookie.getCurrentDirectory(journalDir), 2 * JournalChannel.SECTOR_SIZE,
+        writeV5Journal(BookieImpl.getCurrentDirectory(journalDir), 2 * JournalChannel.SECTOR_SIZE,
                 "testV5Journal".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -426,7 +426,7 @@
             .setLedgerDirNames(new String[] { ledgerDir.getPath() })
             .setMetadataServiceUri(null);
 
-        Bookie b = createBookieAndReadJournal(conf);
+        BookieImpl b = createBookieAndReadJournal(conf);
 
         for (int i = 1; i <= 2 * JournalChannel.SECTOR_SIZE; i++) {
             b.readEntry(1, i);
@@ -450,12 +450,12 @@
     @Test
     public void testAllJunkJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeJunkJournal(Bookie.getCurrentDirectory(journalDir));
+        writeJunkJournal(BookieImpl.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -464,7 +464,7 @@
 
         Bookie b = null;
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start without admin");
         } catch (Throwable t) {
             // correct behaviour
@@ -485,19 +485,19 @@
     @Test
     public void testEmptyJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
+        writePreV2Journal(BookieImpl.getCurrentDirectory(journalDir), 0);
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
             .setLedgerDirNames(new String[] { ledgerDir.getPath() })
             .setMetadataServiceUri(null);
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
     }
 
     /**
@@ -507,19 +507,19 @@
     @Test
     public void testHeaderOnlyJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
+        writeV2Journal(BookieImpl.getCurrentDirectory(journalDir), 0);
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
             .setLedgerDirNames(new String[] { ledgerDir.getPath() })
             .setMetadataServiceUri(null);
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
     }
 
     /**
@@ -529,12 +529,12 @@
     @Test
     public void testJunkEndedJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        JournalChannel jc = writeV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
+        JournalChannel jc = writeV2Journal(BookieImpl.getCurrentDirectory(journalDir), 0);
         jc.getBufferedChannel().write(Unpooled.wrappedBuffer("JunkJunkJunk".getBytes()));
         jc.getBufferedChannel().flushAndForceWrite(false);
 
@@ -547,7 +547,7 @@
 
         Bookie b = null;
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
         } catch (Throwable t) {
             // correct behaviour
         }
@@ -564,20 +564,20 @@
     @Test
     public void testTruncatedInLenJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         JournalChannel jc = writeV2Journal(
-                Bookie.getCurrentDirectory(journalDir), 100);
+                BookieImpl.getCurrentDirectory(journalDir), 100);
         ByteBuffer zeros = ByteBuffer.allocate(2048);
 
         jc.fc.position(jc.getBufferedChannel().position() - 0x429);
         jc.fc.write(zeros);
         jc.fc.force(false);
 
-        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+        writeIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir),
                                 1, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -607,20 +607,20 @@
     @Test
     public void testTruncatedInEntryJournal() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         JournalChannel jc = writeV2Journal(
-                Bookie.getCurrentDirectory(journalDir), 100);
+                BookieImpl.getCurrentDirectory(journalDir), 100);
         ByteBuffer zeros = ByteBuffer.allocate(2048);
 
         jc.fc.position(jc.getBufferedChannel().position() - 0x300);
         jc.fc.write(zeros);
         jc.fc.force(false);
 
-        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+        writeIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir),
                                 1, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -654,8 +654,8 @@
         }
     }
 
-    private Bookie createBookieAndReadJournal(ServerConfiguration conf) throws Exception {
-        Bookie b = new Bookie(conf);
+    private BookieImpl createBookieAndReadJournal(ServerConfiguration conf) throws Exception {
+        BookieImpl b = new BookieImpl(conf);
         for (Journal journal : b.journals) {
             LastLogMark lastLogMark = journal.getLastLogMark().markLog();
             b.readJournal();
@@ -671,17 +671,17 @@
     @Test
     public void testSortedLedgerStorageReplayWithSmallMaxArenaSize() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         JournalChannel jc = writeV2Journal(
-                Bookie.getCurrentDirectory(journalDir), 100);
+                BookieImpl.getCurrentDirectory(journalDir), 100);
 
         jc.fc.force(false);
 
-        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+        writeIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir),
                 1, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -690,7 +690,7 @@
         conf.setJournalDirName(journalDir.getPath())
                 .setLedgerDirNames(new String[] { ledgerDir.getPath() });
 
-        Bookie b = new Bookie(conf);
+        BookieImpl b = new BookieImpl(conf);
         b.readJournal();
         b.ledgerStorage.flush();
         b.readEntry(1, 80);
@@ -719,13 +719,13 @@
     private void testPartialFileInfoPreV3Journal(boolean truncateMasterKey)
         throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 100);
-        writePartialIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+        writePreV2Journal(BookieImpl.getCurrentDirectory(journalDir), 100);
+        writePartialIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir),
                                        1, "testPasswd".getBytes(), truncateMasterKey);
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -735,13 +735,13 @@
 
         if (truncateMasterKey) {
             try {
-                Bookie b = new Bookie(conf);
+                BookieImpl b = new BookieImpl(conf);
                 b.readJournal();
                 fail("Should not reach here!");
             } catch (IOException ie) {
             }
         } else {
-            Bookie b = new Bookie(conf);
+            BookieImpl b = new BookieImpl(conf);
             b.readJournal();
             b.readEntry(1, 100);
             try {
@@ -775,15 +775,15 @@
     private void testPartialFileInfoPostV3Journal(boolean truncateMasterKey)
         throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         byte[] masterKey = "testPasswd".getBytes();
 
-        writeV3Journal(Bookie.getCurrentDirectory(journalDir), 100, masterKey);
-        writePartialIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir), 1, masterKey,
+        writeV3Journal(BookieImpl.getCurrentDirectory(journalDir), 100, masterKey);
+        writePartialIndexFileForLedger(BookieImpl.getCurrentDirectory(ledgerDir), 1, masterKey,
                                        truncateMasterKey);
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -791,7 +791,7 @@
             .setLedgerDirNames(new String[] { ledgerDir.getPath() })
             .setMetadataServiceUri(null);
 
-        Bookie b = new Bookie(conf);
+        BookieImpl b = new BookieImpl(conf);
         b.readJournal();
         b.readEntry(1, 100);
         try {
@@ -808,12 +808,12 @@
     @Test
     public void testJournalScanIOException() throws Exception {
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
-        writeV4Journal(Bookie.getCurrentDirectory(journalDir), 100, "testPasswd".getBytes());
+        writeV4Journal(BookieImpl.getCurrentDirectory(journalDir), 100, "testPasswd".getBytes());
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
@@ -829,7 +829,7 @@
         PowerMockito.mockStatic(JournalChannel.class);
         PowerMockito.when(JournalChannel.openFileChannel(Mockito.any(RandomAccessFile.class))).thenReturn(fileChannel);
 
-        Bookie b = new Bookie(conf);
+        BookieImpl b = new BookieImpl(conf);
 
         for (Journal journal : b.journals) {
             List<Long> journalIds = journal.listJournalIds(journal.getJournalDirectory(), null);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShutdownTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShutdownTest.java
index 3fbb5c8..e5f3a59 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShutdownTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShutdownTest.java
@@ -122,7 +122,7 @@
         killBookie(0);
         final CountDownLatch latch = new CountDownLatch(1);
         final CountDownLatch shutdownComplete = new CountDownLatch(1);
-        Bookie bookie = new Bookie(conf) {
+        Bookie bookie = new BookieImpl(conf) {
             @Override
             public void run() {
                 try {
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 0bd2a97..6c4cdb3 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
@@ -157,12 +157,12 @@
         BookieServer server = startBookie(conf);
         bs.add(server);
         bsConfs.add(conf);
-        Bookie bookie = server.getBookie();
+        BookieImpl bookie = (BookieImpl) server.getBookie();
         // since we are going to set dependency injected dirsMonitor, so we need to shutdown
         // the dirsMonitor which was created as part of the initialization of Bookie
         bookie.dirsMonitor.shutdown();
 
-        LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
+        LedgerDirsManager ledgerDirsManager = ((BookieImpl) bookie).getLedgerDirsManager();
 
         // flag latches
         final CountDownLatch diskWritable = new CountDownLatch(1);
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 9420661..597ca77 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
@@ -57,14 +57,14 @@
  * Test the bookie journal.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({Bookie.class})
+@PrepareForTest({BookieImpl.class})
 @Slf4j
 public class BookieWriteToJournalTest {
 
     @Rule
     public TemporaryFolder tempDir = new TemporaryFolder();
 
-    class NoOpJournalReplayBookie extends Bookie {
+    class NoOpJournalReplayBookie extends BookieImpl {
 
         public NoOpJournalReplayBookie(ServerConfiguration conf)
                 throws IOException, InterruptedException, BookieException {
@@ -84,14 +84,15 @@
     public void testJournalLogAddEntryCalledCorrectly() throws Exception {
 
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
         File ledgerDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
                 .setLedgerDirNames(new String[]{ledgerDir.getPath()})
                 .setMetadataServiceUri(null);
-        BookieId bookieAddress = Bookie.getBookieId(conf);
+
+        BookieId bookieAddress = BookieImpl.getBookieId(conf);
         CountDownLatch journalJoinLatch = new CountDownLatch(1);
         Journal journal = mock(Journal.class);
         MutableBoolean effectiveAckBeforeSync = new MutableBoolean(false);
@@ -150,14 +151,14 @@
     public void testForceLedger() throws Exception {
 
         File journalDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
         File ledgerDir = tempDir.newFolder();
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setJournalDirName(journalDir.getPath())
                 .setLedgerDirNames(new String[]{ledgerDir.getPath()});
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         b.start();
 
         long ledgerId = 1;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CheckpointOnNewLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CheckpointOnNewLedgersTest.java
index e0a8289..e334280 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CheckpointOnNewLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CheckpointOnNewLedgersTest.java
@@ -51,15 +51,15 @@
     public final TemporaryFolder testDir = new TemporaryFolder();
 
     private ServerConfiguration conf;
-    private Bookie bookie;
+    private BookieImpl bookie;
     private CountDownLatch getLedgerDescCalledLatch;
     private CountDownLatch getLedgerDescWaitLatch;
 
     @Before
     public void setup() throws Exception {
         File bkDir = testDir.newFolder("dbLedgerStorageCheckpointTest");
-        File curDir = Bookie.getCurrentDirectory(bkDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(bkDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
         conf = TestBKConfiguration.newServerConfiguration();
@@ -69,7 +69,7 @@
         conf.setLedgerDirNames(new String[] { bkDir.toString() });
         conf.setEntryLogSizeLimit(10 * 1024);
 
-        bookie = spy(new Bookie(conf));
+        bookie = spy(new BookieImpl(conf));
         bookie.start();
 
         getLedgerDescCalledLatch = new CountDownLatch(1);
@@ -175,7 +175,7 @@
         t1.join();
 
         // construct a new bookie to simulate "bookie restart from crash"
-        Bookie newBookie = new Bookie(conf);
+        Bookie newBookie = new BookieImpl(conf);
         newBookie.start();
 
         for (int i = 0; i < numEntries; i++) {
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 e361421..5152040 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
@@ -145,7 +145,7 @@
     private GarbageCollectorThread getGCThread() {
         assertEquals(1, bs.size());
         BookieServer server = bs.get(0);
-        return ((InterleavedLedgerStorage) server.getBookie().ledgerStorage).gcThread;
+        return ((InterleavedLedgerStorage) server.getBookie().getLedgerStorage()).gcThread;
     }
 
     LedgerHandle[] prepareData(int numEntryLogs, boolean changeNum)
@@ -284,10 +284,10 @@
             }
         };
         for (File journalDir : conf.getJournalDirs()) {
-            Bookie.checkDirectoryStructure(journalDir);
+            BookieImpl.checkDirectoryStructure(journalDir);
         }
         for (File dir : dirManager.getAllLedgerDirs()) {
-            Bookie.checkDirectoryStructure(dir);
+            BookieImpl.checkDirectoryStructure(dir);
         }
         runFunctionWithLedgerManagerFactory(conf, lmf -> {
             try (LedgerManager lm = lmf.newLedgerManager()) {
@@ -562,7 +562,7 @@
         assertTrue(getGCThread().enableMinorCompaction);
 
         for (BookieServer bookieServer : bs) {
-            Bookie bookie = bookieServer.getBookie();
+            BookieImpl bookie = ((BookieImpl) bookieServer.getBookie());
             LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
             List<File> ledgerDirs = ledgerDirsManager.getAllLedgerDirs();
             // if all the discs are full then Major and Minor compaction would be disabled since
@@ -631,8 +631,8 @@
         assertTrue(getGCThread().enableMinorCompaction);
 
         for (BookieServer bookieServer : bs) {
-            Bookie bookie = bookieServer.getBookie();
-            bookie.ledgerStorage.flush();
+            BookieImpl bookie = ((BookieImpl) bookieServer.getBookie());
+            bookie.getLedgerStorage().flush();
             bookie.dirsMonitor.shutdown();
             LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
             List<File> ledgerDirs = ledgerDirsManager.getAllLedgerDirs();
@@ -658,7 +658,7 @@
         // allocating newlog
         // we get getWritableLedgerDirsForNewLog() of ledgerDirsManager instead of getWritableLedgerDirs()
         // entry logs ([0,1,2].log) should be compacted.
-        for (File ledgerDirectory : server.getBookie().getLedgerDirsManager().getAllLedgerDirs()) {
+        for (File ledgerDirectory : ((BookieImpl) server.getBookie()).getLedgerDirsManager().getAllLedgerDirs()) {
             assertFalse("Found entry log file ([0,1,2].log that should have not been compacted in ledgerDirectory: "
                     + ledgerDirectory, TestUtils.hasLogFiles(ledgerDirectory.getParentFile(), true, 0, 1, 2));
         }
@@ -669,7 +669,7 @@
 
         // for the sake of validity of test lets make sure that there is no writableLedgerDir in the bookies
         for (BookieServer bookieServer : bs) {
-            Bookie bookie = bookieServer.getBookie();
+            BookieImpl bookie = (BookieImpl) bookieServer.getBookie();
             LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
             try {
                 List<File> ledgerDirs = ledgerDirsManager.getWritableLedgerDirs();
@@ -837,7 +837,7 @@
          * there is only one bookie in the cluster so we should be able to read
          * entries from this bookie.
          */
-        ServerConfiguration bookieServerConfig = bs.get(0).getBookie().conf;
+        ServerConfiguration bookieServerConfig = ((BookieImpl) bs.get(0).getBookie()).conf;
         ServerConfiguration newBookieConf = new ServerConfiguration(bookieServerConfig);
         /*
          * by reusing bookieServerConfig and setting metadataServiceUri to null
@@ -846,7 +846,7 @@
          * purpose.
          */
         newBookieConf.setMetadataServiceUri(null);
-        Bookie newbookie = new Bookie(newBookieConf);
+        Bookie newbookie = new BookieImpl(newBookieConf);
 
         DigestManager digestManager = DigestManager.instantiate(ledgerId, passwdBytes,
                 BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT,
@@ -917,7 +917,7 @@
         restartBookies(baseConf);
 
         assertFalse("There shouldn't be any writable ledgerDir",
-                bs.get(0).getBookie().getLedgerDirsManager().hasWritableLedgerDirs());
+                    ((BookieImpl) bs.get(0).getBookie()).getLedgerDirsManager().hasWritableLedgerDirs());
 
         long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
         long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
@@ -1044,8 +1044,8 @@
         LedgerManager manager = getLedgerManager(ledgers);
 
         File tmpDir = createTempDir("bkTest", ".dir");
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] {tmpDir.toString()});
 
         conf.setEntryLogSizeLimit(EntryLogger.LOGFILE_HEADER_SIZE + 3 * (4 + ENTRY_SIZE));
@@ -1228,8 +1228,8 @@
         tearDown(); // I dont want the test infrastructure
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         File tmpDir = createTempDir("bkTest", ".dir");
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
 
         LedgerDirsManager dirs = new LedgerDirsManager(conf, conf.getLedgerDirs(),
@@ -1278,8 +1278,8 @@
         restartBookies(baseConf);
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         File tmpDir = createTempDir("bkTest", ".dir");
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
 
         LedgerDirsManager dirs = new LedgerDirsManager(conf, conf.getLedgerDirs(),
@@ -1383,10 +1383,10 @@
             }
         };
         for (File journalDir : conf.getJournalDirs()) {
-            Bookie.checkDirectoryStructure(journalDir);
+            BookieImpl.checkDirectoryStructure(journalDir);
         }
         for (File dir : dirManager.getAllLedgerDirs()) {
-            Bookie.checkDirectoryStructure(dir);
+            BookieImpl.checkDirectoryStructure(dir);
         }
         InterleavedLedgerStorage storage = new InterleavedLedgerStorage();
         TestStatsProvider stats = new TestStatsProvider();
@@ -1478,7 +1478,7 @@
         // restart bookies
         restartBookies(baseConf);
 
-        Bookie bookie = bs.get(0).getBookie();
+        BookieImpl bookie = ((BookieImpl) bs.get(0).getBookie());
         InterleavedLedgerStorage storage = (InterleavedLedgerStorage) bookie.ledgerStorage;
 
         // remove ledger2 and ledger3
@@ -1488,7 +1488,7 @@
         LOG.info("Finished deleting the ledgers contains most entries.");
 
         MockTransactionalEntryLogCompactor partialCompactionWorker = new MockTransactionalEntryLogCompactor(
-            ((InterleavedLedgerStorage) bookie.ledgerStorage).gcThread);
+                ((InterleavedLedgerStorage) bookie.getLedgerStorage()).gcThread);
 
         for (long logId = 0; logId < 3; logId++) {
             EntryLogMetadata meta = storage.entryLogger.getEntryLogMetadata(logId);
@@ -1548,7 +1548,7 @@
         LOG.info("Finished deleting the ledgers contains most entries.");
         Thread.sleep(baseConf.getMajorCompactionInterval() * 1000
             + baseConf.getGcWaitTime());
-        Bookie bookie = bs.get(0).getBookie();
+        BookieImpl bookie = (BookieImpl) bs.get(0).getBookie();
         InterleavedLedgerStorage storage = (InterleavedLedgerStorage) bookie.ledgerStorage;
 
         List<File> ledgerDirs = bookie.getLedgerDirsManager().getAllLedgerDirs();
@@ -1610,7 +1610,7 @@
     private Set<File> findCompactedEntryLogFiles() {
         Set<File> compactedLogFiles = new HashSet<>();
         for (File ledgerDirectory : tmpDirs) {
-            File[] files = Bookie.getCurrentDirectory(ledgerDirectory).listFiles(
+            File[] files = BookieImpl.getCurrentDirectory(ledgerDirectory).listFiles(
                 file -> file.getName().endsWith(COMPACTED_SUFFIX));
             if (files != null) {
                 Collections.addAll(compactedLogFiles, files);
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 e244852..e551495 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
@@ -109,7 +109,7 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            Bookie b = new Bookie(conf);
+            Bookie b = new BookieImpl(conf);
         } catch (Exception e) {
             fail("Should not reach here.");
         }
@@ -143,7 +143,7 @@
         c2.writeToDirectory(new File(ledgerDir, "current"));
 
         try {
-            Bookie b = new Bookie(conf2);
+            Bookie b = new BookieImpl(conf2);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -166,13 +166,13 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setLedgerDirNames(new String[] { ledgerDirs[0], ledgerDirs[1] });
         try {
-            Bookie b2 = new Bookie(conf);
+            Bookie b2 = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -180,14 +180,14 @@
 
         conf.setJournalDirName(newDirectory()).setLedgerDirNames(ledgerDirs);
         try {
-            Bookie b2 = new Bookie(conf);
+            Bookie b2 = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
         }
 
         conf.setJournalDirName(journalDir);
-        b = new Bookie(conf);
+        b = new BookieImpl(conf);
         b.start();
         b.shutdown();
     }
@@ -207,15 +207,15 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         File cookieFile =
-            new File(Bookie.getCurrentDirectory(new File(journalDir)), BookKeeperConstants.VERSION_FILENAME);
+            new File(BookieImpl.getCurrentDirectory(new File(journalDir)), BookKeeperConstants.VERSION_FILENAME);
         assertTrue(cookieFile.delete());
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -237,15 +237,15 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         File cookieFile =
-            new File(Bookie.getCurrentDirectory(new File(ledgerDirs[0])), BookKeeperConstants.VERSION_FILENAME);
+            new File(BookieImpl.getCurrentDirectory(new File(ledgerDirs[0])), BookKeeperConstants.VERSION_FILENAME);
         assertTrue(cookieFile.delete());
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -267,20 +267,20 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setLedgerDirNames(new String[] { ledgerDir0, newDirectory() });
         try {
-            Bookie b2 = new Bookie(conf);
+            Bookie b2 = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
         }
 
         conf.setLedgerDirNames(new String[] { ledgerDir0 });
-        b = new Bookie(conf);
+        b = new BookieImpl(conf);
         b.start();
         b.shutdown();
     }
@@ -302,7 +302,7 @@
             .setAllowStorageExpansion(true)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        BookieImpl b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
         b = null;
@@ -318,7 +318,7 @@
         conf.setIndexDirName(iPaths);
 
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
         } catch (BookieException.InvalidCookieException ice) {
             fail("Should have been able to start the bookie");
         }
@@ -350,7 +350,7 @@
         String[] lPaths2 = new String[] { lPaths[0], lPaths[1], newDirectory() };
         conf.setLedgerDirNames(lPaths2);
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Should not have been able to start the bookie");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -361,7 +361,7 @@
         lPaths2 = new String[] { lPaths[0], lPaths[1] };
         conf.setLedgerDirNames(lPaths2);
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Should not have been able to start the bookie");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -385,7 +385,7 @@
             .setAllowStorageExpansion(true)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
         b = null;
@@ -395,12 +395,12 @@
         conf.setLedgerDirNames(lPaths);
 
         // create a file to make the dir non-empty
-        File currentDir = Bookie.getCurrentDirectory(new File(lPaths[1]));
+        File currentDir = BookieImpl.getCurrentDirectory(new File(lPaths[1]));
         new File(currentDir, "foo").createNewFile();
         assertTrue(currentDir.list().length == 1);
 
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -411,12 +411,12 @@
         conf.setIndexDirName(iPaths);
 
         // create a dir to make it non-empty
-        currentDir = Bookie.getCurrentDirectory(new File(iPaths[1]));
+        currentDir = BookieImpl.getCurrentDirectory(new File(iPaths[1]));
         new File(currentDir, "bar").mkdirs();
         assertTrue(currentDir.list().length == 1);
 
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behavior
@@ -437,13 +437,13 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         FileUtils.deleteDirectory(new File(ledgerDir0));
         try {
-            Bookie b2 = new Bookie(conf);
+            Bookie b2 = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -461,13 +461,13 @@
             .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setBookiePort(3182);
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -487,7 +487,7 @@
             .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
@@ -497,7 +497,7 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -522,12 +522,12 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         // Bookie should start successfully for fresh env.
-        new Bookie(bookieConf);
+        new BookieImpl(bookieConf);
 
         // Format metadata one more time.
         BookKeeperAdmin.format(adminConf, false, true);
         try {
-            new Bookie(bookieConf);
+            new BookieImpl(bookieConf);
             fail("Bookie should not start with previous instance id.");
         } catch (BookieException.InvalidCookieException e) {
             assertTrue(
@@ -536,9 +536,9 @@
         }
 
         // Now format the Bookie and restart.
-        Bookie.format(bookieConf, false, true);
+        BookieImpl.format(bookieConf, false, true);
         // After bookie format bookie should be able to start again.
-        new Bookie(bookieConf);
+        new BookieImpl(bookieConf);
     }
 
     /**
@@ -558,7 +558,7 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            Bookie b = new Bookie(conf);
+            Bookie b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -583,7 +583,7 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
-            Bookie b = new Bookie(conf);
+            Bookie b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -605,13 +605,13 @@
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setUseHostNameAsBookieID(true);
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Should not start a bookie with hostname if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -632,13 +632,13 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         conf.setUseHostNameAsBookieID(false);
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setAdvertisedAddress("unknown");
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Should not start a bookie with ip if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -659,13 +659,13 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         conf.setUseHostNameAsBookieID(true);
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
 
         conf.setUseHostNameAsBookieID(false);
         try {
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Should not start a bookie with ip if the bookie has been started with an ip");
         } catch (InvalidCookieException e) {
             // expected
@@ -690,7 +690,7 @@
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         try {
             conf.setUseHostNameAsBookieID(true);
-            new Bookie(conf);
+            new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException ice) {
             // correct behaviour
@@ -711,7 +711,7 @@
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
         Versioned<Cookie> zkCookie = Cookie.readFromRegistrationManager(rm, conf);
@@ -743,7 +743,7 @@
             .setLedgerDirNames(ledgerDirs)
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
-        Bookie b = new Bookie(conf); // should work fine
+        Bookie b = new BookieImpl(conf); // should work fine
         b.start();
         b.shutdown();
         Versioned<Cookie> zkCookie = Cookie.readFromRegistrationManager(rm, conf);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
index d205245..215e159 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
@@ -323,20 +323,20 @@
                 expectedPreAllocatedLogIDDuringInitialization + 1, entryLoggerAllocator.getPreallocatedLogId());
 
         for (int i = 0; i < numDirs - 1; i++) {
-            ledgerDirsManager.addToFilledDirs(Bookie.getCurrentDirectory(new File(ledgerDirs[i])));
+            ledgerDirsManager.addToFilledDirs(BookieImpl.getCurrentDirectory(new File(ledgerDirs[i])));
         }
 
         /*
          * this is the only non-filled ledgerDir so it should be used for creating new entryLog
          */
-        File nonFilledLedgerDir = Bookie.getCurrentDirectory(new File(ledgerDirs[numDirs - 1]));
+        File nonFilledLedgerDir = BookieImpl.getCurrentDirectory(new File(ledgerDirs[numDirs - 1]));
 
         entryLogManager.createNewLog(ledgerId);
         BufferedLogChannel newLogChannel = entryLogManager.getCurrentLogForLedger(ledgerId);
         Assert.assertEquals("Directory of newly created BufferedLogChannel file", nonFilledLedgerDir.getAbsolutePath(),
                 newLogChannel.getLogFile().getParentFile().getAbsolutePath());
 
-        ledgerDirsManager.addToFilledDirs(Bookie.getCurrentDirectory(new File(ledgerDirs[numDirs - 1])));
+        ledgerDirsManager.addToFilledDirs(BookieImpl.getCurrentDirectory(new File(ledgerDirs[numDirs - 1])));
 
         // new entrylog creation should succeed, though there is no writable ledgerDir
         entryLogManager.createNewLog(ledgerId);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexCorruptionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexCorruptionTest.java
index b12dfe9..8eeec6e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexCorruptionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexCorruptionTest.java
@@ -55,7 +55,7 @@
     public void testNoSuchLedger() throws Exception {
         LOG.debug("Testing NoSuchLedger");
 
-        SyncThread syncThread = bs.get(0).getBookie().syncThread;
+        SyncThread syncThread = ((BookieImpl) bs.get(0).getBookie()).syncThread;
         syncThread.suspendSync();
         // Create a ledger
         LedgerHandle lh = bkc.createLedger(1, 1, digestType, "".getBytes());
@@ -96,7 +96,7 @@
     public void testEmptyIndexPage() throws Exception {
         LOG.debug("Testing EmptyIndexPage");
 
-        SyncThread syncThread = bs.get(0).getBookie().syncThread;
+        SyncThread syncThread = ((BookieImpl) bs.get(0).getBookie()).syncThread;
         assertNotNull("Not found SyncThread.", syncThread);
 
         syncThread.suspendSync();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
index 54f6c86..898ef53 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
@@ -72,8 +72,8 @@
         ledgerDir.delete();
         ledgerDir.mkdir();
         // Create current directories
-        Bookie.getCurrentDirectory(journalDir).mkdir();
-        Bookie.getCurrentDirectory(ledgerDir).mkdir();
+        BookieImpl.getCurrentDirectory(journalDir).mkdir();
+        BookieImpl.getCurrentDirectory(ledgerDir).mkdir();
 
         conf = new ServerConfiguration();
         conf.setMetadataServiceUri(null);
@@ -390,7 +390,7 @@
     }
 
     void preCreateFileInfoForLedger(long ledgerId, int headerVersion) throws IOException {
-        File ledgerCurDir = Bookie.getCurrentDirectory(ledgerDir);
+        File ledgerCurDir = BookieImpl.getCurrentDirectory(ledgerDir);
         String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
         File indexFile = new File(ledgerCurDir, ledgerName);
         indexFile.getParentFile().mkdirs();
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 63bd7f7..1942974 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
@@ -74,7 +74,7 @@
 
     private final List<File> tempDirs = new ArrayList<File>();
 
-    private Bookie bookie;
+    private BookieImpl bookie;
 
     @Before
     public void setUp() throws Exception {
@@ -87,10 +87,10 @@
         conf.setMetadataServiceUri(null);
         conf.setJournalDirName(txnDir.getPath());
         conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
-        bookie = new Bookie(conf);
+        bookie = new BookieImpl(conf);
 
         activeLedgers = new SnapshotMap<Long, Boolean>();
-        ledgerCache = ((InterleavedLedgerStorage) bookie.ledgerStorage.getUnderlyingLedgerStorage()).ledgerCache;
+        ledgerCache = ((InterleavedLedgerStorage) bookie.getLedgerStorage().getUnderlyingLedgerStorage()).ledgerCache;
     }
 
     @After
@@ -99,7 +99,7 @@
             flushThread.interrupt();
             flushThread.join();
         }
-        bookie.ledgerStorage.shutdown();
+        bookie.getLedgerStorage().shutdown();
         FileUtils.deleteDirectory(txnDir);
         FileUtils.deleteDirectory(ledgerDir);
         for (File dir : tempDirs) {
@@ -117,8 +117,8 @@
         if (ledgerCache != null) {
             ledgerCache.close();
         }
-        ledgerCache = ((InterleavedLedgerStorage) bookie.ledgerStorage.getUnderlyingLedgerStorage())
-                .ledgerCache = new LedgerCacheImpl(conf, activeLedgers, bookie.getIndexDirsManager());
+        ledgerCache = ((InterleavedLedgerStorage) bookie.getLedgerStorage().getUnderlyingLedgerStorage())
+            .ledgerCache = new LedgerCacheImpl(conf, activeLedgers, bookie.getIndexDirsManager());
         flushThread = new Thread() {
                 public void run() {
                     while (true) {
@@ -275,9 +275,9 @@
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setLedgerDirNames(new String[] { ledgerDir1.getAbsolutePath(), ledgerDir2.getAbsolutePath() });
 
-        Bookie bookie = new Bookie(conf);
+        BookieImpl bookie = new BookieImpl(conf);
         InterleavedLedgerStorage ledgerStorage =
-                ((InterleavedLedgerStorage) bookie.ledgerStorage.getUnderlyingLedgerStorage());
+            ((InterleavedLedgerStorage) bookie.getLedgerStorage().getUnderlyingLedgerStorage());
         LedgerCacheImpl ledgerCache = (LedgerCacheImpl) ledgerStorage.ledgerCache;
         // Create ledger index file
         ledgerStorage.setMasterKey(1, "key".getBytes());
@@ -315,10 +315,10 @@
     public void testIndexPageEvictionWriteOrder() throws Exception {
         final int numLedgers = 10;
         File journalDir = createTempDir("bookie", "journal");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(journalDir));
 
         File ledgerDir = createTempDir("bookie", "ledger");
-        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(ledgerDir));
 
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setMetadataServiceUri(null);
@@ -328,11 +328,11 @@
             .setPageLimit(1)
             .setLedgerStorageClass(InterleavedLedgerStorage.class.getName());
 
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         b.start();
         for (int i = 1; i <= numLedgers; i++) {
             ByteBuf packet = generateEntry(i, 1);
-            b.addEntry(packet, false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
+            b.addEntry(packet, false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
         }
 
         conf = TestBKConfiguration.newServerConfiguration();
@@ -340,7 +340,7 @@
         conf.setJournalDirName(journalDir.getPath())
             .setLedgerDirNames(new String[] { ledgerDir.getPath() });
 
-        b = new Bookie(conf);
+        b = new BookieImpl(conf);
         for (int i = 1; i <= numLedgers; i++) {
             try {
                 b.readEntry(i, 1);
@@ -714,8 +714,8 @@
     @Test
     public void testEntryMemTableFlushFailure() throws Exception {
         File tmpDir = createTempDir("bkTest", ".dir");
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -723,14 +723,15 @@
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
         conf.setLedgerStorageClass(FlushTestSortedLedgerStorage.class.getName());
 
-        Bookie bookie = new Bookie(conf);
-        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.ledgerStorage;
+        Bookie bookie = new BookieImpl(conf);
+        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.
+                getLedgerStorage();
         EntryMemTable memTable = flushTestSortedLedgerStorage.memTable;
 
         // this bookie.addEntry call is required. FileInfo for Ledger 1 would be created with this call.
         // without the fileinfo, 'flushTestSortedLedgerStorage.addEntry' calls will fail
         // because of BOOKKEEPER-965 change.
-        bookie.addEntry(generateEntry(1, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(1, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
 
         flushTestSortedLedgerStorage.addEntry(generateEntry(1, 2));
         assertFalse("Bookie is expected to be in ReadWrite mode", bookie.isReadOnly());
@@ -759,8 +760,8 @@
     public void testSortedLedgerFlushFailure() throws Exception {
         // most of the code is same to the testEntryMemTableFlushFailure
         File tmpDir = createTempDir("bkTest", ".dir");
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -769,12 +770,13 @@
             .setJournalDirName(tmpDir.toString())
             .setLedgerStorageClass(FlushTestSortedLedgerStorage.class.getName());
 
-        Bookie bookie = new Bookie(conf);
+        Bookie bookie = new BookieImpl(conf);
         bookie.start();
-        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.ledgerStorage;
+        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.
+                getLedgerStorage();
         EntryMemTable memTable = flushTestSortedLedgerStorage.memTable;
 
-        bookie.addEntry(generateEntry(1, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(1, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
         flushTestSortedLedgerStorage.addEntry(generateEntry(1, 2));
         assertFalse("Bookie is expected to be in ReadWrite mode", bookie.isReadOnly());
         assertTrue("EntryMemTable SnapShot is expected to be empty", memTable.snapshot.isEmpty());
@@ -819,8 +821,9 @@
         // enable entrylog per ledger
         conf.setEntryLogPerLedgerEnabled(true);
 
-        Bookie bookie = new Bookie(conf);
-        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.ledgerStorage;
+        Bookie bookie = new BookieImpl(conf);
+        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.
+                getLedgerStorage();
         EntryMemTable memTable = flushTestSortedLedgerStorage.memTable;
 
         /*
@@ -829,9 +832,9 @@
          * 'flushTestSortedLedgerStorage.addEntry' calls will fail because of
          * BOOKKEEPER-965 change.
          */
-        bookie.addEntry(generateEntry(1, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
-        bookie.addEntry(generateEntry(2, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
-        bookie.addEntry(generateEntry(3, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(1, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(2, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(3, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
 
         flushTestSortedLedgerStorage.addEntry(generateEntry(1, 2));
         flushTestSortedLedgerStorage.addEntry(generateEntry(2, 2));
@@ -859,8 +862,9 @@
         // enable entrylog per ledger
         conf.setEntryLogPerLedgerEnabled(true);
 
-        Bookie bookie = new Bookie(conf);
-        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.ledgerStorage;
+        Bookie bookie = new BookieImpl(conf);
+        FlushTestSortedLedgerStorage flushTestSortedLedgerStorage = (FlushTestSortedLedgerStorage) bookie.
+                getLedgerStorage();
         EntryMemTable memTable = flushTestSortedLedgerStorage.memTable;
 
         /*
@@ -869,9 +873,9 @@
          * 'flushTestSortedLedgerStorage.addEntry' calls will fail because of
          * BOOKKEEPER-965 change.
          */
-        bookie.addEntry(generateEntry(1, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
-        bookie.addEntry(generateEntry(2, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
-        bookie.addEntry(generateEntry(3, 1), false, new Bookie.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(1, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(2, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
+        bookie.addEntry(generateEntry(3, 1), false, new BookieImpl.NopWriteCallback(), null, "passwd".getBytes());
 
         flushTestSortedLedgerStorage.addEntry(generateEntry(1, 4));
         flushTestSortedLedgerStorage.addEntry(generateEntry(2, 4));
@@ -902,8 +906,8 @@
         String[] ledgerDirsPath = new String[numOfLedgerDirs];
         for (int i = 0; i < numOfLedgerDirs; i++) {
             ledgerDir = createTempDir("bkTest", ".dir");
-            curDir = Bookie.getCurrentDirectory(ledgerDir);
-            Bookie.checkDirectoryStructure(curDir);
+            curDir = BookieImpl.getCurrentDirectory(ledgerDir);
+            BookieImpl.checkDirectoryStructure(curDir);
             ledgerDirsPath[i] = ledgerDir.getAbsolutePath();
         }
         return ledgerDirsPath;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java
index 6a877e4..31b6da3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java
@@ -91,8 +91,8 @@
         PowerMockito.mockStatic(Executors.class);
 
         File tmpDir = createTempDir("bkTest", ".dir");
-        curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         conf = TestBKConfiguration.newServerConfiguration();
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
@@ -306,12 +306,12 @@
         HashMap<File, Float> usageMap;
 
         File tmpDir1 = createTempDir("bkTest", ".dir");
-        File curDir1 = Bookie.getCurrentDirectory(tmpDir1);
-        Bookie.checkDirectoryStructure(curDir1);
+        File curDir1 = BookieImpl.getCurrentDirectory(tmpDir1);
+        BookieImpl.checkDirectoryStructure(curDir1);
 
         File tmpDir2 = createTempDir("bkTest", ".dir");
-        File curDir2 = Bookie.getCurrentDirectory(tmpDir2);
-        Bookie.checkDirectoryStructure(curDir2);
+        File curDir2 = BookieImpl.getCurrentDirectory(tmpDir2);
+        BookieImpl.checkDirectoryStructure(curDir2);
 
         conf.setDiskUsageThreshold(nospace);
         conf.setDiskLowWaterMarkUsageThreshold(lwm);
@@ -390,12 +390,12 @@
         final float lwm = 0.80f;
 
         File tmpDir1 = createTempDir("bkTest", ".dir");
-        File curDir1 = Bookie.getCurrentDirectory(tmpDir1);
-        Bookie.checkDirectoryStructure(curDir1);
+        File curDir1 = BookieImpl.getCurrentDirectory(tmpDir1);
+        BookieImpl.checkDirectoryStructure(curDir1);
 
         File tmpDir2 = createTempDir("bkTest", ".dir");
-        File curDir2 = Bookie.getCurrentDirectory(tmpDir2);
-        Bookie.checkDirectoryStructure(curDir2);
+        File curDir2 = BookieImpl.getCurrentDirectory(tmpDir2);
+        BookieImpl.checkDirectoryStructure(curDir2);
 
         conf.setDiskUsageThreshold(nospace);
         conf.setDiskLowWaterMarkUsageThreshold(lwm);
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 9d69f3e..eaa7505 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
@@ -216,7 +216,7 @@
                 .setLedgerStorageClass(ledgerStorageClassName);
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
-        File ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs())[0];
+        File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(conf);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
@@ -237,7 +237,7 @@
             handle.close();
         }
 
-        LastLogMark lastLogMarkAfterFirstSetOfAdds = server.getBookie().journals.get(0).getLastLogMark();
+        LastLogMark lastLogMarkAfterFirstSetOfAdds = ((BookieImpl) server.getBookie()).journals.get(0).getLastLogMark();
         LogMark curMarkAfterFirstSetOfAdds = lastLogMarkAfterFirstSetOfAdds.getCurMark();
 
         File lastMarkFile = new File(ledgerDir, "lastMark");
@@ -258,7 +258,7 @@
         Assert.assertTrue("lastMark file must be existing, because checkpoint should have happened",
                 lastMarkFile.exists());
 
-        LastLogMark lastLogMarkAfterCheckpoint = server.getBookie().journals.get(0).getLastLogMark();
+        LastLogMark lastLogMarkAfterCheckpoint = ((BookieImpl) server.getBookie()).journals.get(0).getLastLogMark();
         LogMark curMarkAfterCheckpoint = lastLogMarkAfterCheckpoint.getCurMark();
 
         LogMark rolledLogMark = readLastMarkFile(lastMarkFile);
@@ -291,7 +291,8 @@
         // wait for flushInterval for SyncThread to do next iteration of checkpoint
         executorController.advance(Duration.ofMillis(conf.getFlushInterval()));
 
-        LastLogMark lastLogMarkAfterSecondSetOfAdds = server.getBookie().journals.get(0).getLastLogMark();
+        LastLogMark lastLogMarkAfterSecondSetOfAdds = ((BookieImpl) server.getBookie()).
+                journals.get(0).getLastLogMark();
         LogMark curMarkAfterSecondSetOfAdds = lastLogMarkAfterSecondSetOfAdds.getCurMark();
 
         rolledLogMark = readLastMarkFile(lastMarkFile);
@@ -344,13 +345,13 @@
 
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
-        File ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs())[0];
+        File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(conf);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         BookKeeper bkClient = new BookKeeper(clientConf);
-        InterleavedLedgerStorage ledgerStorage = (InterleavedLedgerStorage) server.getBookie().ledgerStorage;
+        InterleavedLedgerStorage ledgerStorage = (InterleavedLedgerStorage) server.getBookie().getLedgerStorage();
 
         int numOfEntries = 5;
         byte[] dataBytes = "data".getBytes();
@@ -422,7 +423,7 @@
 
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
-        File ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs())[0];
+        File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(conf);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
@@ -485,13 +486,13 @@
 
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
-        File ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs())[0];
+        File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(conf);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         BookKeeper bkClient = new BookKeeper(clientConf);
-        InterleavedLedgerStorage ledgerStorage = (InterleavedLedgerStorage) server.getBookie().ledgerStorage;
+        InterleavedLedgerStorage ledgerStorage = (InterleavedLedgerStorage) server.getBookie().getLedgerStorage();
         EntryLogger entryLogger = ledgerStorage.entryLogger;
         EntryLogManagerForEntryLogPerLedger entryLogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger
                 .getEntryLogManager();
@@ -589,7 +590,7 @@
 
         Assert.assertEquals("Number of JournalDirs", 1, conf.getJournalDirs().length);
         // we know there is only one ledgerDir
-        File ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs())[0];
+        File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(conf);
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
@@ -653,7 +654,7 @@
         // Journal file
         File[] journalDirs = conf.getJournalDirs();
         for (File journalDir : journalDirs) {
-            File journalDirectory = Bookie.getCurrentDirectory(journalDir);
+            File journalDirectory = BookieImpl.getCurrentDirectory(journalDir);
             List<Long> journalLogsId = Journal.listJournalIds(journalDirectory, null);
             for (long journalId : journalLogsId) {
                 File journalFile = new File(journalDirectory, Long.toHexString(journalId) + ".txn");
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java
index 76244b7..f5f2c2d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java
@@ -54,7 +54,7 @@
 
     @Test
     public void testLedgerDeleteNotification() throws Exception {
-        LedgerStorage ledgerStorage = bs.get(0).getBookie().ledgerStorage;
+        LedgerStorage ledgerStorage = bs.get(0).getBookie().getLedgerStorage();
 
         long deletedLedgerId = 5;
         ledgerStorage.setMasterKey(deletedLedgerId, new byte[0]);
@@ -141,7 +141,7 @@
          * purpose.
          */
         newBookieConf.setMetadataServiceUri(null);
-        Bookie newbookie = new Bookie(newBookieConf);
+        BookieImpl newbookie = new BookieImpl(newBookieConf);
         /*
          * since 'newbookie' uses the same data as original Bookie, it should be
          * able to read journal of the original bookie and hence explicitLac buf
@@ -226,9 +226,10 @@
         /*
          * flush ledgerStorage so that header of fileinfo is flushed.
          */
-        bs.get(0).getBookie().ledgerStorage.flush();
+        bs.get(0).getBookie().getLedgerStorage().flush();
 
-        ReadOnlyFileInfo fileInfo = getFileInfo(ledgerId, Bookie.getCurrentDirectories(bsConfs.get(0).getLedgerDirs()));
+        ReadOnlyFileInfo fileInfo = getFileInfo(ledgerId,
+                                                BookieImpl.getCurrentDirectories(bsConfs.get(0).getLedgerDirs()));
         fileInfo.readHeader();
         ByteBuf explicitLacBufReadFromFileInfo = fileInfo.getExplicitLac();
 
@@ -305,7 +306,7 @@
          * purpose.
          */
         newBookieConf.setMetadataServiceUri(null);
-        Bookie newbookie = new Bookie(newBookieConf);
+        BookieImpl newbookie = new BookieImpl(newBookieConf);
         /*
          * since 'newbookie' uses the same data as original Bookie, it should be
          * able to read journal of the original bookie.
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTestBase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTestBase.java
index 3de15cd..f483cee 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTestBase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTestBase.java
@@ -60,8 +60,8 @@
         ledgerDir = createTempDir("ledger");
 
         // create current directories
-        Bookie.getCurrentDirectory(journalDir).mkdir();
-        Bookie.getCurrentDirectory(ledgerDir).mkdir();
+        BookieImpl.getCurrentDirectory(journalDir).mkdir();
+        BookieImpl.getCurrentDirectory(ledgerDir).mkdir();
 
         // build the configuration
         conf.setMetadataServiceUri(null);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SingleBookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SingleBookieInitializationTest.java
index 20856aa..c631afa 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SingleBookieInitializationTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SingleBookieInitializationTest.java
@@ -89,7 +89,7 @@
         conf.setMinUsableSizeForEntryLogCreation(Long.MIN_VALUE);
         conf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName());
 
-        bookie = new Bookie(conf);
+        bookie = new BookieImpl(conf);
         bookie.start();
 
         CompletableFuture<Integer> writeFuture = new CompletableFuture<>();
@@ -111,7 +111,7 @@
         conf.setMinUsableSizeForEntryLogCreation(Long.MAX_VALUE);
         conf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName());
 
-        bookie = new Bookie(conf);
+        bookie = new BookieImpl(conf);
         bookie.start();
 
         try {
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 01383e2..c1ead05 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
@@ -101,8 +101,8 @@
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
         ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java
index 3e54b76a..5f72dd9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java
@@ -98,8 +98,8 @@
     @Before
     public void setUp() throws Exception {
         this.rootDir = createTempDir("bkTest", ".dir");
-        this.curDir = Bookie.getCurrentDirectory(rootDir);
-        Bookie.checkDirectoryStructure(curDir);
+        this.curDir = BookieImpl.getCurrentDirectory(rootDir);
+        BookieImpl.checkDirectoryStructure(curDir);
         this.conf = TestBKConfiguration.newServerConfiguration();
         this.dirsMgr = new LedgerDirsManager(
             conf,
@@ -307,7 +307,7 @@
         conf.setJournalDirName(ledgerDir1.toString());
         conf.setLedgerDirNames(new String[] { ledgerDir1.getAbsolutePath(),
                 ledgerDir2.getAbsolutePath() });
-        Bookie bookie = new Bookie(conf);
+        BookieImpl bookie = new BookieImpl(conf);
         EntryLogger entryLogger = new EntryLogger(conf,
                 bookie.getLedgerDirsManager());
         InterleavedLedgerStorage ledgerStorage =
@@ -687,7 +687,7 @@
         conf.setLedgerDirNames(new String[] { ledgerDir.getAbsolutePath()});
         conf.setLedgerStorageClass(ledgerStorageClass);
         conf.setEntryLogPerLedgerEnabled(entryLogPerLedgerEnabled);
-        Bookie bookie = new Bookie(conf);
+        BookieImpl bookie = new BookieImpl(conf);
         CompactableLedgerStorage ledgerStorage = (CompactableLedgerStorage) bookie.ledgerStorage;
         Random rand = new Random(0);
 
@@ -821,8 +821,8 @@
         String[] ledgerDirsPath = new String[numOfLedgerDirs];
         for (int i = 0; i < numOfLedgerDirs; i++) {
             ledgerDir = createTempDir("bkTest", ".dir");
-            curDir = Bookie.getCurrentDirectory(ledgerDir);
-            Bookie.checkDirectoryStructure(curDir);
+            curDir = BookieImpl.getCurrentDirectory(ledgerDir);
+            BookieImpl.checkDirectoryStructure(curDir);
             ledgerDirsPath[i] = ledgerDir.getAbsolutePath();
         }
         return ledgerDirsPath;
@@ -1616,8 +1616,8 @@
         File curDir;
         for (int i = 0; i < numberOfLedgerDirs; i++) {
             ledgerDir = createTempDir("bkTest", ".dir").getAbsoluteFile();
-            curDir = Bookie.getCurrentDirectory(ledgerDir);
-            Bookie.checkDirectoryStructure(curDir);
+            curDir = BookieImpl.getCurrentDirectory(ledgerDir);
+            BookieImpl.checkDirectoryStructure(curDir);
             ledgerDirs.add(ledgerDir);
             ledgerDirsPath[i] = ledgerDir.getPath();
             curDirs.add(curDir);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java
index 59afa0c..050d0e9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java
@@ -150,8 +150,8 @@
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         conf.setLedgerDirNames(new String[]{tmpDir.toString()});
         ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
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 840f2bd..35913ba 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
@@ -148,8 +148,10 @@
         Cookie cookie = Cookie.readFromRegistrationManager(rm, conf).getValue();
         Cookie.Builder cookieBuilder = Cookie.newBuilder(cookie);
         conf.setUseHostNameAsBookieID(false); // sets to hostname
-        final String newBookieHost = Bookie.getBookieAddress(conf).toString();
+
+        final String newBookieHost = BookieImpl.getBookieAddress(conf).toString();
         cookieBuilder.setBookieId(newBookieHost);
+
         cookieBuilder.build().writeToRegistrationManager(rm, conf, Version.NEW);
         verifyCookieInZooKeeper(conf, 2);
 
@@ -185,7 +187,7 @@
         bks.shutdown();
 
         String zkCookiePath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf)
-            + "/" + COOKIE_NODE + "/" + Bookie.getBookieAddress(conf);
+            + "/" + COOKIE_NODE + "/" + BookieImpl.getBookieAddress(conf);
         Assert.assertNotNull("Cookie path doesn't still exists!", zkc.exists(zkCookiePath, false));
         zkc.delete(zkCookiePath, -1);
         Assert.assertNull("Cookie path still exists!", zkc.exists(zkCookiePath, false));
@@ -237,11 +239,11 @@
         verifyCookieInZooKeeper(newconf, 1);
 
         for (File journalDir : conf.getJournalDirs()) {
-            journalDir = Bookie.getCurrentDirectory(journalDir);
+            journalDir = BookieImpl.getCurrentDirectory(journalDir);
             Cookie jCookie = Cookie.readFromDirectory(journalDir);
             jCookie.verify(cookie);
         }
-        File[] ledgerDir = Bookie.getCurrentDirectories(conf.getLedgerDirs());
+        File[] ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs());
         for (File dir : ledgerDir) {
             Cookie lCookie = Cookie.readFromDirectory(dir);
             lCookie.verify(cookie);
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 b49950b..bb95286 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
@@ -158,7 +158,7 @@
             .setBookiePort(bookiePort);
         Bookie b = null;
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException e) {
             // correct behaviour
@@ -166,14 +166,14 @@
         }
 
         FileSystemUpgrade.upgrade(conf); // should work fine
-        b = new Bookie(conf);
+        b = new BookieImpl(conf);
         b.start();
         b.shutdown();
         b = null;
 
         FileSystemUpgrade.rollback(conf);
         try {
-            b = new Bookie(conf);
+            b = new BookieImpl(conf);
             fail("Shouldn't have been able to start");
         } catch (BookieException.InvalidCookieException e) {
             // correct behaviour
@@ -182,7 +182,7 @@
 
         FileSystemUpgrade.upgrade(conf);
         FileSystemUpgrade.finalizeUpgrade(conf);
-        b = new Bookie(conf);
+        b = new BookieImpl(conf);
         b.start();
         b.shutdown();
         b = null;
@@ -221,7 +221,7 @@
             .setBookiePort(bookiePort)
             .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
         FileSystemUpgrade.upgrade(conf); // should work fine with current directory
-        Bookie b = new Bookie(conf);
+        Bookie b = new BookieImpl(conf);
         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 5c24633..14c1e5c 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
@@ -22,18 +22,14 @@
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.Set;
-
 import lombok.extern.slf4j.Slf4j;
-
-import org.apache.bookkeeper.bookie.Bookie;
+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;
@@ -83,8 +79,8 @@
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         log.info("Using temp directory: {}", tmpDir);
 
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 780b8ec..1890ac4 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
@@ -22,17 +22,14 @@
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.Set;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.Bookie.NoLedgerException;
+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;
@@ -80,8 +77,8 @@
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         System.out.println(tmpDir);
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
index f5d3cef..f2a3bec 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
@@ -23,20 +23,17 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import com.google.common.collect.Lists;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufUtil;
 import io.netty.buffer.Unpooled;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.Bookie.NoEntryException;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.EntryLocation;
 import org.apache.bookkeeper.bookie.EntryLogger;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -61,15 +58,15 @@
         tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
         conf.setGcWaitTime(gcWaitTime);
         conf.setLedgerStorageClass(DbLedgerStorage.class.getName());
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
-        Bookie bookie = new Bookie(conf);
+        BookieImpl bookie = new BookieImpl(conf);
 
         ledgerDirsManager = bookie.getLedgerDirsManager();
         storage = (DbLedgerStorage) bookie.getLedgerStorage();
@@ -254,7 +251,7 @@
         conf.setLedgerDirNames(new String[] { firstDir.getCanonicalPath(), secondDir.getCanonicalPath() });
 
         // Should not fail
-        Bookie bookie = new Bookie(conf);
+        Bookie bookie = new BookieImpl(conf);
         assertEquals(2, ((DbLedgerStorage) bookie.getLedgerStorage()).getLedgerStorageList().size());
 
         bookie.shutdown();
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 e5feef3..635a8cb 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
@@ -22,17 +22,15 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.Unpooled;
-
 import java.io.File;
 import java.io.IOException;
 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;
@@ -108,8 +106,8 @@
         tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
         ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
@@ -118,7 +116,7 @@
         conf.setProperty(DbLedgerStorage.WRITE_CACHE_MAX_SIZE_MB, 1);
         conf.setProperty(DbLedgerStorage.MAX_THROTTLE_TIME_MILLIS, 1000);
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
-        Bookie bookie = new Bookie(conf);
+        Bookie bookie = new BookieImpl(conf);
 
         storage = (DbLedgerStorage) bookie.getLedgerStorage();
     }
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 7bdbcd5..1b296fb 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
@@ -21,19 +21,15 @@
 package org.apache.bookkeeper.bookie.storage.ldb;
 
 import static org.junit.Assert.assertEquals;
-
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.Set;
-
-import org.apache.bookkeeper.bookie.Bookie;
+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;
@@ -80,8 +76,8 @@
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
-        File curDir = Bookie.getCurrentDirectory(tmpDir);
-        Bookie.checkDirectoryStructure(curDir);
+        File curDir = BookieImpl.getCurrentDirectory(tmpDir);
+        BookieImpl.checkDirectoryStructure(curDir);
 
         System.out.println(tmpDir);
 
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 a824b4a..3baa1bb 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
@@ -30,7 +30,6 @@
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import com.google.common.net.InetAddresses;
 import java.io.File;
 import java.util.ArrayList;
@@ -46,8 +45,7 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.component.ComponentStarter;
@@ -210,7 +208,7 @@
         }
         Assert.assertFalse("initBookie shouldn't have succeeded, since cookie in ZK is not deleted yet",
                 BookKeeperAdmin.initBookie(confOfExistingBookie));
-        String bookieId = Bookie.getBookieId(confOfExistingBookie).toString();
+        String bookieId = BookieImpl.getBookieId(confOfExistingBookie).toString();
         String bookieCookiePath =
             ZKMetadataDriverBase.resolveZkLedgersRootPath(confOfExistingBookie)
                 + "/" + BookKeeperConstants.COOKIE_NODE
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
index 168d9af..f614dcd 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
@@ -38,6 +38,7 @@
 
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
 import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
@@ -75,7 +76,7 @@
     private void restartBookieSlow() throws Exception{
         ServerConfiguration conf = killBookie(0);
 
-        Bookie delayBookie = new Bookie(conf) {
+        Bookie delayBookie = new BookieImpl(conf) {
                 @Override
                 public void recoveryAddEntry(ByteBuf entry, WriteCallback cb,
                                              Object ctx, byte[] masterKey)
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
index a7fe962..c7db87d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
@@ -31,6 +31,7 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.common.testing.annotations.FlakyTest;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -77,7 +78,7 @@
             BookKeeperCheckInfoReader client, ServerConfiguration conf, final long initialFreeDiskSpace,
             final long finalFreeDiskSpace, final AtomicBoolean useFinal) throws Exception {
         final AtomicBoolean ready = useFinal == null ? new AtomicBoolean(false) : useFinal;
-        Bookie bookieWithCustomFreeDiskSpace = new Bookie(conf) {
+        Bookie bookieWithCustomFreeDiskSpace = new BookieImpl(conf) {
             long startTime = System.currentTimeMillis();
             @Override
             public long getTotalFreeSpace() {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java
index 1fddabb..020fabb 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java
@@ -20,11 +20,9 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-
 import java.util.Iterator;
-
 import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BKException.BKIllegalOpException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.common.testing.annotations.FlakyTest;
@@ -88,7 +86,7 @@
          * this decommisionBookie should make sure that there are no
          * underreplicated ledgers because of this bookie
          */
-        bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf));
+        bkAdmin.decommissionBookie(BookieImpl.getBookieId(killedBookieConf));
         bkAdmin.triggerAudit();
         Thread.sleep(500);
         Iterator<UnderreplicatedLedger> ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null);
@@ -101,7 +99,7 @@
         }
 
         killedBookieConf = killBookie(0);
-        bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf));
+        bkAdmin.decommissionBookie(BookieImpl.getBookieId(killedBookieConf));
         bkAdmin.triggerAudit();
         Thread.sleep(500);
         ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null);
@@ -161,7 +159,7 @@
          * info. Check BOOKKEEPER-237 and BOOKKEEPER-325. But later
          * ReplicationWorker will fence the ledger.
          */
-        bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf));
+        bkAdmin.decommissionBookie(BookieImpl.getBookieId(killedBookieConf));
         bkAdmin.triggerAudit();
         Thread.sleep(500);
         Iterator<UnderreplicatedLedger> ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null);
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 7c1e3da..128bef4 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
@@ -28,14 +28,12 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import com.google.common.collect.Lists;
 import io.netty.buffer.AbstractByteBufAllocator;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -51,9 +49,8 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 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.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException.BKLedgerClosedException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -1487,7 +1484,7 @@
         }
     }
 
-    static class CorruptReadBookie extends Bookie {
+    static class CorruptReadBookie extends BookieImpl {
 
         static final Logger LOG = LoggerFactory.getLogger(CorruptReadBookie.class);
         ByteBuf localBuf;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
index 5ece1a1..8a0a1c2 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
@@ -35,6 +35,7 @@
 
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -196,7 +197,7 @@
 
     private void startUnauthorizedBookie(ServerConfiguration conf, final CountDownLatch latch)
             throws Exception {
-        Bookie sBookie = new Bookie(conf) {
+        Bookie sBookie = new BookieImpl(conf) {
             @Override
             public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
@@ -221,7 +222,7 @@
     // simulate slow adds, then become normal when recover,
     // so no ensemble change when recovering ledger on this bookie.
     private void startDeadBookie(ServerConfiguration conf, final CountDownLatch latch) throws Exception {
-        Bookie dBookie = new Bookie(conf) {
+        Bookie dBookie = new BookieImpl(conf) {
             @Override
             public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCmdTest.java
index 4055917..83d9310 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCmdTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCmdTest.java
@@ -27,6 +27,7 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.bookie.BookieAccessor;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieShell;
 import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
@@ -68,7 +69,7 @@
 
         bs.forEach(bookieServer -> {
             try {
-                BookieAccessor.forceFlush(bookieServer.getBookie());
+                BookieAccessor.forceFlush((BookieImpl) bookieServer.getBookie());
             } catch (IOException e) {
                 LOG.error("Error forceFlush:", e);
             }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
index b89c370..e8f83d1 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
@@ -36,6 +36,7 @@
 
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -188,7 +189,7 @@
         BookieId host = beforelh.getCurrentEnsemble().get(slowBookieIdx);
         ServerConfiguration conf = killBookie(host);
 
-        Bookie fakeBookie = new Bookie(conf) {
+        Bookie fakeBookie = new BookieImpl(conf) {
             @Override
             public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
@@ -247,7 +248,7 @@
 
         // Add a dead bookie to the cluster
         ServerConfiguration conf = newServerConfiguration();
-        Bookie deadBookie1 = new Bookie(conf) {
+        Bookie deadBookie1 = new BookieImpl(conf) {
             @Override
             public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
@@ -328,7 +329,7 @@
 
         // Add a dead bookie to the cluster
         ServerConfiguration conf = newServerConfiguration();
-        Bookie deadBookie1 = new Bookie(conf) {
+        Bookie deadBookie1 = new BookieImpl(conf) {
             @Override
             public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
@@ -411,7 +412,7 @@
     }
 
     private void startDeadBookie(ServerConfiguration conf) throws Exception {
-        Bookie rBookie = new Bookie(conf) {
+        Bookie rBookie = new BookieImpl(conf) {
             @Override
             public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey)
                     throws IOException, BookieException {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java
index d50dd96..8ec887e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java
@@ -38,6 +38,7 @@
 import lombok.extern.slf4j.Slf4j;
 
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -182,7 +183,7 @@
         for (int i = 0; i < 3; ++i) {
             Bookie bookie = bs.get(i).getBookie();
             File[] ledgerDirs = bsConfs.get(i).getLedgerDirs();
-            LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
+            LedgerDirsManager ledgerDirsManager = ((BookieImpl) bookie).getLedgerDirsManager();
             ledgerDirsManager.addToFilledDirs(new File(ledgerDirs[0], "current"));
         }
 
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 0b52501..ef4d26b 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
@@ -23,10 +23,8 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
-
 import java.io.IOException;
 import java.util.Enumeration;
 import java.util.concurrent.CompletableFuture;
@@ -38,9 +36,8 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
@@ -486,7 +483,7 @@
         assertEquals("recovery callback should be triggered only once", 0, numFailureCalls.get());
     }
 
-    static class DelayResponseBookie extends Bookie {
+    static class DelayResponseBookie extends BookieImpl {
 
         static final class WriteCallbackEntry {
 
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 e6069d6..f60161d 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
@@ -23,9 +23,7 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-
 import io.netty.buffer.ByteBuf;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
@@ -34,9 +32,9 @@
 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.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.SortedLedgerStorage;
@@ -78,7 +76,7 @@
         });
     }
 
-    static class FakeBookie extends Bookie {
+    static class FakeBookie extends BookieImpl {
 
         final long expectedEntryToFail;
         final boolean stallOrRespondNull;
@@ -176,7 +174,7 @@
         assertEquals(BKException.Code.OK, rcHolder.get());
     }
 
-    static class SlowReadLacBookie extends Bookie {
+    static class SlowReadLacBookie extends BookieImpl {
 
         private final long lacToSlowRead;
         private final CountDownLatch 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 cad0718..62286e1 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
@@ -21,7 +21,6 @@
 package org.apache.bookkeeper.client;
 
 import static org.junit.Assert.assertEquals;
-
 import java.io.IOException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -29,8 +28,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieShell;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -75,10 +73,9 @@
         String[] argv = new String[] { "updateledgers", "-b", "hostname", "-v", "true", "-p", "2" };
         final ServerConfiguration conf = bsConfs.get(0);
         conf.setUseHostNameAsBookieID(true);
-        BookieSocketAddress toBookieId = Bookie.getBookieAddress(conf);
+        BookieSocketAddress toBookieId = BookieImpl.getBookieAddress(conf);
         BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":"
                 + conf.getBookiePort()).toBookieId();
-
         updateLedgerCmd(argv, 0, conf);
 
         int updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
index 788922d..52cbd7e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
@@ -24,14 +24,12 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -106,7 +104,7 @@
             BookieSocketAddress curBookieAddr = bk.getBookieAddressResolver().resolve(ensemble.get(0));
             baseConf.setUseHostNameAsBookieID(true);
             baseConf.setUseShortHostName(useShortHostName);
-            BookieSocketAddress curBookieId = Bookie.getBookieAddress(baseConf);
+            BookieSocketAddress curBookieId = BookieImpl.getBookieAddress(baseConf);
             BookieId toBookieAddr = new BookieSocketAddress(curBookieId.getHostName() + ":"
                     + curBookieAddr.getPort()).toBookieId();
             UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin);
@@ -145,7 +143,8 @@
 
             BookieId curBookieAddr = ensemble.get(0);
             baseConf.setUseHostNameAsBookieID(true);
-            BookieSocketAddress toBookieId = Bookie.getBookieAddress(baseConf);
+
+            BookieSocketAddress toBookieId = BookieImpl.getBookieAddress(baseConf);
             BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":"
                     + bk.getBookieAddressResolver().resolve(curBookieAddr).getPort()).toBookieId();
             UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin);
@@ -208,7 +207,7 @@
             assertNotNull("Couldn't find the bookie in ledger metadata!", curBookieAddr);
             baseConf.setUseHostNameAsBookieID(true);
             baseConf.setUseShortHostName(useShortHostName);
-            BookieSocketAddress toBookieId = Bookie.getBookieAddress(baseConf);
+            BookieSocketAddress toBookieId = BookieImpl.getBookieAddress(baseConf);
             BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":"
                     + curBookieAddr.getPort()).toBookieId();
             UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin);
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 cfb17ec..d463d4e 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
@@ -27,11 +27,9 @@
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import com.google.common.collect.Lists;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -51,9 +49,8 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.CheckpointSource;
 import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
 import org.apache.bookkeeper.bookie.Checkpointer;
@@ -96,7 +93,7 @@
     }
 
     private void createLedgers(int numLedgers, final Set<Long> createdLedgers) throws IOException {
-        BookieId selfBookie = Bookie.getBookieId(baseConf);
+        BookieId selfBookie = BookieImpl.getBookieId(baseConf);
         createLedgers(numLedgers, createdLedgers, selfBookie);
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java
index bab83fb..3442ade 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java
@@ -35,6 +35,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest;
@@ -92,7 +93,7 @@
             wc.writeComplete(
                 0,
                 request.getForceLedgerRequest().getLedgerId(),
-                Bookie.METAENTRY_ID_FORCE_LEDGER,
+                BookieImpl.METAENTRY_ID_FORCE_LEDGER,
                 null,
                 null);
             return null;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java
index fcd74b1..4d18645 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java
@@ -27,21 +27,19 @@
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-
-import com.google.common.util.concurrent.SettableFuture;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelPromise;
 import io.netty.channel.DefaultChannelPromise;
 import io.netty.channel.EventLoop;
-
 import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest;
 import org.apache.bookkeeper.proto.BookieProtocol.Response;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -84,7 +82,7 @@
     }
 
     private void testAsynchronousRequest(boolean result, int errorCode) throws Exception {
-        SettableFuture<Boolean> fenceResult = SettableFuture.create();
+        CompletableFuture<Boolean> fenceResult = FutureUtils.createFuture();
         when(bookie.fenceLedger(anyLong(), any())).thenReturn(fenceResult);
 
         ChannelPromise promise = new DefaultChannelPromise(channel);
@@ -104,7 +102,7 @@
         ReadEntryProcessor processor = ReadEntryProcessor.create(request, channel, requestProcessor, service, true);
         processor.run();
 
-        fenceResult.set(result);
+        fenceResult.complete(result);
         latch.await();
         verify(channel, times(1)).writeAndFlush(any(Response.class));
 
@@ -128,7 +126,7 @@
     }
 
     private void testSynchronousRequest(boolean result, int errorCode) throws Exception {
-        SettableFuture<Boolean> fenceResult = SettableFuture.create();
+        CompletableFuture<Boolean> fenceResult = FutureUtils.createFuture();
         when(bookie.fenceLedger(anyLong(), any())).thenReturn(fenceResult);
         ChannelPromise promise = new DefaultChannelPromise(channel);
         AtomicReference<Object> writtenObject = new AtomicReference<>();
@@ -144,7 +142,7 @@
         ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId,
                 1, BookieProtocol.FLAG_DO_FENCING, new byte[]{});
         ReadEntryProcessor processor = ReadEntryProcessor.create(request, channel, requestProcessor, null, true);
-        fenceResult.set(result);
+        fenceResult.complete(result);
         processor.run();
 
         latch.await();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
index db6f2c5..17eb474 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
@@ -38,6 +38,7 @@
 import org.apache.bookkeeper.auth.AuthProviderFactoryFactory;
 import org.apache.bookkeeper.auth.ClientAuthProvider;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.common.util.OrderedExecutor;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -236,7 +237,7 @@
     public void testRequestCompletesAfterDisconnectRace() throws Exception {
         ServerConfiguration conf = killBookie(0);
 
-        Bookie delayBookie = new Bookie(conf) {
+        Bookie delayBookie = new BookieImpl(conf) {
             @Override
             public ByteBuf readEntry(long ledgerId, long entryId)
                     throws IOException, NoLedgerException {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
index ea5b33f..93714c5 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
@@ -46,7 +46,7 @@
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import lombok.Cleanup;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -307,8 +307,9 @@
         ServerConfiguration bookieConf = bsConfs.get(bkIndex);
         BookieServer bk = bs.get(bkIndex);
         bookieConf.setReadOnlyModeEnabled(true);
-        bk.getBookie().getStateManager().doTransitionToReadOnlyMode();
-        bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS);
+
+        ((BookieImpl) bk.getBookie()).getStateManager().doTransitionToReadOnlyMode();
+        bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS);
 
         // grace period for publishing the bk-ledger
         LOG.debug("Waiting for Auditor to finish ledger check.");
@@ -334,8 +335,9 @@
         ServerConfiguration bookieConf = bsConfs.get(bkIndex);
         BookieServer bk = bs.get(bkIndex);
         bookieConf.setReadOnlyModeEnabled(true);
-        bk.getBookie().getStateManager().doTransitionToReadOnlyMode();
-        bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS);
+
+        ((BookieImpl) bk.getBookie()).getStateManager().doTransitionToReadOnlyMode();
+        bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS);
 
         // grace period for publishing the bk-ledger
         LOG.debug("Waiting for Auditor to finish ledger check.");
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 8ea6636..a1b6148 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
@@ -49,6 +49,7 @@
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieAccessor;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.IndexPersistenceMgr;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException;
@@ -151,11 +152,11 @@
         }
         lh.close();
 
-        BookieAccessor.forceFlush(bs.get(0).getBookie());
+        BookieAccessor.forceFlush((BookieImpl) bs.get(0).getBookie());
 
 
         File ledgerDir = bsConfs.get(0).getLedgerDirs()[0];
-        ledgerDir = Bookie.getCurrentDirectory(ledgerDir);
+        ledgerDir = BookieImpl.getCurrentDirectory(ledgerDir);
         // corrupt of entryLogs
         File[] entryLogs = ledgerDir.listFiles(new FilenameFilter() {
                 public boolean accept(File dir, String name) {
@@ -207,10 +208,10 @@
         }
         lh.close();
 
-        BookieAccessor.forceFlush(bs.get(0).getBookie());
+        BookieAccessor.forceFlush((BookieImpl) bs.get(0).getBookie());
 
         File ledgerDir = bsConfs.get(0).getLedgerDirs()[0];
-        ledgerDir = Bookie.getCurrentDirectory(ledgerDir);
+        ledgerDir = BookieImpl.getCurrentDirectory(ledgerDir);
 
         // corrupt of entryLogs
         File index = new File(ledgerDir, IndexPersistenceMgr.getLedgerName(ledgerToCorrupt));
@@ -273,7 +274,7 @@
         final AtomicInteger numReads = new AtomicInteger(0);
         ServerConfiguration conf = killBookie(0);
 
-        Bookie deadBookie = new Bookie(conf) {
+        Bookie deadBookie = new BookieImpl(conf) {
             @Override
             public ByteBuf readEntry(long ledgerId, long entryId)
                     throws IOException, NoLedgerException {
@@ -338,7 +339,7 @@
         }
 
         try (final Auditor auditor = new Auditor(
-                Bookie.getBookieId(bsConfs.get(0)).toString(),
+                BookieImpl.getBookieId(bsConfs.get(0)).toString(),
                 bsConfs.get(0), NullStatsLogger.INSTANCE)) {
             final AtomicBoolean exceptionCaught = new AtomicBoolean(false);
             final CountDownLatch latch = new CountDownLatch(1);
@@ -401,7 +402,8 @@
         servConf.setAuditorPeriodicCheckInterval(auditorPeriodicCheckInterval);
         servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0);
         servConf.setAuditorPeriodicBookieCheckInterval(0);
-        final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false,
+
+        final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false,
                 statsLogger);
         CountDownLatch latch = auditor.getLatch();
         assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 0, checkAllLedgersStatsLogger.getSuccessCount());
@@ -494,7 +496,8 @@
         servConf.setAuditorPeriodicPlacementPolicyCheckInterval(auditorPeriodicPlacementPolicyCheckInterval);
         servConf.setAuditorPeriodicCheckInterval(0);
         servConf.setAuditorPeriodicBookieCheckInterval(0);
-        final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false,
+
+        final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false,
                 statsLogger);
         CountDownLatch latch = auditor.getLatch();
         assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount());
@@ -598,7 +601,7 @@
         servConf.setAuditorPeriodicReplicasCheckInterval(auditorPeriodicReplicasCheckInterval);
         servConf.setAuditorPeriodicCheckInterval(0);
         servConf.setAuditorPeriodicBookieCheckInterval(0);
-        final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false,
+        final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false,
                 statsLogger);
         CountDownLatch latch = auditor.getLatch();
         assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount());
@@ -716,7 +719,7 @@
 
         LOG.info("Killing bookie " + bs.get(bookieIdx).getBookieId());
         ServerConfiguration conf = killBookie(bookieIdx);
-        Bookie writeFailingBookie = new Bookie(conf) {
+        Bookie writeFailingBookie = new BookieImpl(conf) {
             @Override
             public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
                              Object ctx, byte[] masterKey)
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 21ca25e..9e134ee 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
@@ -24,7 +24,6 @@
 import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -33,8 +32,7 @@
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
 import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy;
 import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicy;
@@ -687,7 +685,7 @@
         TestOpStatsLogger placementPolicyCheckStatsLogger = (TestOpStatsLogger) statsLogger
                 .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME);
 
-        final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf,
+        final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf,
                 statsLogger);
         auditorRef.setValue(auditor);
         CountDownLatch latch = auditor.getLatch();
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 a17ecfa..4d3d7a3 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
@@ -23,7 +23,6 @@
 import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -35,9 +34,8 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
@@ -149,7 +147,7 @@
         TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger
                 .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME);
 
-        final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, true,
+        final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, true,
                 new TestBookKeeperAdmin(bkc, statsLogger, expectedReturnAvailabilityOfEntriesOfLedger,
                         errorReturnValueForGetAvailabilityOfEntriesOfLedger),
                 true, statsLogger);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java
index 23c7a7b..c222e59 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java
@@ -22,8 +22,7 @@
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver;
 import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
@@ -99,8 +98,9 @@
         ZKMetadataClientDriver zkMetadataClientDriver1 = startAutoRecoveryMain(main1);
         ZooKeeper zk1 = zkMetadataClientDriver1.getZk();
         Auditor auditor1 = main1.auditorElector.getAuditor();
+
         BookieId currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(0), zk1);
-        assertTrue("Current Auditor should be AR1", currentAuditor.equals(Bookie.getBookieId(bsConfs.get(0))));
+        assertTrue("Current Auditor should be AR1", currentAuditor.equals(BookieImpl.getBookieId(bsConfs.get(0))));
         assertTrue("Auditor of AR1 should be running", auditor1.isRunning());
 
         /*
@@ -116,7 +116,7 @@
          * auditors are not running.
          */
         assertTrue("Current Auditor should still be AR1",
-                currentAuditor.equals(Bookie.getBookieId(bsConfs.get(0))));
+                currentAuditor.equals(BookieImpl.getBookieId(bsConfs.get(0))));
         Auditor auditor2 = main2.auditorElector.getAuditor();
         Auditor auditor3 = main3.auditorElector.getAuditor();
         assertTrue("AR2's Auditor should not be running", (auditor2 == null || !auditor2.isRunning()));
@@ -154,7 +154,7 @@
          * the AR3 should be current auditor.
          */
         currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(2), zk3);
-        assertTrue("Current Auditor should be AR3", currentAuditor.equals(Bookie.getBookieId(bsConfs.get(2))));
+        assertTrue("Current Auditor should be AR3", currentAuditor.equals(BookieImpl.getBookieId(bsConfs.get(2))));
         auditor3 = main3.auditorElector.getAuditor();
         assertTrue("Auditor of AR3 should be running", auditor3.isRunning());
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
index fc2df20..c0f84d3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
@@ -24,7 +24,6 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
@@ -33,8 +32,7 @@
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.security.auth.login.Configuration;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.BKUnauthorizedAccessException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -86,7 +84,7 @@
         // this is just to calculate "localhostName" the same way the bookie does
         ServerConfiguration bookieConf = TestBKConfiguration.newServerConfiguration();
         bookieConf.setUseHostNameAsBookieID(true);
-        String localhostName = Bookie.getBookieAddress(bookieConf).getHostName();
+        String localhostName = BookieImpl.getBookieAddress(bookieConf).getHostName();
 
         String principalServerNoRealm = non_default_sasl_service_name + "/" + localhostName;
         String principalServer = non_default_sasl_service_name + "/" + localhostName + "@" + kdc.getRealm();
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 685544a..565e398 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
@@ -22,13 +22,10 @@
 package org.apache.bookkeeper.test;
 
 
-
 import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE;
 import static org.junit.Assert.assertTrue;
-
 import com.google.common.base.Stopwatch;
 import io.netty.buffer.ByteBufAllocator;
-
 import java.io.File;
 import java.io.IOException;
 import java.net.UnknownHostException;
@@ -43,9 +40,9 @@
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
@@ -420,7 +417,7 @@
     public void setBookieToReadOnly(BookieId addr) throws InterruptedException, UnknownHostException {
         for (BookieServer server : bs) {
             if (server.getBookieId().equals(addr)) {
-                server.getBookie().getStateManager().doTransitionToReadOnlyMode();
+                ((BookieImpl) server.getBookie()).getStateManager().doTransitionToReadOnlyMode();
                 break;
             }
         }
@@ -678,8 +675,9 @@
     protected BookieServer startBookie(ServerConfiguration conf)
             throws Exception {
         TestStatsProvider provider = new TestStatsProvider();
+
         BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), null);
-        BookieId address = Bookie.getBookieId(conf);
+        BookieId address = BookieImpl.getBookieId(conf);
         bsLoggers.put(address, provider);
 
         if (bkc == null) {
@@ -720,7 +718,7 @@
             }
         };
 
-        BookieId address = Bookie.getBookieId(conf);
+        BookieId address = BookieImpl.getBookieId(conf);
         if (bkc == null) {
             bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider());
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java
index e8ecba5..5432545 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java
@@ -22,10 +22,8 @@
 package org.apache.bookkeeper.test;
 
 import static org.junit.Assert.assertEquals;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -33,9 +31,9 @@
 import java.util.List;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.net.BookieId;
@@ -87,7 +85,7 @@
         conf.setMetadataServiceUri(null);
         conf.setJournalDirName(txnDir.getPath());
         conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
-        bookie = new Bookie(conf);
+        bookie = new BookieImpl(conf);
         bookie.start();
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ForceReadOnlyBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ForceReadOnlyBookieTest.java
index 28ce82a..614d3bc 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ForceReadOnlyBookieTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ForceReadOnlyBookieTest.java
@@ -27,6 +27,7 @@
 import java.util.Enumeration;
 
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -89,7 +90,7 @@
         LOG.info("successed read entry from ReadOnlyBookie");
 
         // test will not transfer to Writable mode.
-        LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
+        LedgerDirsManager ledgerDirsManager = ((BookieImpl) bookie).getLedgerDirsManager();
         ledgerDirsManager.addToWritableDirs(new File(ledgerDirs[0], "current"), true);
         assertTrue("Bookie should be running and in readonly mode",
                 bookie.isRunning() && bookie.isReadOnly());
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 8bfbd8e..1a9d7c1 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
@@ -28,6 +28,7 @@
 import java.util.Enumeration;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.client.BKException;
@@ -66,7 +67,7 @@
         assertEquals("Only one ledger dir should be present", 1,
                 ledgerDirs.length);
         Bookie bookie = bs.get(1).getBookie();
-        LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
+        LedgerDirsManager ledgerDirsManager = ((BookieImpl) bookie).getLedgerDirsManager();
 
         for (int i = 0; i < 10; i++) {
             ledger.addEntry("data".getBytes());
@@ -108,7 +109,7 @@
         File[] ledgerDirs = bsConfs.get(1).getLedgerDirs();
         assertEquals("Only one ledger dir should be present", 1,
                 ledgerDirs.length);
-        Bookie bookie = bs.get(1).getBookie();
+        BookieImpl bookie = (BookieImpl) bs.get(1).getBookie();
         LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
 
         for (int i = 0; i < 10; i++) {
@@ -127,7 +128,7 @@
             // Expected
         }
 
-        bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(1)))
+        bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(bsConfs.get(1)))
             .get(30, TimeUnit.SECONDS);
 
         LOG.info("bookie is running {}, readonly {}.", bookie.isRunning(), bookie.isReadOnly());
@@ -145,7 +146,7 @@
         // Now add the current ledger dir back to writable dirs list
         ledgerDirsManager.addToWritableDirs(testDir, true);
 
-        bkc.waitForWritableBookie(Bookie.getBookieId(bsConfs.get(1)))
+        bkc.waitForWritableBookie(BookieImpl.getBookieId(bsConfs.get(1)))
             .get(30, TimeUnit.SECONDS);
 
         LOG.info("bookie is running {}, readonly {}.", bookie.isRunning(), bookie.isReadOnly());
@@ -175,7 +176,7 @@
         File[] ledgerDirs = bsConfs.get(1).getLedgerDirs();
         assertEquals("Only one ledger dir should be present", 1,
                 ledgerDirs.length);
-        Bookie bookie = bs.get(1).getBookie();
+        BookieImpl bookie = (BookieImpl) bs.get(1).getBookie();
         LedgerHandle ledger = bkc.createLedger(2, 2, DigestType.MAC,
                 "".getBytes());
         LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
@@ -213,7 +214,7 @@
         File[] ledgerDirs = bsConfs.get(1).getLedgerDirs();
         assertEquals("Only one ledger dir should be present", 2,
                 ledgerDirs.length);
-        Bookie bookie = bs.get(1).getBookie();
+        BookieImpl bookie = (BookieImpl) bs.get(1).getBookie();
         LedgerHandle ledger = bkc.createLedger(2, 2, DigestType.MAC,
                 "".getBytes());
         LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
@@ -260,9 +261,9 @@
         killBookie(1);
         baseConf.setReadOnlyModeEnabled(true);
         startNewBookie();
-        bs.get(1).getBookie().getStateManager().doTransitionToReadOnlyMode();
+        bs.get(1).getBookie().getStateManager().transitionToReadOnlyMode().get();
         try {
-            bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(1)))
+            bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(bsConfs.get(1)))
                 .get(30, TimeUnit.SECONDS);
 
             bkc.createLedger(2, 2, DigestType.CRC32, "".getBytes());
@@ -288,7 +289,7 @@
         // Check new bookie with readonly mode enabled.
         File[] ledgerDirs = bsConfs.get(1).getLedgerDirs();
         assertEquals("Only one ledger dir should be present", 1, ledgerDirs.length);
-        Bookie bookie = bs.get(1).getBookie();
+        BookieImpl bookie = (BookieImpl) bs.get(1).getBookie();
         LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
 
         // Now add the current ledger dir to filled dirs list
@@ -307,4 +308,4 @@
             assertEquals("Entry should contain correct data", "data", new String(entry.getEntry()));
         }
     }
-}
\ No newline at end of file
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java
index 1dc30d4..7a15b5d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java
@@ -29,7 +29,7 @@
 import java.util.function.BooleanSupplier;
 
 import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.api.ReadHandle;
 
@@ -47,7 +47,7 @@
     public static boolean hasLogFiles(File ledgerDirectory, boolean partial, Integer... logsId) {
         boolean result = partial ? false : true;
         Set<Integer> logs = new HashSet<Integer>();
-        for (File file : Bookie.getCurrentDirectory(ledgerDirectory).listFiles()) {
+        for (File file : BookieImpl.getCurrentDirectory(ledgerDirectory).listFiles()) {
             if (file.isFile()) {
                 String name = file.getName();
                 if (!name.endsWith(".log")) {
diff --git a/stream/distributedlog/core/build.gradle b/stream/distributedlog/core/build.gradle
index 7ca40df..e4a65ea 100644
--- a/stream/distributedlog/core/build.gradle
+++ b/stream/distributedlog/core/build.gradle
@@ -40,6 +40,11 @@
     implementation depLibs.thrift
     implementation depLibs.zookeeper
 
+    testRuntimeOnly depLibs.metricsCore
+    testRuntimeOnly depLibs.snappy
+    testRuntimeOnly depLibs.commonsBeanutils
+
+
     testImplementation project(path: ':bookkeeper-common', configuration: 'testArtifacts')
     testImplementation project(path: ':stream:distributedlog:common', configuration: 'testArtifacts')
     testImplementation depLibs.junit
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java
index cc37087..ef6468e 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java
@@ -29,12 +29,11 @@
 import static org.powermock.api.mockito.PowerMockito.verifyNew;
 import static org.powermock.api.mockito.PowerMockito.when;
 import static org.powermock.api.mockito.PowerMockito.whenNew;
-
 import com.google.common.util.concurrent.UncheckedExecutionException;
 import java.util.Iterator;
 import java.util.Vector;
 import java.util.stream.LongStream;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerCache;
 import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
@@ -52,7 +51,7 @@
  * Unit test for {@link ConvertToDBStorageCommand}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ Bookie.class, ConvertToDBStorageCommand.class })
+@PrepareForTest({ BookieImpl.class, ConvertToDBStorageCommand.class })
 public class ConvertToDBStorageCommandTest extends BookieCommandTestBase {
 
     private InterleavedLedgerStorage interleavedLedgerStorage;
@@ -87,10 +86,10 @@
         when(dbStorage.addLedgerToIndex(anyLong(), anyBoolean(), eq(new byte[0]),
             any(LedgerCache.PageEntriesIterable.class))).thenReturn(1L);
 
-        PowerMockito.mockStatic(Bookie.class);
-        PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(conf), eq(interleavedLedgerStorage)))
+        PowerMockito.mockStatic(BookieImpl.class);
+        PowerMockito.when(BookieImpl.mountLedgerStorageOffline(eq(conf), eq(interleavedLedgerStorage)))
             .thenReturn(PowerMockito.mock(InterleavedLedgerStorage.class));
-        PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(conf), eq(dbStorage))).thenReturn(dbStorage);
+        PowerMockito.when(BookieImpl.mountLedgerStorageOffline(eq(conf), eq(dbStorage))).thenReturn(dbStorage);
     }
 
     private Iterator<Long> getLedgerId() {
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java
index 5a99ab9..b7df7f3 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java
@@ -25,8 +25,7 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 import static org.powermock.api.mockito.PowerMockito.verifyNew;
-
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.UpdateLedgerOp;
@@ -47,7 +46,7 @@
  * Unit test for {@link FlipBookieIdCommand}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ FlipBookieIdCommand.class, Bookie.class, UpdateLedgerOp.class })
+@PrepareForTest({ FlipBookieIdCommand.class, BookieImpl.class, UpdateLedgerOp.class })
 public class FlipBookieIdCommandTest extends BookieCommandTestBase {
 
     @Mock
@@ -85,8 +84,8 @@
                     .thenReturn(updateLedgerOp);
         PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class)
                     .withArguments(eq(conf)).thenReturn(serverConfiguration);
-        PowerMockito.mockStatic(Bookie.class);
-        PowerMockito.when(Bookie.getBookieId(eq(serverConfiguration))).thenReturn(bookieSocketAddress);
+        PowerMockito.mockStatic(BookieImpl.class);
+        PowerMockito.when(BookieImpl.getBookieId(eq(serverConfiguration))).thenReturn(bookieSocketAddress);
     }
 
     @Test
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java
index 1023458..881ad42 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java
@@ -23,9 +23,8 @@
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
-
 import java.util.Iterator;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerCache;
 import org.apache.bookkeeper.bookie.LedgerEntryPage;
@@ -46,7 +45,7 @@
  * Unit test for {@link LedgerCommand}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ DbLedgerStorage.class, SortedLedgerStorage.class, InterleavedLedgerStorage.class, Bookie.class,
+@PrepareForTest({ DbLedgerStorage.class, SortedLedgerStorage.class, InterleavedLedgerStorage.class, BookieImpl.class,
         LedgerStorage.class, LedgerCache.PageEntries.class, LedgerCache.PageEntriesIterable.class, LedgerCommand.class,
         LedgerCache.LedgerIndexMetadata.class })
 public class LedgerCommandTest extends BookieCommandTestBase {
@@ -79,8 +78,8 @@
         InterleavedLedgerStorage interleavedLedgerStorage = PowerMockito.mock(InterleavedLedgerStorage.class);
         PowerMockito.whenNew(InterleavedLedgerStorage.class).withNoArguments().thenReturn(interleavedLedgerStorage);
 
-        PowerMockito.mockStatic(Bookie.class);
-        PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(tConf), eq(interleavedLedgerStorage)))
+        PowerMockito.mockStatic(BookieImpl.class);
+        PowerMockito.when(BookieImpl.mountLedgerStorageOffline(eq(tConf), eq(interleavedLedgerStorage)))
                 .thenReturn(PowerMockito.mock(LedgerStorage.class));
 
         LedgerCache.PageEntries e = PowerMockito.mock(LedgerCache.PageEntries.class);
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java
index 3d32f9b..17ebf75 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java
@@ -22,10 +22,9 @@
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.verifyNew;
-
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
@@ -41,7 +40,7 @@
  * Unit test for {@link LocalConsistencyCheckCommand}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ LocalConsistencyCheckCommand.class, Bookie.class })
+@PrepareForTest({ LocalConsistencyCheckCommand.class, BookieImpl.class })
 public class LocalConsistencyCheckCommandTest extends BookieCommandTestBase {
 
     @Mock
@@ -60,8 +59,8 @@
 
         PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
         PowerMockito.whenNew(ServerConfiguration.class).withArguments(eq(conf)).thenReturn(serverConfiguration);
-        PowerMockito.mockStatic(Bookie.class);
-        PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(serverConfiguration), eq(null)))
+        PowerMockito.mockStatic(BookieImpl.class);
+        PowerMockito.when(BookieImpl.mountLedgerStorageOffline(eq(serverConfiguration), eq(null)))
                     .thenReturn(ledgerStorage);
         List<LedgerStorage.DetectedInconsistency> errors = new ArrayList<>();
         PowerMockito.when(ledgerStorage.localConsistencyCheck(eq(java.util.Optional.empty()))).thenReturn(errors);
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java
index f1455c5..054a309 100644
--- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java
@@ -19,13 +19,13 @@
 package org.apache.bookkeeper.tools.cli.commands.bookies;
 
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.mock;
 import static org.powermock.api.mockito.PowerMockito.verifyNew;
 import static org.powermock.api.mockito.PowerMockito.when;
-
 import java.util.UUID;
 import java.util.function.Function;
 import org.apache.bookkeeper.bookie.Bookie;
@@ -82,10 +82,9 @@
         PowerMockito.whenNew(ClientConfiguration.class).withArguments(eq(conf)).thenReturn(clientConfiguration);
         PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class)
                     .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin);
-        PowerMockito.mockStatic(Bookie.class);
-        PowerMockito.when(Bookie.getBookieId(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress);
+        PowerMockito.whenNew(BookieId.class).withArguments(anyString()).thenReturn(bookieSocketAddress);
+        conf.setBookieId(bookieSocketAddress.getId());
         PowerMockito.doNothing().when(bookKeeperAdmin).decommissionBookie(eq(bookieSocketAddress));
-
         RegistrationManager registrationManager = mock(RegistrationManager.class);
         PowerMockito.mockStatic(MetadataDrivers.class);
         PowerMockito.doAnswer(invocationOnMock -> {
@@ -96,7 +95,7 @@
                 any(Function.class));
 
         PowerMockito.mockStatic(Cookie.class);
-        PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress)))
+        when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress)))
                     .thenReturn(cookieVersioned);
         when(cookieVersioned.getValue()).thenReturn(cookie);
         when(cookieVersioned.getVersion()).thenReturn(version);
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 2d710ad..a530ff5 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
@@ -28,13 +28,13 @@
 import static org.powermock.api.mockito.PowerMockito.mock;
 import static org.powermock.api.mockito.PowerMockito.verifyNew;
 import static org.powermock.api.mockito.PowerMockito.when;
-
 import java.io.File;
 import java.io.IOException;
 import java.net.UnknownHostException;
 import java.util.function.Function;
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.Cookie;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
@@ -58,7 +58,8 @@
  * Unit test for {@link AdminCommand}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ AdminCommand.class, MetadataDrivers.class, Cookie.class, Bookie.class, RegistrationManager.class })
+@PrepareForTest({ AdminCommand.class, MetadataDrivers.class,
+        Cookie.class, Bookie.class, BookieImpl.class, RegistrationManager.class })
 public class AdminCommandTest extends BookieCommandTestBase {
 
     @Mock
@@ -98,6 +99,7 @@
         PowerMockito.mockStatic(Cookie.class);
         PowerMockito.mockStatic(MetadataDrivers.class);
         PowerMockito.mockStatic(Bookie.class);
+        PowerMockito.mockStatic(BookieImpl.class);
 
         mockUpdateBookieIdInCookie();
         mockVerifyCookie();
@@ -111,7 +113,7 @@
         File[] files = new File[1];
         files[0] = testDir.getRoot();
         testDir.newFile(BookKeeperConstants.VERSION_FILENAME);
-        PowerMockito.when(Bookie.getCurrentDirectories(any())).thenReturn(files);
+        PowerMockito.when(BookieImpl.getCurrentDirectories(any())).thenReturn(files);
     }
 
     private void mockUpdateBookieIdInCookie() throws Exception {
@@ -123,7 +125,7 @@
         }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class),
                 any(Function.class));
 
-        PowerMockito.when(Bookie.getBookieId(eq(serverConfiguration))).thenReturn(bookieSocketAddress);
+        serverConfiguration.setBookieId(bookieSocketAddress.getId());
         PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(serverConfiguration)))
                     .thenReturn(cookieVersioned);
         PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress)))
@@ -165,14 +167,11 @@
             return true;
         }).when(MetadataDrivers.class, "runFunctionWithMetadataBookieDriver", any(ServerConfiguration.class),
                 any(Function.class));
-        PowerMockito.doNothing()
-                    .when(Bookie.class, "checkEnvironmentWithStorageExpansion", any(ServerConfiguration.class),
-                          eq(metadataBookieDriver), any(), any());
     }
 
     private void mockListOrDeleteCookies() throws UnknownHostException {
 
-        when(Bookie.getBookieId(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress);
+        when(BookieImpl.getBookieId(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress);
     }
 
     @Test
diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java
index 1f38c42..67c3f74 100644
--- a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java
+++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java
@@ -17,7 +17,6 @@
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE;
-
 import com.beust.jcommander.Parameter;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
@@ -42,8 +41,8 @@
 import lombok.extern.slf4j.Slf4j;
 import org.HdrHistogram.Histogram;
 import org.HdrHistogram.Recorder;
-import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
 import org.apache.bookkeeper.bookie.Journal;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -477,7 +476,7 @@
         conf.setJournalSyncData(flags.journalSyncEnabled);
         conf.setLedgerDirNames(flags.journalDirs.toArray(new String[0]));
         conf.setStatsProviderClass(PrometheusMetricsProvider.class);
-        File[] currentDirs = Bookie.getCurrentDirectories(conf.getLedgerDirs());
+        File[] currentDirs = BookieImpl.getCurrentDirectories(conf.getLedgerDirs());
         for (File dir : currentDirs) {
             if (dir.mkdirs()) {
                 log.info("Successfully created dir {}", dir);