You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/02/19 08:18:35 UTC

[bookkeeper] branch master updated: BP-29 (task 2) : Make LedgerManagerFactory a pure interface

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 1329ec0  BP-29 (task 2) : Make LedgerManagerFactory a pure interface
1329ec0 is described below

commit 1329ec09f2024dc52d8b772e4ee70d9a9cdcfd0d
Author: Sijie Guo <si...@apache.org>
AuthorDate: Mon Feb 19 00:18:28 2018 -0800

    BP-29 (task 2) : Make LedgerManagerFactory a pure interface
    
    Descriptions of the changes in this PR:
    
        The logic in LedgerManagerFactory#newLedgerManagerFactory is effectively the logic works for zookeeper based implementation.
        In order to produce a clean metadata api, this change is moving this logic to AbstractZkLedgerManagerFactory.
    
    Master Issue: #1123
    
    This change is based on #1127. Only the second commit ce35173 is to review in this PR.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>
    
    This closes #1128 from sijie/make_ledger_manager_factory_interface
---
 .../java/org/apache/bookkeeper/bookie/Bookie.java  |   3 +-
 .../org/apache/bookkeeper/bookie/BookieShell.java  |   9 +-
 .../org/apache/bookkeeper/client/BookKeeper.java   |   3 +-
 .../bookkeeper/discover/ZKRegistrationManager.java |   6 +-
 .../meta/AbstractZkLedgerManagerFactory.java       | 170 +++++++++++++++++-
 .../bookkeeper/meta/LedgerManagerFactory.java      | 193 ++-------------------
 .../bookkeeper/meta/MSLedgerManagerFactory.java    |  22 ++-
 .../bookkeeper/meta/zk/ZKMetadataDriverBase.java   |   3 +-
 .../org/apache/bookkeeper/replication/Auditor.java |   3 +-
 .../bookkeeper/replication/ReplicationWorker.java  |   3 +-
 .../apache/bookkeeper/bookie/CompactionTest.java   |   6 +-
 .../apache/bookkeeper/bookie/LedgerCacheTest.java  |   3 +-
 .../bookkeeper/client/TestWatchEnsembleChange.java |   3 +-
 .../bookkeeper/meta/LedgerManagerIteratorTest.java |   2 +
 .../bookkeeper/meta/LedgerManagerTestCase.java     |   2 +-
 .../apache/bookkeeper/meta/TestLedgerManager.java  |  18 +-
 .../meta/zk/ZKMetadataDriverBaseTest.java          |  11 +-
 .../replication/AuditorLedgerCheckerTest.java      |   4 +-
 .../AuditorPeriodicBookieCheckTest.java            |   3 +-
 .../replication/AuditorPeriodicCheckTest.java      |   9 +-
 .../replication/AuditorRollingRestartTest.java     |   3 +-
 .../replication/BookieAutoRecoveryTest.java        |   5 +-
 .../replication/BookieLedgerIndexTest.java         |   7 +-
 .../TestLedgerUnderreplicationManager.java         |   5 +-
 .../replication/TestReplicationWorker.java         |   7 +-
 .../bookkeeper/server/http/TestHttpService.java    |   3 +-
 26 files changed, 267 insertions(+), 239 deletions(-)

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 e091199..a3d1247 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.bookie.LedgerDirsManager.NoWritableLedgerDirExcepti
 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 class Bookie extends BookieCriticalThread {
         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 c9e4d93..ea43a96 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.ClientConfiguration;
 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 @@ public class BookieShell implements Tool {
 
             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 @@ public class BookieShell implements Tool {
 
         @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 class BookieShell implements Tool {
 
             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 @@ public class BookieShell implements Tool {
 
             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 9eb9da9..f0fff6e 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.discover.ZKRegistrationClient;
 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;
@@ -524,7 +525,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
         // 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 b82f406..11452f2 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.client.BKException.MetaStoreException;
 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 class ZKRegistrationManager implements RegistrationManager {
         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 class ZKRegistrationManager implements RegistrationManager {
             }
         }
 
-        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 b8dcc03..c6f2817 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.io.IOException;
 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 @@ import org.apache.zookeeper.ZooKeeper;
  * 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 abstract class AbstractZkLedgerManagerFactory extends LedgerManagerFactor
                 }
             }
         }
-        // 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 abstract class AbstractZkLedgerManagerFactory extends LedgerManagerFactor
                 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 e41554b..8568617 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 @@ package org.apache.bookkeeper.meta;
 
 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 @@ public abstract class LedgerManagerFactory implements AutoCloseable {
      * @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 class LedgerManagerFactory implements AutoCloseable {
      * @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 abstract class LedgerManagerFactory implements AutoCloseable {
      *
      * @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 abstract class LedgerManagerFactory implements AutoCloseable {
      * @return ledger manager
      * @see LedgerManager
      */
-    public abstract LedgerManager newLedgerManager();
+    LedgerManager newLedgerManager();
 
     /**
      * Return a ledger underreplication manager, which is used to
@@ -93,155 +83,10 @@ public abstract class LedgerManagerFactory implements AutoCloseable {
      * @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.
      *
      * @param conf
@@ -249,20 +94,8 @@ public abstract class LedgerManagerFactory implements AutoCloseable {
      * @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 abstract class LedgerManagerFactory implements AutoCloseable {
      * @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 cb8e8c3..1d189c9 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.StringUtils;
 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 @@ import org.slf4j.LoggerFactory;
 
 /**
  * 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 class MSLedgerManagerFactory extends LedgerManagerFactory {
     public static final String META_FIELD = ".META";
 
     AbstractConfiguration conf;
-    ZooKeeper zk;
     MetaStore metastore;
 
     @Override
@@ -723,7 +728,7 @@ public class MSLedgerManagerFactory extends LedgerManagerFactory {
     }
 
     @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 class MSLedgerManagerFactory extends LedgerManagerFactory {
         }
         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 1a60349..f394622 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 java.util.Optional;
 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 class ZKMetadataDriverBase implements AutoCloseable {
             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 3e9cfd3..b9cb2da 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.LedgerFragment;
 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 @@ public class Auditor {
                 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 f6ed7d1..8318b2f 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.LedgerHandle;
 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 class ReplicationWorker implements Runnable {
         } 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 cbf844f..a950ca7 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.client.LedgerMetadata;
 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 abstract class CompactionTest extends BookKeeperClusterTestCase {
         InterleavedLedgerStorage storage = new InterleavedLedgerStorage();
         storage.initialize(
             conf,
-            LedgerManagerFactory
+            AbstractZkLedgerManagerFactory
                 .newLedgerManagerFactory(
                     conf,
                     RegistrationManager.instantiateRegistrationManager(conf).getLayoutManager())
@@ -949,7 +949,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase {
         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 b628244..663f0f8 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.Bookie.NoLedgerException;
 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 class LedgerCacheTest {
         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 5d7be5d..b841baa 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.LedgerIdGenerator;
 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 class TestWatchEnsembleChange extends BookKeeperClusterTestCase {
                 { 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 7d421aa..c82198d 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 class LedgerManagerIteratorTest extends LedgerManagerTestCase {
         }
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testLedgerParentNode() throws Throwable {
         /*
@@ -503,6 +504,7 @@ public class LedgerManagerIteratorTest extends LedgerManagerTestCase {
         }
     }
 
+    @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 78d3b08..deecc19 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 abstract class LedgerManagerTestCase extends BookKeeperClusterTestCase {
     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 a466590..6a02e2c 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 class TestLedgerManager extends BookKeeperClusterTestCase {
             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 class TestLedgerManager extends BookKeeperClusterTestCase {
         // 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 class TestLedgerManager extends BookKeeperClusterTestCase {
 
         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 class TestLedgerManager extends BookKeeperClusterTestCase {
             conf.getZkLedgersRootPath(),
             ZkUtils.getACLs(conf));
 
-        LedgerManagerFactory m = LedgerManagerFactory.newLedgerManagerFactory(
+        LedgerManagerFactory m = AbstractZkLedgerManagerFactory.newLedgerManagerFactory(
             conf,
             zkLayoutManager);
 
@@ -152,7 +152,7 @@ public class TestLedgerManager extends BookKeeperClusterTestCase {
 
         // 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 class TestLedgerManager extends BookKeeperClusterTestCase {
         // 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 class TestLedgerManager extends BookKeeperClusterTestCase {
         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 class TestLedgerManager extends BookKeeperClusterTestCase {
         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 class TestLedgerManager extends BookKeeperClusterTestCase {
 
             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 60e768e..f1ed0c0 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 static org.powermock.api.mockito.PowerMockito.verifyStatic;
 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 @@ import org.powermock.modules.junit4.PowerMockRunner;
  * 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 class ZKMetadataDriverBaseTest extends ZKMetadataDriverTestBase {
     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 class ZKMetadataDriverBaseTest extends ZKMetadataDriverTestBase {
 
         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 bb56e77..e0f2f97 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.LedgerHandle;
 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 class AuditorLedgerCheckerTest extends BookKeeperClusterTestCase {
             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 bfd88e3..9191084 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.client.LedgerMetadata;
 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 class AuditorPeriodicBookieCheckTest extends BookKeeperClusterTestCase {
     @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 35eb958..ddb163f 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.LedgerHandle;
 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 class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
      */
     @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 class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
      */
     @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 class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
      */
     @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 class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
      */
     @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 9b967e9..d4c918f 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.BookKeeper.DigestType;
 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 class AuditorRollingRestartTest extends BookKeeperClusterTestCase {
      */
     @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 7c758e4..281c3c7 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.LedgerHandle;
 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 class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
         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 43933fb..6aef30d 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 java.util.Set;
 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 class BookieLedgerIndexTest extends BookKeeperClusterTestCase {
         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 class BookieLedgerIndexTest extends BookKeeperClusterTestCase {
     /**
      * 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 322c6ea..97b0d97 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.AtomicBoolean;
 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 class TestLedgerUnderreplicationManager {
         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 b60d3f6..b1a5f03 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.LedgerHandle;
 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 class TestReplicationWorker extends BookKeeperClusterTestCase {
         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 class TestReplicationWorker extends BookKeeperClusterTestCase {
         baseConf.setOpenLedgerRereplicationGracePeriod("3000");
         ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
 
-        LedgerManagerFactory mFactory = LedgerManagerFactory
+        LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory
             .newLedgerManagerFactory(
                 baseClientConf,
                 RegistrationManager
@@ -483,7 +484,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
         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 467d6f2..bc95fc2 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.HttpServer;
 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 class TestHttpService extends BookKeeperClusterTestCase {
 
         //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());
 

-- 
To stop receiving notification emails like this one, please contact
sijie@apache.org.