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/03 17:31:14 UTC

[GitHub] sijie closed pull request #1115: Change default ledger manager factory from `Flat` to `Hierachical`

sijie closed pull request #1115: Change default ledger manager factory from `Flat` to `Hierachical`
URL: https://github.com/apache/bookkeeper/pull/1115
 
 
   

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/conf/bk_server.conf b/bookkeeper-server/conf/bk_server.conf
index 7c82de287..464ff4aa0 100755
--- a/bookkeeper-server/conf/bk_server.conf
+++ b/bookkeeper-server/conf/bk_server.conf
@@ -402,7 +402,7 @@ ledgerDirectories=/tmp/bk-data
 ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory
 
 # @Drepcated - `ledgerManagerType` is deprecated in favor of using `ledgerManagerFactoryClass`.
-# ledgerManagerType=flat
+# ledgerManagerType=hierarchical
 
 # Root Zookeeper path to store ledger metadata
 # This parameter is used by zookeeper-based ledger manager as a root znode to
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java
index 5e2ced5dc..12b5e189f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java
@@ -30,7 +30,10 @@
 
 /**
  * Flat Ledger Manager Factory.
+ *
+ * @deprecated since 4.7.0. The implementation will be still available but not recommended to use.
  */
+@Deprecated
 public class FlatLedgerManagerFactory extends AbstractZkLedgerManagerFactory {
 
     public static final String NAME = "flat";
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 62ff092b0..e41554b7a 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
@@ -106,6 +106,7 @@ public abstract LedgerUnderreplicationManager newLedgerUnderreplicationManager()
      * @return new ledger manager factory
      * @throws IOException
      */
+    @SuppressWarnings("deprecation")
     public static LedgerManagerFactory newLedgerManagerFactory(
         final AbstractConfiguration<?> conf, LayoutManager layoutManager)
             throws IOException, InterruptedException {
@@ -146,7 +147,6 @@ public static LedgerManagerFactory newLedgerManagerFactory(
         // handle pre V2 layout
         if (layout.getLayoutFormatVersion() <= V1) {
             // pre V2 layout we use type of ledger manager
-            @SuppressWarnings("deprecation")
             String lmType = conf.getLedgerManagerType();
             if (lmType != null && !layout.getManagerFactoryClass().equals(lmType)) {
                 throw new IOException("Configured layout " + lmType
@@ -194,6 +194,7 @@ public static LedgerManagerFactory newLedgerManagerFactory(
      * 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)
@@ -204,10 +205,9 @@ private static LedgerManagerFactory createNewLMFactory(
         // use default ledger manager factory if no one provided
         if (factoryClass == null) {
             // for backward compatibility, check manager type
-            @SuppressWarnings("deprecation")
             String lmType = conf.getLedgerManagerType();
             if (lmType == null) {
-                factoryClass = FlatLedgerManagerFactory.class;
+                factoryClass = HierarchicalLedgerManagerFactory.class;
             } else {
                 if (FlatLedgerManagerFactory.NAME.equals(lmType)) {
                     factoryClass = FlatLedgerManagerFactory.class;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
index fd2bf5540..a422276ef 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
@@ -33,14 +33,13 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.bookkeeper.bookie.GarbageCollector.GarbageCleaner;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.meta.FlatLedgerManagerFactory;
+import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerTestCase;
 import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager;
@@ -77,7 +76,7 @@ public TestGcOverreplicatedLedger(Class<? extends LedgerManagerFactory> lmFactor
 
     @Parameters
     public static Collection<Object[]> configs() {
-        return Arrays.asList(new Object[][] { { FlatLedgerManagerFactory.class } });
+        return Arrays.asList(new Object[][] { { HierarchicalLedgerManagerFactory.class } });
     }
 
     @Test
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 f79c27e8b..5d7be5dcc 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
@@ -31,7 +31,6 @@
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.meta.FlatLedgerManagerFactory;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerIdGenerator;
 import org.apache.bookkeeper.meta.LedgerManager;
@@ -71,10 +70,11 @@ public TestWatchEnsembleChange(Class<? extends LedgerManagerFactory> lmFactoryCl
         baseConf.setLedgerManagerFactoryClass(lmFactoryCls);
     }
 
+    @SuppressWarnings("deprecation")
     @Parameters
     public static Collection<Object[]> configs() {
         return Arrays.asList(new Object[][] {
-                { FlatLedgerManagerFactory.class },
+                { org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class },
                 { HierarchicalLedgerManagerFactory.class },
                 { LongHierarchicalLedgerManagerFactory.class },
                 { MSLedgerManagerFactory.class },
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 414857581..78d3b08b1 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
@@ -89,6 +89,7 @@ public LedgerIdGenerator getLedgerIdGenerator() throws IOException {
         return ledgerIdGenerator;
     }
 
+    @SuppressWarnings("deprecation")
     @Parameters
     public static Collection<Object[]> configs() {
         return Arrays.asList(new Object[][] {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java
index 655713e6f..62315fe2e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java
@@ -38,16 +38,16 @@
             HierarchicalLedgerManagerFactory.class.getName(),
             2);
 
-    private static final LedgerLayout flat1 =
+    private static final LedgerLayout longHierarchical =
         new LedgerLayout(
-            FlatLedgerManagerFactory.class.getName(),
+            LongHierarchicalLedgerManagerFactory.class.getName(),
             1);
 
     @Test
     public void testEquals() {
         assertEquals(hierarchical1, hierarchical1);
         assertNotEquals(hierarchical1, hierarchical2);
-        assertNotEquals(hierarchical1, flat1);
+        assertNotEquals(hierarchical1, longHierarchical);
     }
 
     @Test
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 744b9afd4..a466590e3 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
@@ -88,11 +88,11 @@ public void testBadConf() throws Exception {
             conf,
             zkLayoutManager);
         assertTrue("Ledger manager is unexpected type",
-                   (m instanceof FlatLedgerManagerFactory));
+                   (m instanceof HierarchicalLedgerManagerFactory));
         m.close();
 
         // mismatching conf
-        conf.setLedgerManagerFactoryClass(HierarchicalLedgerManagerFactory.class);
+        conf.setLedgerManagerFactoryClass(LongHierarchicalLedgerManagerFactory.class);
         try {
             LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
@@ -132,10 +132,10 @@ public void testBadConfV1() throws Exception {
                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         conf.setZkLedgersRootPath(root0);
         // write v1 layout
-        writeLedgerLayout(root0, FlatLedgerManagerFactory.NAME,
-                          FlatLedgerManagerFactory.CUR_VERSION, 1);
+        writeLedgerLayout(root0, HierarchicalLedgerManagerFactory.NAME,
+                          HierarchicalLedgerManagerFactory.CUR_VERSION, 1);
 
-        conf.setLedgerManagerFactoryClass(FlatLedgerManagerFactory.class);
+        conf.setLedgerManagerFactoryClass(HierarchicalLedgerManagerFactory.class);
 
         ZkLayoutManager zkLayoutManager = new ZkLayoutManager(
             zkc,
@@ -147,18 +147,18 @@ public void testBadConfV1() throws Exception {
             zkLayoutManager);
 
         assertTrue("Ledger manager is unexpected type",
-                   (m instanceof FlatLedgerManagerFactory));
+                   (m instanceof HierarchicalLedgerManagerFactory));
         m.close();
 
         // v2 setting doesn't effect v1
         conf.setLedgerManagerFactoryClass(HierarchicalLedgerManagerFactory.class);
         m = LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
         assertTrue("Ledger manager is unexpected type",
-                   (m instanceof FlatLedgerManagerFactory));
+                   (m instanceof HierarchicalLedgerManagerFactory));
         m.close();
 
         // mismatching conf
-        conf.setLedgerManagerType(HierarchicalLedgerManagerFactory.NAME);
+        conf.setLedgerManagerType(LongHierarchicalLedgerManagerFactory.NAME);
         try {
             LedgerManagerFactory.newLedgerManagerFactory(conf, zkLayoutManager);
             fail("Shouldn't reach here");
@@ -203,7 +203,7 @@ public void testBadZkContents() throws Exception {
                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         conf.setZkLedgersRootPath(root1);
 
-        LedgerLayout layout1 = new LedgerLayout(FlatLedgerManagerFactory.class.getName(),
+        LedgerLayout layout1 = new LedgerLayout(HierarchicalLedgerManagerFactory.class.getName(),
                          0xdeadbeef);
         ZkLayoutManager zkLayoutManager1 = new ZkLayoutManager(zkc, root1, ZooDefs.Ids.OPEN_ACL_UNSAFE);
         zkLayoutManager1.storeLedgerLayout(layout1);
@@ -279,7 +279,7 @@ public void testConcurrent1() throws Exception {
         List<CreateLMThread> threads = new ArrayList<CreateLMThread>(numThreads);
         for (int i = 0; i < numThreads; i++) {
             CreateLMThread t = new CreateLMThread(zkUtil.getZooKeeperConnectString(),
-                    root0, FlatLedgerManagerFactory.class.getName(), barrier);
+                    root0, HierarchicalLedgerManagerFactory.class.getName(), barrier);
             t.start();
             threads.add(t);
         }
@@ -309,14 +309,14 @@ public void testConcurrent2() throws Exception {
         List<CreateLMThread> threadsA = new ArrayList<CreateLMThread>(numThreadsEach);
         for (int i = 0; i < numThreadsEach; i++) {
             CreateLMThread t = new CreateLMThread(zkUtil.getZooKeeperConnectString(),
-                    root0, FlatLedgerManagerFactory.class.getName(), barrier);
+                    root0, HierarchicalLedgerManagerFactory.class.getName(), barrier);
             t.start();
             threadsA.add(t);
         }
         List<CreateLMThread> threadsB = new ArrayList<CreateLMThread>(numThreadsEach);
         for (int i = 0; i < numThreadsEach; i++) {
             CreateLMThread t = new CreateLMThread(zkUtil.getZooKeeperConnectString(),
-                    root0, HierarchicalLedgerManagerFactory.class.getName(), barrier);
+                    root0, LongHierarchicalLedgerManagerFactory.class.getName(), barrier);
             t.start();
             threadsB.add(t);
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerLayoutTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerLayoutTest.java
index cbf89aca0..a72cce5d9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerLayoutTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerLayoutTest.java
@@ -87,8 +87,8 @@ public void testBadVersionLedgerLayout() throws Exception {
         ClientConfiguration conf = new ClientConfiguration();
         // write bad version ledger layout
         writeLedgerLayout(conf.getZkLedgersRootPath(),
-                          FlatLedgerManagerFactory.class.getName(),
-                          FlatLedgerManagerFactory.CUR_VERSION,
+                          HierarchicalLedgerManagerFactory.class.getName(),
+                          HierarchicalLedgerManagerFactory.CUR_VERSION,
                           LedgerLayout.LAYOUT_FORMAT_VERSION + 1);
 
         ZkLayoutManager zkLayoutManager = new ZkLayoutManager(zkc, conf.getZkLedgersRootPath(), Ids.OPEN_ACL_UNSAFE);
@@ -130,7 +130,7 @@ public void testBaseLedgerManagerLayout() throws Exception {
         // write bad format ledger layout
         StringBuilder sb = new StringBuilder();
         sb.append(LedgerLayout.LAYOUT_FORMAT_VERSION).append("\n")
-          .append(FlatLedgerManagerFactory.class.getName());
+          .append(HierarchicalLedgerManagerFactory.class.getName());
         zkc.create(ledgersLayout, sb.toString().getBytes(),
                                  Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         ZkLayoutManager zkLayoutManager = new ZkLayoutManager(zkc, rootPath, Ids.OPEN_ACL_UNSAFE);
@@ -148,15 +148,15 @@ public void testReadV1LedgerManagerLayout() throws Exception {
         ClientConfiguration conf = new ClientConfiguration();
         // write v1 ledger layout
         writeLedgerLayout(conf.getZkLedgersRootPath(),
-                          FlatLedgerManagerFactory.NAME,
-                          FlatLedgerManagerFactory.CUR_VERSION, 1);
+                          HierarchicalLedgerManagerFactory.NAME,
+                          HierarchicalLedgerManagerFactory.CUR_VERSION, 1);
         ZkLayoutManager zkLayoutManager = new ZkLayoutManager(zkc, conf.getZkLedgersRootPath(), Ids.OPEN_ACL_UNSAFE);
 
         LedgerLayout layout = zkLayoutManager.readLedgerLayout();
 
         assertNotNull("Should not be null", layout);
-        assertEquals(FlatLedgerManagerFactory.NAME, layout.getManagerFactoryClass());
-        assertEquals(FlatLedgerManagerFactory.CUR_VERSION, layout.getManagerVersion());
+        assertEquals(HierarchicalLedgerManagerFactory.NAME, layout.getManagerFactoryClass());
+        assertEquals(HierarchicalLedgerManagerFactory.CUR_VERSION, layout.getManagerVersion());
         assertEquals(1, layout.getLayoutFormatVersion());
     }
 }
diff --git a/deploy/kubernetes/gke/bookkeeper.statefulset.yml b/deploy/kubernetes/gke/bookkeeper.statefulset.yml
index d72cb2cc8..03ea637df 100644
--- a/deploy/kubernetes/gke/bookkeeper.statefulset.yml
+++ b/deploy/kubernetes/gke/bookkeeper.statefulset.yml
@@ -39,8 +39,6 @@ data:
     BK_ledgerDirectories: "/bookkeeper/data/ledgers"
     BK_indexDirectories: "/bookkeeper/data/ledgers"
     BK_zkServers: zookeeper
-    # the default manager is flat, which is not good for supporting large number of ledgers
-    BK_ledgerManagerType: "hierarchical"
     # TODO: Issue 458: https://github.com/apache/bookkeeper/issues/458
     #BK_statsProviderClass: org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
     # use hostname as bookie id for StatefulSets deployment
diff --git a/deploy/kubernetes/gke/bookkeeper.yaml b/deploy/kubernetes/gke/bookkeeper.yaml
index 587629499..7ad58af04 100644
--- a/deploy/kubernetes/gke/bookkeeper.yaml
+++ b/deploy/kubernetes/gke/bookkeeper.yaml
@@ -30,8 +30,6 @@ data:
     BK_ledgerDirectories: "/bookkeeper/data/ledgers"
     BK_indexDirectories: "/bookkeeper/data/ledgers"
     BK_zkServers: zookeeper
-    # the default manager is flat, which is not good for supporting large number of ledgers
-    BK_ledgerManagerType: "hierarchical"
     # TODO: Issue 458: https://github.com/apache/bookkeeper/issues/458
     #BK_statsProviderClass: org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
 ---
diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml
index 54a26c4be..30882c876 100644
--- a/site/_data/config/bk_server.yaml
+++ b/site/_data/config/bk_server.yaml
@@ -270,8 +270,8 @@ groups:
 
 - name: Ledger manager settings
   params:
-  - param: ledgerManagerType
-    description: The ledger manager type, which defines how ledgers are stored, managed, and garbage collected. See the [Ledger Manager](../../getting-started/concepts#ledger-manager) guide for more details.
+  - param: ledgerManagerFactoryClass
+    description: The ledger manager factory class, which defines how ledgers are stored, managed, and garbage collected. See the [Ledger Manager](../../getting-started/concepts#ledger-manager) guide for more details.
     default: flat
   - param: zkLedgersRootPath
     description: Root Zookeeper path to store ledger metadata. This parameter is used by zookeeper-based ledger manager as a root znode to store all ledgers.
diff --git a/site/docs/latest/overview/releaseNotes.md b/site/docs/latest/overview/releaseNotes.md
index 45f928913..259da6e6d 100644
--- a/site/docs/latest/overview/releaseNotes.md
+++ b/site/docs/latest/overview/releaseNotes.md
@@ -10,3 +10,9 @@ If you want to learn the progress of `{{ site.latest_version }}`, you can do:
 - Or [subscribe](mailto:dev-subscribe@bookkeeper.apache.org) the [dev@bookkeeper.apache.org](mailto:dev@bookkeeper.apache.org)
     to join development discussions, propose new ideas and connect with contributors.
 - Or [join us on Slack](https://apachebookkeeper.herokuapp.com/) to connect with Apache BookKeeper committers and contributors.
+
+### Existing API changes
+
+- The default ledger manager factory is changed from FlatLedgerManagerFactory to HierarchicalLedgerManagerFactory if `ledgerManagerFactoryClass`
+  is specified. If you have a cluster running with older versions and no `ledgerManagerFactoryClass` is set before, you need to set
+  `ledgerManagerFactoryClass` explicitly to `org.apache.bookkeeper.meta.FlatLedgerManagerFactory` in your bookie configuration before upgrade.


 

----------------------------------------------------------------
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