You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/02/19 08:18:42 UTC

[GitHub] sijie closed pull request #1128: BP-29 (task 2) : Make LedgerManagerFactory a pure interface

sijie closed pull request #1128: BP-29 (task 2) : Make LedgerManagerFactory a pure interface
URL: https://github.com/apache/bookkeeper/pull/1128
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 e091199b0..a3d124796 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
@@ -69,6 +69,7 @@
 import org.apache.bookkeeper.common.util.Watcher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -633,7 +634,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger)
         try {
             if (registrationManager != null) {
                 // current the registration manager is zookeeper only
-                ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(
+                ledgerManagerFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                     conf,
                     registrationManager.getLayoutManager());
                 LOG.info("instantiate ledger manager {}", ledgerManagerFactory.getClass().getName());
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 c9e4d9373..ea43a9646 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
@@ -87,6 +87,7 @@
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.discover.ZKRegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRange;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator;
@@ -891,7 +892,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
 
             try (RegistrationManager rm = RegistrationManager.instantiateRegistrationManager(bkConf)) {
                 try (LedgerManagerFactory mFactory =
-                         LedgerManagerFactory.newLedgerManagerFactory(bkConf, rm.getLayoutManager())) {
+                         AbstractZkLedgerManagerFactory.newLedgerManagerFactory(bkConf, rm.getLayoutManager())) {
                     LedgerUnderreplicationManager underreplicationManager =
                         mFactory.newLedgerUnderreplicationManager();
                     Iterator<Long> iter = underreplicationManager.listLedgersToRereplicate(predicate);
@@ -921,7 +922,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            try (LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+            try (LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                     bkConf,
                     RegistrationManager.instantiateRegistrationManager(bkConf).getLayoutManager())) {
                 try (LedgerManager m = mFactory.newLedgerManager()) {
@@ -1024,7 +1025,7 @@ public int runCmd(CommandLine cmdLine) throws Exception {
 
             try (RegistrationManager rm = RegistrationManager.instantiateRegistrationManager(bkConf)) {
                 try (LedgerManagerFactory mFactory =
-                         LedgerManagerFactory.newLedgerManagerFactory(bkConf, rm.getLayoutManager())){
+                         AbstractZkLedgerManagerFactory.newLedgerManagerFactory(bkConf, rm.getLayoutManager())){
                     try (LedgerManager m = mFactory.newLedgerManager()) {
                         ReadMetadataCallback cb = new ReadMetadataCallback(lid);
                         m.readLedgerMetadata(lid, cb);
@@ -1647,7 +1648,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
 
             try (RegistrationManager rm =
                      RegistrationManager.instantiateRegistrationManager(bkConf)){
-                try (LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+                try (LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                     bkConf,
                     rm.getLayoutManager())) {
                     LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index ca24a99bd..1d71584c5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -63,6 +63,7 @@
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.CleanupLedgerManager;
 import org.apache.bookkeeper.meta.LedgerIdGenerator;
 import org.apache.bookkeeper.meta.LedgerManager;
@@ -521,7 +522,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo
         // initialize ledger manager
         try {
             this.ledgerManagerFactory =
-                LedgerManagerFactory.newLedgerManagerFactory(conf, regClient.getLayoutManager());
+                AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, regClient.getLayoutManager());
         } catch (IOException | InterruptedException e) {
             throw e;
         }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
index b82f406d1..11452f25a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java
@@ -50,6 +50,7 @@
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LayoutManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.ZkLayoutManager;
@@ -491,7 +492,7 @@ public boolean initNewCluster() throws Exception {
         zk.create(zkAvailableBookiesPath, "".getBytes(UTF_8), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
         // creates the new layout and stores in zookeeper
-        LedgerManagerFactory.newLedgerManagerFactory(conf, layoutManager);
+        AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, layoutManager);
 
         // create INSTANCEID
         String instanceId = UUID.randomUUID().toString();
@@ -544,7 +545,8 @@ public boolean nukeExistingCluster() throws Exception {
             }
         }
 
-        LedgerManagerFactory ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(conf, layoutManager);
+        LedgerManagerFactory ledgerManagerFactory =
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, layoutManager);
         return ledgerManagerFactory.validateAndNukeExistingCluster(conf, layoutManager);
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java
index b8dcc0314..c6f2817d9 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java
@@ -21,6 +21,9 @@
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.meta.LayoutManager.LedgerLayoutExistsException;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.ZooKeeper;
@@ -30,11 +33,12 @@
  * methods such as format and validateAndNukeExistingCluster.
  */
 @Slf4j
-public abstract class AbstractZkLedgerManagerFactory extends LedgerManagerFactory {
+public abstract class AbstractZkLedgerManagerFactory implements LedgerManagerFactory {
+
     protected ZooKeeper zk;
 
     @Override
-    public void format(AbstractConfiguration conf, LayoutManager layoutManager)
+    public void format(AbstractConfiguration<?> conf, LayoutManager layoutManager)
             throws InterruptedException, KeeperException, IOException {
         try (AbstractZkLedgerManager ledgerManager = (AbstractZkLedgerManager) newLedgerManager()) {
             String ledgersRootPath = conf.getZkLedgersRootPath();
@@ -45,8 +49,17 @@ public void format(AbstractConfiguration conf, LayoutManager layoutManager)
                 }
             }
         }
-        // Delete and recreate the LAYOUT information.
-        super.format(conf, layoutManager);
+
+        Class<? extends LedgerManagerFactory> factoryClass;
+        try {
+            factoryClass = conf.getLedgerManagerFactoryClass();
+        } catch (ConfigurationException e) {
+            throw new IOException("Failed to get ledger manager factory class from configuration : ", e);
+        }
+
+        layoutManager.deleteLedgerLayout();
+        // Create new layout information again.
+        createNewLMFactory(conf, layoutManager, factoryClass);
     }
 
     @Override
@@ -92,4 +105,153 @@ public boolean validateAndNukeExistingCluster(AbstractConfiguration<?> conf, Lay
                 zkServers, zkLedgersRootPath);
         return true;
     }
+
+    // v1 layout
+    static final int V1 = 1;
+
+    /**
+     * Create new Ledger Manager Factory.
+     *
+     * @param conf
+     *          Configuration Object.
+     * @param layoutManager
+     *          layout manager
+     * @return new ledger manager factory
+     * @throws IOException
+     */
+    @SuppressWarnings("deprecation")
+    public static LedgerManagerFactory newLedgerManagerFactory(
+        final AbstractConfiguration<?> conf, LayoutManager layoutManager)
+            throws IOException, InterruptedException {
+        Class<? extends LedgerManagerFactory> factoryClass;
+        try {
+            factoryClass = conf.getLedgerManagerFactoryClass();
+        } catch (Exception e) {
+            throw new IOException("Failed to get ledger manager factory class from configuration : ", e);
+        }
+        String ledgerRootPath = conf.getZkLedgersRootPath();
+
+        if (null == ledgerRootPath || ledgerRootPath.length() == 0) {
+            throw new IOException("Empty Ledger Root Path.");
+        }
+
+        // if layoutManager is null, return the default ledger manager
+        if (layoutManager == null) {
+            return new FlatLedgerManagerFactory()
+                   .initialize(conf, null, FlatLedgerManagerFactory.CUR_VERSION);
+        }
+
+        LedgerManagerFactory lmFactory;
+
+        // check that the configured ledger manager is
+        // compatible with the existing layout
+        LedgerLayout layout = layoutManager.readLedgerLayout();
+
+        if (layout == null) { // no existing layout
+            lmFactory = createNewLMFactory(conf, layoutManager, factoryClass);
+            return lmFactory
+                    .initialize(conf, layoutManager, lmFactory.getCurrentVersion());
+        }
+        if (log.isDebugEnabled()) {
+            log.debug("read ledger layout {}", layout);
+        }
+
+        // there is existing layout, we need to look into the layout.
+        // handle pre V2 layout
+        if (layout.getLayoutFormatVersion() <= V1) {
+            // pre V2 layout we use type of ledger manager
+            String lmType = conf.getLedgerManagerType();
+            if (lmType != null && !layout.getManagerFactoryClass().equals(lmType)) {
+                throw new IOException("Configured layout " + lmType
+                        + " does not match existing layout "  + layout.getManagerFactoryClass());
+            }
+
+            // create the ledger manager
+            if (FlatLedgerManagerFactory.NAME.equals(layout.getManagerFactoryClass())) {
+                lmFactory = new FlatLedgerManagerFactory();
+            } else if (HierarchicalLedgerManagerFactory.NAME.equals(layout.getManagerFactoryClass())) {
+                lmFactory = new HierarchicalLedgerManagerFactory();
+            } else {
+                throw new IOException("Unknown ledger manager type: " + lmType);
+            }
+            return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion());
+        }
+
+        // handle V2 layout case
+        if (factoryClass != null && !layout.getManagerFactoryClass().equals(factoryClass.getName())
+                && conf.getProperty(AbstractConfiguration.LEDGER_MANAGER_FACTORY_DISABLE_CLASS_CHECK) == null) {
+                // Disable should ONLY happen during compatibility testing.
+
+            throw new IOException("Configured layout " + factoryClass.getName()
+                                + " does not match existing layout "  + layout.getManagerFactoryClass());
+        }
+        if (factoryClass == null) {
+            // no factory specified in configuration
+            try {
+                Class<?> theCls = Class.forName(layout.getManagerFactoryClass());
+                if (!LedgerManagerFactory.class.isAssignableFrom(theCls)) {
+                    throw new IOException("Wrong ledger manager factory " + layout.getManagerFactoryClass());
+                }
+                factoryClass = theCls.asSubclass(LedgerManagerFactory.class);
+            } catch (ClassNotFoundException cnfe) {
+                throw new IOException("Failed to instantiate ledger manager factory "
+                        + layout.getManagerFactoryClass());
+            }
+        }
+        // instantiate a factory
+        lmFactory = ReflectionUtils.newInstance(factoryClass);
+        return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion());
+    }
+
+    /**
+     * Creates the new layout and stores in zookeeper and returns the
+     * LedgerManagerFactory instance.
+     */
+    @SuppressWarnings("deprecation")
+    protected static LedgerManagerFactory createNewLMFactory(
+            final AbstractConfiguration conf, final LayoutManager layoutManager,
+            Class<? extends LedgerManagerFactory> factoryClass)
+            throws IOException, InterruptedException {
+
+        LedgerManagerFactory lmFactory;
+        LedgerLayout layout;
+        // use default ledger manager factory if no one provided
+        if (factoryClass == null) {
+            // for backward compatibility, check manager type
+            String lmType = conf.getLedgerManagerType();
+            if (lmType == null) {
+                factoryClass = HierarchicalLedgerManagerFactory.class;
+            } else {
+                if (FlatLedgerManagerFactory.NAME.equals(lmType)) {
+                    factoryClass = FlatLedgerManagerFactory.class;
+                } else if (HierarchicalLedgerManagerFactory.NAME.equals(lmType)) {
+                    factoryClass = HierarchicalLedgerManagerFactory.class;
+                } else if (LongHierarchicalLedgerManagerFactory.NAME.equals(lmType)) {
+                    factoryClass = LongHierarchicalLedgerManagerFactory.class;
+                } else {
+                    throw new IOException("Unknown ledger manager type: "
+                            + lmType);
+                }
+            }
+        }
+
+        lmFactory = ReflectionUtils.newInstance(factoryClass);
+
+        layout = new LedgerLayout(factoryClass.getName(),
+                lmFactory.getCurrentVersion());
+        try {
+            layoutManager.storeLedgerLayout(layout);
+        } catch (LedgerLayoutExistsException e) {
+            LedgerLayout layout2 = layoutManager.readLedgerLayout();
+            if (!layout2.equals(layout)) {
+                throw new IOException(
+                        "Contention writing to layout to zookeeper, "
+                                + " other layout " + layout2
+                                + " is incompatible with our " + "layout "
+                                + layout);
+            }
+        }
+        return lmFactory;
+    }
+
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
index e41554b7a..8568617e0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
@@ -19,29 +19,20 @@
 
 import java.io.IOException;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
-import org.apache.bookkeeper.meta.LayoutManager.LedgerLayoutExistsException;
 import org.apache.bookkeeper.replication.ReplicationException;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.ConfigurationException;
 import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A factory for creating ledger managers.
  */
-public abstract class LedgerManagerFactory implements AutoCloseable {
-
-    static final Logger LOG = LoggerFactory.getLogger(LedgerManagerFactory.class);
-    // v1 layout
-    static final int V1 = 1;
+public interface LedgerManagerFactory extends AutoCloseable {
 
     /**
      * Return current factory version.
      *
      * @return current version used by factory.
      */
-    public abstract int getCurrentVersion();
+    int getCurrentVersion();
 
     /**
      * Initialize a factory.
@@ -55,10 +46,10 @@
      * @return ledger manager factory instance
      * @throws IOException when fail to initialize the factory.
      */
-    public abstract LedgerManagerFactory initialize(AbstractConfiguration conf,
-                                                    LayoutManager layoutManager,
-                                                    int factoryVersion)
-    throws IOException;
+    LedgerManagerFactory initialize(AbstractConfiguration conf,
+                                    LayoutManager layoutManager,
+                                    int factoryVersion)
+        throws IOException;
 
     /**
      * Uninitialize the factory.
@@ -66,8 +57,7 @@ public abstract LedgerManagerFactory initialize(AbstractConfiguration conf,
      * @throws IOException when fail to uninitialize the factory.
      */
     @Override
-    public void close() throws IOException {
-    }
+    void close() throws IOException;
 
     /**
      * Return the ledger id generator, which is used for global unique ledger id
@@ -75,7 +65,7 @@ public void close() throws IOException {
      *
      * @return ledger id generator.
      */
-    public abstract LedgerIdGenerator newLedgerIdGenerator();
+    LedgerIdGenerator newLedgerIdGenerator();
 
     /**
      * return ledger manager for client-side to manage ledger metadata.
@@ -83,7 +73,7 @@ public void close() throws IOException {
      * @return ledger manager
      * @see LedgerManager
      */
-    public abstract LedgerManager newLedgerManager();
+    LedgerManager newLedgerManager();
 
     /**
      * Return a ledger underreplication manager, which is used to
@@ -93,154 +83,9 @@ public void close() throws IOException {
      * @return ledger underreplication manager
      * @see LedgerUnderreplicationManager
      */
-    public abstract LedgerUnderreplicationManager newLedgerUnderreplicationManager()
+    LedgerUnderreplicationManager newLedgerUnderreplicationManager()
             throws KeeperException, InterruptedException, ReplicationException.CompatibilityException;
 
-    /**
-     * Create new Ledger Manager Factory.
-     *
-     * @param conf
-     *          Configuration Object.
-     * @param layoutManager
-     *          layout manager
-     * @return new ledger manager factory
-     * @throws IOException
-     */
-    @SuppressWarnings("deprecation")
-    public static LedgerManagerFactory newLedgerManagerFactory(
-        final AbstractConfiguration<?> conf, LayoutManager layoutManager)
-            throws IOException, InterruptedException {
-        Class<? extends LedgerManagerFactory> factoryClass;
-        try {
-            factoryClass = conf.getLedgerManagerFactoryClass();
-        } catch (Exception e) {
-            throw new IOException("Failed to get ledger manager factory class from configuration : ", e);
-        }
-        String ledgerRootPath = conf.getZkLedgersRootPath();
-
-        if (null == ledgerRootPath || ledgerRootPath.length() == 0) {
-            throw new IOException("Empty Ledger Root Path.");
-        }
-
-        // if layoutManager is null, return the default ledger manager
-        if (layoutManager == null) {
-            return new FlatLedgerManagerFactory()
-                   .initialize(conf, null, FlatLedgerManagerFactory.CUR_VERSION);
-        }
-
-        LedgerManagerFactory lmFactory;
-
-        // check that the configured ledger manager is
-        // compatible with the existing layout
-        LedgerLayout layout = layoutManager.readLedgerLayout();
-
-        if (layout == null) { // no existing layout
-            lmFactory = createNewLMFactory(conf, layoutManager, factoryClass);
-            return lmFactory
-                    .initialize(conf, layoutManager, lmFactory.getCurrentVersion());
-        }
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("read ledger layout {}", layout);
-        }
-
-        // there is existing layout, we need to look into the layout.
-        // handle pre V2 layout
-        if (layout.getLayoutFormatVersion() <= V1) {
-            // pre V2 layout we use type of ledger manager
-            String lmType = conf.getLedgerManagerType();
-            if (lmType != null && !layout.getManagerFactoryClass().equals(lmType)) {
-                throw new IOException("Configured layout " + lmType
-                        + " does not match existing layout "  + layout.getManagerFactoryClass());
-            }
-
-            // create the ledger manager
-            if (FlatLedgerManagerFactory.NAME.equals(layout.getManagerFactoryClass())) {
-                lmFactory = new FlatLedgerManagerFactory();
-            } else if (HierarchicalLedgerManagerFactory.NAME.equals(layout.getManagerFactoryClass())) {
-                lmFactory = new HierarchicalLedgerManagerFactory();
-            } else {
-                throw new IOException("Unknown ledger manager type: " + lmType);
-            }
-            return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion());
-        }
-
-        // handle V2 layout case
-        if (factoryClass != null && !layout.getManagerFactoryClass().equals(factoryClass.getName())
-                && conf.getProperty(AbstractConfiguration.LEDGER_MANAGER_FACTORY_DISABLE_CLASS_CHECK) == null) {
-                // Disable should ONLY happen during compatibility testing.
-
-            throw new IOException("Configured layout " + factoryClass.getName()
-                                + " does not match existing layout "  + layout.getManagerFactoryClass());
-        }
-        if (factoryClass == null) {
-            // no factory specified in configuration
-            try {
-                Class<?> theCls = Class.forName(layout.getManagerFactoryClass());
-                if (!LedgerManagerFactory.class.isAssignableFrom(theCls)) {
-                    throw new IOException("Wrong ledger manager factory " + layout.getManagerFactoryClass());
-                }
-                factoryClass = theCls.asSubclass(LedgerManagerFactory.class);
-            } catch (ClassNotFoundException cnfe) {
-                throw new IOException("Failed to instantiate ledger manager factory "
-                        + layout.getManagerFactoryClass());
-            }
-        }
-        // instantiate a factory
-        lmFactory = ReflectionUtils.newInstance(factoryClass);
-        return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion());
-    }
-
-    /**
-     * Creates the new layout and stores in zookeeper and returns the
-     * LedgerManagerFactory instance.
-     */
-    @SuppressWarnings("deprecation")
-    private static LedgerManagerFactory createNewLMFactory(
-            final AbstractConfiguration conf, final LayoutManager layoutManager,
-            Class<? extends LedgerManagerFactory> factoryClass)
-            throws IOException, InterruptedException {
-
-        LedgerManagerFactory lmFactory;
-        LedgerLayout layout;
-        // use default ledger manager factory if no one provided
-        if (factoryClass == null) {
-            // for backward compatibility, check manager type
-            String lmType = conf.getLedgerManagerType();
-            if (lmType == null) {
-                factoryClass = HierarchicalLedgerManagerFactory.class;
-            } else {
-                if (FlatLedgerManagerFactory.NAME.equals(lmType)) {
-                    factoryClass = FlatLedgerManagerFactory.class;
-                } else if (HierarchicalLedgerManagerFactory.NAME.equals(lmType)) {
-                    factoryClass = HierarchicalLedgerManagerFactory.class;
-                } else if (LongHierarchicalLedgerManagerFactory.NAME.equals(lmType)) {
-                    factoryClass = LongHierarchicalLedgerManagerFactory.class;
-                } else {
-                    throw new IOException("Unknown ledger manager type: "
-                            + lmType);
-                }
-            }
-        }
-
-        lmFactory = ReflectionUtils.newInstance(factoryClass);
-
-        layout = new LedgerLayout(factoryClass.getName(),
-                lmFactory.getCurrentVersion());
-        try {
-            layoutManager.storeLedgerLayout(layout);
-        } catch (LedgerLayoutExistsException e) {
-            LedgerLayout layout2 = layoutManager.readLedgerLayout();
-            if (!layout2.equals(layout)) {
-                throw new IOException(
-                        "Contention writing to layout to zookeeper, "
-                                + " other layout " + layout2
-                                + " is incompatible with our " + "layout "
-                                + layout);
-            }
-        }
-        return lmFactory;
-    }
-
     /**
      * Format the ledger metadata for LedgerManager.
      *
@@ -249,20 +94,8 @@ private static LedgerManagerFactory createNewLMFactory(
      * @param lm
      *            Layout manager
      */
-    public void format(final AbstractConfiguration<?> conf, final LayoutManager lm)
-            throws InterruptedException, KeeperException, IOException {
-
-        Class<? extends LedgerManagerFactory> factoryClass;
-        try {
-            factoryClass = conf.getLedgerManagerFactoryClass();
-        } catch (ConfigurationException e) {
-            throw new IOException("Failed to get ledger manager factory class from configuration : ", e);
-        }
-
-        lm.deleteLedgerLayout();
-        // Create new layout information again.
-        createNewLMFactory(conf, lm, factoryClass);
-    }
+    void format(final AbstractConfiguration<?> conf, final LayoutManager lm)
+            throws InterruptedException, KeeperException, IOException;
 
     /**
      * This method makes sure there are no unexpected znodes under ledgersRootPath
@@ -277,6 +110,6 @@ public void format(final AbstractConfiguration<?> conf, final LayoutManager lm)
      * @throws KeeperException
      * @throws InterruptedException
      */
-    public abstract boolean validateAndNukeExistingCluster(AbstractConfiguration<?> conf, LayoutManager lm)
+    boolean validateAndNukeExistingCluster(AbstractConfiguration<?> conf, LayoutManager lm)
             throws InterruptedException, KeeperException, IOException;
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
index cb8e8c382..1d189c9ea 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
@@ -63,6 +63,7 @@
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
@@ -73,9 +74,14 @@
 
 /**
  * MetaStore Based Ledger Manager Factory.
+ *
+ * <p>MSLedgerManagerFactory is a legacy abstraction that mixing zookeeper with a metadata store
+ * interface. It is not used by any production systems. It should be deprecated soon.
+ *
+ * @deprecated since 4.7.0
  */
 @Slf4j
-public class MSLedgerManagerFactory extends LedgerManagerFactory {
+public class MSLedgerManagerFactory extends AbstractZkLedgerManagerFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(MSLedgerManagerFactory.class);
 
@@ -87,7 +93,6 @@
     public static final String META_FIELD = ".META";
 
     AbstractConfiguration conf;
-    ZooKeeper zk;
     MetaStore metastore;
 
     @Override
@@ -723,7 +728,7 @@ public void run() {
     }
 
     @Override
-    public void format(AbstractConfiguration conf, LayoutManager layoutManager)
+    public void format(AbstractConfiguration<?> conf, LayoutManager layoutManager)
         throws InterruptedException, KeeperException, IOException {
         MetastoreTable ledgerTable;
         try {
@@ -739,7 +744,16 @@ public void format(AbstractConfiguration conf, LayoutManager layoutManager)
         }
         LOG.info("Finished cleaning up table {}.", TABLE_NAME);
         // Delete and recreate the LAYOUT information.
-        super.format(conf, layoutManager);
+        Class<? extends LedgerManagerFactory> factoryClass;
+        try {
+            factoryClass = conf.getLedgerManagerFactoryClass();
+        } catch (ConfigurationException e) {
+            throw new IOException("Failed to get ledger manager factory class from configuration : ", e);
+        }
+
+        layoutManager.deleteLedgerLayout();
+        // Create new layout information again.
+        createNewLMFactory(conf, layoutManager, factoryClass);
     }
 
     @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
index 1a6034978..f3946225d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
@@ -29,6 +29,7 @@
 import lombok.SneakyThrows;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LayoutManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
@@ -185,7 +186,7 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory()
             throws MetadataException {
         if (null == lmFactory) {
             try {
-                lmFactory = LedgerManagerFactory.newLedgerManagerFactory(
+                lmFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                     conf,
                     layoutManager);
             } catch (IOException e) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java
index 3e9cfd337..b9cb2dad5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java
@@ -45,6 +45,7 @@
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
@@ -137,7 +138,7 @@ private void initialize(ServerConfiguration conf, ZooKeeper zkc)
                 clientConfiguration.getClientAuthProviderFactoryClass());
             this.bkc = new BookKeeper(clientConfiguration, zkc);
 
-            LedgerManagerFactory ledgerManagerFactory = LedgerManagerFactory
+            LedgerManagerFactory ledgerManagerFactory = AbstractZkLedgerManagerFactory
                     .newLedgerManagerFactory(
                         conf,
                         bkc.getRegClient().getLayoutManager());
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
index 65c0c2ce2..67eda3972 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
@@ -48,6 +48,7 @@
 import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -131,7 +132,7 @@ public ReplicationWorker(final ZooKeeper zkc,
         } catch (BKException e) {
             throw new IOException("Failed to instantiate replication worker", e);
         }
-        LedgerManagerFactory mFactory = LedgerManagerFactory
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory
                 .newLedgerManagerFactory(
                     this.conf,
                     bkc.getRegClient().getLayoutManager());
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 cbf844f26..a950ca715 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
@@ -55,8 +55,8 @@
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
@@ -252,7 +252,7 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact)
         InterleavedLedgerStorage storage = new InterleavedLedgerStorage();
         storage.initialize(
             conf,
-            LedgerManagerFactory
+            AbstractZkLedgerManagerFactory
                 .newLedgerManagerFactory(
                     conf,
                     RegistrationManager.instantiateRegistrationManager(conf).getLayoutManager())
@@ -949,7 +949,7 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact)
         TestStatsProvider stats = new TestStatsProvider();
         storage.initialize(
             conf,
-            LedgerManagerFactory
+            AbstractZkLedgerManagerFactory
                 .newLedgerManagerFactory(
                     conf,
                     RegistrationManager.instantiateRegistrationManager(conf).getLayoutManager())
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 b6282443f..663f0f8cf 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
@@ -41,6 +41,7 @@
 import org.apache.bookkeeper.bookie.FileInfoBackingCache.CachedFileInfo;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -88,7 +89,7 @@ public void setUp() throws Exception {
         bookie = new Bookie(conf);
 
         ledgerManagerFactory =
-            LedgerManagerFactory.newLedgerManagerFactory(conf, null);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, null);
         activeLedgers = new SnapshotMap<Long, Boolean>();
         ledgerCache = ((InterleavedLedgerStorage) bookie.ledgerStorage).ledgerCache;
     }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
index 5d7be5dcc..b841baae1 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
@@ -36,7 +36,6 @@
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory;
-import org.apache.bookkeeper.meta.MSLedgerManagerFactory;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
@@ -77,7 +76,7 @@ public TestWatchEnsembleChange(Class<? extends LedgerManagerFactory> lmFactoryCl
                 { org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class },
                 { HierarchicalLedgerManagerFactory.class },
                 { LongHierarchicalLedgerManagerFactory.class },
-                { MSLedgerManagerFactory.class },
+                { org.apache.bookkeeper.meta.MSLedgerManagerFactory.class },
         });
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
index 7d421aa97..c82198d7a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
@@ -479,6 +479,7 @@ public void checkConcurrentModifications() throws Throwable {
         }
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testLedgerParentNode() throws Throwable {
         /*
@@ -503,6 +504,7 @@ public void testLedgerParentNode() throws Throwable {
         }
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testLedgerManagerFormat() throws Throwable {
         /*
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
index 78d3b08b1..deecc197a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
@@ -106,7 +106,7 @@ public LedgerIdGenerator getLedgerIdGenerator() throws IOException {
     public void setUp() throws Exception {
         super.setUp();
         baseConf.setZkServers(zkUtil.getZooKeeperConnectString());
-        ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        ledgerManagerFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             baseConf,
             RegistrationManager
                 .instantiateRegistrationManager(baseConf).getLayoutManager());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java
index a466590e3..6a02e2c6b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java
@@ -84,7 +84,7 @@ public void testBadConf() throws Exception {
             conf.getZkLedgersRootPath(),
             ZkUtils.getACLs(conf));
 
-        LedgerManagerFactory m = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory m = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             conf,
             zkLayoutManager);
         assertTrue("Ledger manager is unexpected type",
@@ -94,7 +94,7 @@ public void testBadConf() throws Exception {
         // mismatching conf
         conf.setLedgerManagerFactoryClass(LongHierarchicalLedgerManagerFactory.class);
         try {
-            LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
         } catch (Exception e) {
             LOG.error("Received exception", e);
@@ -110,7 +110,7 @@ public void testBadConf() throws Exception {
 
         conf.setLedgerManagerFactoryClassName("DoesNotExist");
         try {
-            LedgerManagerFactory f = LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
         } catch (Exception e) {
             LOG.error("Received exception", e);
@@ -142,7 +142,7 @@ public void testBadConfV1() throws Exception {
             conf.getZkLedgersRootPath(),
             ZkUtils.getACLs(conf));
 
-        LedgerManagerFactory m = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory m = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             conf,
             zkLayoutManager);
 
@@ -152,7 +152,7 @@ public void testBadConfV1() throws Exception {
 
         // v2 setting doesn't effect v1
         conf.setLedgerManagerFactoryClass(HierarchicalLedgerManagerFactory.class);
-        m = LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
+        m = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
         assertTrue("Ledger manager is unexpected type",
                    (m instanceof HierarchicalLedgerManagerFactory));
         m.close();
@@ -160,7 +160,7 @@ public void testBadConfV1() throws Exception {
         // mismatching conf
         conf.setLedgerManagerType(LongHierarchicalLedgerManagerFactory.NAME);
         try {
-            LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
         } catch (Exception e) {
             LOG.error("Received exception", e);
@@ -189,7 +189,7 @@ public void testBadZkContents() throws Exception {
         zkLayoutManager.storeLedgerLayout(layout);
 
         try {
-            LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
         } catch (Exception e) {
             LOG.error("Received exception", e);
@@ -209,7 +209,7 @@ public void testBadZkContents() throws Exception {
         zkLayoutManager1.storeLedgerLayout(layout1);
 
         try {
-            LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager1);
+            AbstractZkLedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager1);
             fail("Shouldn't reach here");
         } catch (Exception e) {
             LOG.error("Received exception", e);
@@ -243,7 +243,7 @@ public void run() {
 
             try {
                 barrier.await();
-                LedgerManagerFactory factory = LedgerManagerFactory.newLedgerManagerFactory(
+                LedgerManagerFactory factory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                     conf,
                     RegistrationManager
                         .instantiateRegistrationManager(new ServerConfiguration(conf)).getLayoutManager());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseTest.java
index 60e768e2f..f1ed0c025 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseTest.java
@@ -37,6 +37,7 @@
 import java.net.URI;
 import java.util.Optional;
 import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -52,7 +53,7 @@
  * Unit test of {@link ZKMetadataDriverBase}.
  */
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({ ZKMetadataDriverBase.class, ZooKeeperClient.class, LedgerManagerFactory.class })
+@PrepareForTest({ ZKMetadataDriverBase.class, ZooKeeperClient.class, AbstractZkLedgerManagerFactory.class })
 public class ZKMetadataDriverBaseTest extends ZKMetadataDriverTestBase {
 
     private ZKMetadataDriverBase driver;
@@ -127,10 +128,10 @@ public void testInitializeExternalZooKeeper() throws Exception {
     public void testGetLedgerManagerFactory() throws Exception {
         driver.initialize(conf, NullStatsLogger.INSTANCE, Optional.empty());
 
-        mockStatic(LedgerManagerFactory.class);
+        mockStatic(AbstractZkLedgerManagerFactory.class);
         LedgerManagerFactory factory = mock(LedgerManagerFactory.class);
         PowerMockito.when(
-            LedgerManagerFactory.class,
+            AbstractZkLedgerManagerFactory.class,
             "newLedgerManagerFactory",
             same(conf),
             same(driver.layoutManager))
@@ -138,8 +139,8 @@ public void testGetLedgerManagerFactory() throws Exception {
 
         assertSame(factory, driver.getLedgerManagerFactory());
         assertSame(factory, driver.lmFactory);
-        verifyStatic(LedgerManagerFactory.class, times(1));
-        LedgerManagerFactory.newLedgerManagerFactory(
+        verifyStatic(AbstractZkLedgerManagerFactory.class, times(1));
+        AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             same(conf),
             same(driver.layoutManager));
 
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 bb56e7792..e0f2f9780 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
@@ -50,8 +50,8 @@
 import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
@@ -537,7 +537,7 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception {
             ensemble.add(new BookieSocketAddress("11.11.11.11:1111"));
             ensemble.add(new BookieSocketAddress("88.88.88.88:8888"));
             metadata.addEnsemble(0, ensemble);
-            LedgerManager ledgerManager = LedgerManagerFactory.newLedgerManagerFactory(
+            LedgerManager ledgerManager = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                 baseClientConf,
                 RegistrationManager
                     .instantiateRegistrationManager(new ServerConfiguration(baseClientConf))
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java
index bfd88e3db..919108432 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java
@@ -30,6 +30,7 @@
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
@@ -96,7 +97,7 @@ public void tearDown() throws Exception {
     @Test
     public void testPeriodicBookieCheckInterval() throws Exception {
         bsConfs.get(0).setZkServers(zkUtil.getZooKeeperConnectString());
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
 
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 35eb9585f..ddb163fbe 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
@@ -50,6 +50,7 @@
 import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -126,7 +127,7 @@ public void tearDown() throws Exception {
      */
     @Test
     public void testEntryLogCorruption() throws Exception {
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
         LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager();
@@ -177,7 +178,7 @@ public boolean accept(File dir, String name) {
      */
     @Test
     public void testIndexCorruption() throws Exception {
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
 
@@ -227,7 +228,7 @@ public void testIndexCorruption() throws Exception {
      */
     @Test
     public void testPeriodicCheckWhenDisabled() throws Exception {
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
         final LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager();
@@ -404,7 +405,7 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
      */
     @Test
     public void testFailedWriteRecovery() throws Exception {
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
                 bsConfs.get(0),
                 RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
         LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java
index 9b967e9e3..d4c918f2e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java
@@ -26,6 +26,7 @@
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -49,7 +50,7 @@ public AuditorRollingRestartTest() {
      */
     @Test
     public void testAuditingDuringRollingRestart() throws Exception {
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
index 7c758e4bf..281c3c79f 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
@@ -37,6 +37,7 @@
 import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LayoutManager;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
@@ -99,12 +100,12 @@ public void setUp() throws Exception {
         LayoutManager layoutManager = RegistrationManager
             .instantiateRegistrationManager(new ServerConfiguration(baseClientConf)).getLayoutManager();
 
-        mFactory = LedgerManagerFactory
+        mFactory = AbstractZkLedgerManagerFactory
             .newLedgerManagerFactory(
                 baseClientConf,
                 layoutManager);
         underReplicationManager = mFactory.newLedgerUnderreplicationManager();
-        LedgerManagerFactory newLedgerManagerFactory = LedgerManagerFactory
+        LedgerManagerFactory newLedgerManagerFactory = AbstractZkLedgerManagerFactory
             .newLedgerManagerFactory(
                 baseClientConf,
                 layoutManager);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java
index 43933fb33..6aef30d52 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java
@@ -33,9 +33,9 @@
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
-import org.apache.bookkeeper.meta.MSLedgerManagerFactory;
 import org.apache.bookkeeper.meta.ZkLayoutManager;
 import org.apache.bookkeeper.replication.ReplicationException.BKAuditException;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
@@ -91,7 +91,7 @@ public void setUp() throws Exception {
         entries = new ArrayList<byte[]>(); // initialize the entries list
         ledgerList = new ArrayList<Long>(3);
         // initialize ledger manager
-        newLedgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        newLedgerManagerFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             baseConf,
             new ZkLayoutManager(zkc, baseConf.getZkLedgersRootPath(), ZkUtils.getACLs(baseConf)));
 
@@ -144,12 +144,13 @@ public void testSimpleBookieLedgerMapping() throws Exception {
     /**
      * Verify ledger index with failed bookies and throws exception.
      */
+    @SuppressWarnings("deprecation")
     @Test
     public void testWithoutZookeeper() throws Exception {
         // This test case is for ledger metadata that stored in ZooKeeper. As
         // far as MSLedgerManagerFactory, ledger metadata are stored in other
         // storage. So this test is not suitable for MSLedgerManagerFactory.
-        if (newLedgerManagerFactory instanceof MSLedgerManagerFactory) {
+        if (newLedgerManagerFactory instanceof org.apache.bookkeeper.meta.MSLedgerManagerFactory) {
             return;
         }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java
index 322c6ea32..97b0d976a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java
@@ -45,6 +45,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.meta.ZkLayoutManager;
@@ -111,13 +112,13 @@ public void setupZooKeeper() throws Exception {
         urLedgerPath = basePath
                 + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH;
 
-        lmf1 = LedgerManagerFactory.newLedgerManagerFactory(
+        lmf1 = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             conf,
             new ZkLayoutManager(
                 zkc1,
                 conf.getZkLedgersRootPath(),
                 ZkUtils.getACLs(conf)));
-        lmf2 = LedgerManagerFactory.newLedgerManagerFactory(
+        lmf2 = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             conf,
             new ZkLayoutManager(
                 zkc2,
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
index b60d3f6f5..b1a5f0340 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
@@ -36,6 +36,7 @@
 import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -87,7 +88,7 @@ public void setUp() throws Exception {
         super.setUp();
         baseConf.setZkServers(zkUtil.getZooKeeperConnectString());
         // initialize urReplicationManager
-        mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             baseClientConf,
             RegistrationManager
                 .instantiateRegistrationManager(new ServerConfiguration(baseClientConf)).getLayoutManager());
@@ -416,7 +417,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR()
         baseConf.setOpenLedgerRereplicationGracePeriod("3000");
         ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
 
-        LedgerManagerFactory mFactory = LedgerManagerFactory
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory
             .newLedgerManagerFactory(
                 baseClientConf,
                 RegistrationManager
@@ -483,7 +484,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR()
         ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
 
         baseClientConf.setZkServers(zkUtil.getZooKeeperConnectString());
-        LedgerManagerFactory mFactory = LedgerManagerFactory
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory
             .newLedgerManagerFactory(
                 baseClientConf,
                 RegistrationManager
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
index 467d6f268..bc95fc225 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java
@@ -39,6 +39,7 @@
 import org.apache.bookkeeper.http.service.HttpEndpointService;
 import org.apache.bookkeeper.http.service.HttpServiceRequest;
 import org.apache.bookkeeper.http.service.HttpServiceResponse;
+import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
@@ -663,7 +664,7 @@ public void testListUnderReplicatedLedgerService() throws Exception {
 
         //2,  GET, should return success.
         // first put ledger into rereplicate. then use api to list ur ledger.
-        LedgerManagerFactory mFactory = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             bsConfs.get(0),
             RegistrationManager.instantiateRegistrationManager(bsConfs.get(0)).getLayoutManager());
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services