You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by sj...@apache.org on 2018/09/07 22:08:59 UTC

[bookkeeper] branch master updated: ISSUE #1662: LocalBookKeeper use layout manager and ledger path configs

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

sjust 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 74bd926  ISSUE #1662: LocalBookKeeper use layout manager and ledger path configs
74bd926 is described below

commit 74bd92605bc4986705031aea8943905dbc9f9f85
Author: Samuel Just <sj...@salesforce.com>
AuthorDate: Fri Sep 7 15:08:57 2018 -0700

    ISSUE #1662: LocalBookKeeper use layout manager and ledger path configs
    
    The URI patch had a side effect of causing LocalBookKeeper to always
    use a URI string of the form zk:// which defaults to HLM.  This
    patch updates LocalBookKeeper to generate zk+<layout>:// based on
    the legacy LEDGER_MANAGER_FACTORY_CLASS config when present.  +null
    will be used otherwise and defaults to HLM.  Further, use the
    (admittedly deprecated) zkLedgerRootPath config to determine the
    ledger path (defaults to /ledger, so the behavior will be unchanged
    if unspecified).
    
    (bug W-5415276)
    (rev cguttapalem)
    Signed-off-by: Samuel Just <sjustsalesforce.com>
    
    Author: Samuel Just <sj...@salesforce.com>
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #1669 from athanatos/forupstream/wip-1662, closes #1662
---
 .../bookkeeper/conf/AbstractConfiguration.java     | 58 +++++++++++++---------
 .../apache/bookkeeper/util/LocalBookKeeper.java    | 52 +++++++++++--------
 .../bookkeeper/conf/AbstractConfigurationTest.java |  3 +-
 3 files changed, 67 insertions(+), 46 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
index 1ce4cf3..9f9547c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
@@ -225,35 +225,12 @@ public abstract class AbstractConfiguration<T extends AbstractConfiguration>
      * @return metadata service uri.
      * @throws ConfigurationException if the metadata service uri is invalid.
      */
-    @SuppressWarnings("deprecation")
     public String getMetadataServiceUri() throws ConfigurationException {
         String serviceUri = getString(METADATA_SERVICE_URI);
         if (null == serviceUri) {
             // no service uri is defined, fallback to old settings
             String ledgerManagerType;
-            Class<? extends LedgerManagerFactory> factoryClass = getLedgerManagerFactoryClass();
-            if (factoryClass == null) {
-                // set the ledger manager type to "null", so the driver implementation knows that the type is not set.
-                ledgerManagerType = "null";
-            } else {
-                if (!AbstractZkLedgerManagerFactory.class.isAssignableFrom(factoryClass)) {
-                    // this is a non-zk implementation
-                    throw new UnsupportedOperationException("metadata service uri is not supported for "
-                        + factoryClass);
-                }
-                if (factoryClass == HierarchicalLedgerManagerFactory.class) {
-                    ledgerManagerType = HierarchicalLedgerManagerFactory.NAME;
-                } else if (factoryClass == org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class) {
-                    ledgerManagerType = org.apache.bookkeeper.meta.FlatLedgerManagerFactory.NAME;
-                } else if (factoryClass == LongHierarchicalLedgerManagerFactory.class) {
-                    ledgerManagerType = LongHierarchicalLedgerManagerFactory.NAME;
-                } else if (factoryClass == org.apache.bookkeeper.meta.MSLedgerManagerFactory.class) {
-                    ledgerManagerType = org.apache.bookkeeper.meta.MSLedgerManagerFactory.NAME;
-                } else {
-                    throw new IllegalArgumentException("Unknown zookeeper based ledger manager factory : "
-                        + factoryClass);
-                }
-            }
+            ledgerManagerType = getLedgerManagerLayoutStringFromFactoryClass();
             String zkServers = getZkServers();
             if (null != zkServers) {
                 // URI doesn't accept ','
@@ -427,6 +404,39 @@ public abstract class AbstractConfiguration<T extends AbstractConfiguration>
     }
 
     /**
+     * Get layout string ("null" if unconfigured).
+     *
+     * @return null, hierarchical, longhierarchical, or flat based on LEDGER_MANAGER_FACTORY_CLASS
+     */
+    @SuppressWarnings("deprecation")
+    public String getLedgerManagerLayoutStringFromFactoryClass() throws ConfigurationException {
+        String ledgerManagerType;
+        Class<? extends LedgerManagerFactory> factoryClass = getLedgerManagerFactoryClass();
+        if (factoryClass == null) {
+            // set the ledger manager type to "null", so the driver implementation knows that the type is not set.
+            ledgerManagerType = "null";
+        } else {
+            if (!AbstractZkLedgerManagerFactory.class.isAssignableFrom(factoryClass)) {
+                // this is a non-zk implementation
+                throw new ConfigurationException("metadata service uri is not supported for " + factoryClass);
+            }
+            if (factoryClass == HierarchicalLedgerManagerFactory.class) {
+                ledgerManagerType = HierarchicalLedgerManagerFactory.NAME;
+            } else if (factoryClass == org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class) {
+                ledgerManagerType = org.apache.bookkeeper.meta.FlatLedgerManagerFactory.NAME;
+            } else if (factoryClass == LongHierarchicalLedgerManagerFactory.class) {
+                ledgerManagerType = LongHierarchicalLedgerManagerFactory.NAME;
+            } else if (factoryClass == org.apache.bookkeeper.meta.MSLedgerManagerFactory.class) {
+                ledgerManagerType = org.apache.bookkeeper.meta.MSLedgerManagerFactory.NAME;
+            } else {
+                throw new IllegalArgumentException("Unknown zookeeper based ledger manager factory : "
+                        + factoryClass);
+            }
+        }
+        return ledgerManagerType;
+    }
+
+    /**
      * Set Ledger Manager Factory Class.
      *
      * @param factoryClass
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
index 72ba48a..9555474 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
@@ -37,7 +37,6 @@ import java.util.List;
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.proto.BookieServer;
@@ -47,6 +46,7 @@ import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShimFactory;
 import org.apache.bookkeeper.tls.SecurityException;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.io.FileUtils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -62,8 +62,8 @@ public class LocalBookKeeper {
     protected static final Logger LOG = LoggerFactory.getLogger(LocalBookKeeper.class);
     public static final int CONNECTION_TIMEOUT = 30000;
 
-    private static String newMetadataServiceUri(String zkServers, int port) {
-        return "zk://" + zkServers + ":" + port + "/ledgers";
+    private static String newMetadataServiceUri(String zkServers, int port, String layout, String ledgerPath) {
+        return "zk+" + layout + "://" + zkServers + ":" + port + ledgerPath;
     }
 
     int numberOfBookies;
@@ -73,13 +73,18 @@ public class LocalBookKeeper {
     }
 
     public LocalBookKeeper(int numberOfBookies) {
-        this(numberOfBookies, 5000, defaultLocalBookiesConfigDir);
+        this(numberOfBookies, 5000, new ServerConfiguration(), defaultLocalBookiesConfigDir);
     }
 
-    public LocalBookKeeper(int numberOfBookies, int initialPort, String localBookiesConfigDirName) {
+    public LocalBookKeeper(
+            int numberOfBookies,
+            int initialPort,
+            ServerConfiguration baseConf,
+            String localBookiesConfigDirName) {
         this.numberOfBookies = numberOfBookies;
         this.initialPort = initialPort;
         this.localBookiesConfigDir = new File(localBookiesConfigDirName);
+        this.baseConf = baseConf;
         LOG.info("Running {} bookie(s) on zkServer {}.", this.numberOfBookies);
     }
 
@@ -94,6 +99,7 @@ public class LocalBookKeeper {
     BookieServer bs[];
     ServerConfiguration bsConfs[];
     Integer initialPort = 5000;
+    private ServerConfiguration baseConf;
 
     File localBookiesConfigDir;
     /**
@@ -121,7 +127,7 @@ public class LocalBookKeeper {
     }
 
     @SuppressWarnings("deprecation")
-    private void initializeZookeeper(AbstractConfiguration conf, String zkHost, int zkPort) throws IOException {
+    private void initializeZookeeper(String zkHost, int zkPort) throws IOException {
         LOG.info("Instantiate ZK Client");
         //initialize the zk client with values
         try (ZooKeeperClient zkc = ZooKeeperClient.newBuilder()
@@ -129,7 +135,7 @@ public class LocalBookKeeper {
                     .sessionTimeoutMs(zkSessionTimeOut)
                     .build()) {
             List<Op> multiOps = Lists.newArrayListWithExpectedSize(3);
-            String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+            String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf);
             multiOps.add(
                 Op.create(zkLedgersRootPath, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
             multiOps.add(
@@ -158,12 +164,13 @@ public class LocalBookKeeper {
         }
     }
 
-    private List<File> runBookies(ServerConfiguration baseConf, String dirSuffix)
+    private List<File> runBookies(String dirSuffix)
             throws IOException, KeeperException, InterruptedException, BookieException,
-            UnavailableException, CompatibilityException, SecurityException, BKException {
+            UnavailableException, CompatibilityException, SecurityException, BKException,
+            ConfigurationException {
         List<File> tempDirs = new ArrayList<File>();
         try {
-            runBookies(baseConf, tempDirs, dirSuffix);
+            runBookies(tempDirs, dirSuffix);
             return tempDirs;
         } catch (IOException ioe) {
             cleanupDirectories(tempDirs);
@@ -188,9 +195,9 @@ public class LocalBookKeeper {
     }
 
     @SuppressWarnings("deprecation")
-    private void runBookies(ServerConfiguration baseConf, List<File> tempDirs, String dirSuffix)
+    private void runBookies(List<File> tempDirs, String dirSuffix)
             throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException,
-            CompatibilityException, SecurityException, BKException {
+            CompatibilityException, SecurityException, BKException, ConfigurationException {
         LOG.info("Starting Bookie(s)");
         // Create Bookie Servers (B1, B2, B3)
 
@@ -256,8 +263,7 @@ public class LocalBookKeeper {
             }
 
             if (null == baseConf.getMetadataServiceUriUnchecked()) {
-                bsConfs[i].setMetadataServiceUri(
-                    newMetadataServiceUri(InetAddress.getLocalHost().getHostAddress(), zooKeeperDefaultPort));
+                bsConfs[i].setMetadataServiceUri(baseConf.getMetadataServiceUri());
             }
 
             bsConfs[i].setJournalDirName(journalDirs[i].getPath());
@@ -285,8 +291,7 @@ public class LocalBookKeeper {
         ServerConfiguration baseConfWithCorrectZKServers = new ServerConfiguration(
                 (ServerConfiguration) baseConf.clone());
         if (null == baseConf.getMetadataServiceUriUnchecked()) {
-            baseConfWithCorrectZKServers.setMetadataServiceUri(
-                newMetadataServiceUri(InetAddress.getLocalHost().getHostAddress(), zooKeeperDefaultPort));
+            baseConfWithCorrectZKServers.setMetadataServiceUri(baseConf.getMetadataServiceUri());
         }
         serializeLocalBookieConfig(baseConfWithCorrectZKServers, "baseconf.conf");
     }
@@ -340,7 +345,14 @@ public class LocalBookKeeper {
                                           String zkDataDir,
                                           String localBookiesConfigDirName)
             throws Exception {
-        LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, localBookiesConfigDirName);
+
+        conf.setMetadataServiceUri(
+                newMetadataServiceUri(
+                        zkHost,
+                        zkPort,
+                        conf.getLedgerManagerLayoutStringFromFactoryClass(),
+                        conf.getZkLedgersRootPath()));
+        LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, conf, localBookiesConfigDirName);
 
         ZooKeeperServerShim zks = null;
         File zkTmpDir = null;
@@ -363,10 +375,8 @@ public class LocalBookKeeper {
                 zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
             }
 
-            conf.setMetadataServiceUri(newMetadataServiceUri(zkHost, zkPort));
-
-            lb.initializeZookeeper(conf, zkHost, zkPort);
-            bkTmpDirs = lb.runBookies(conf, dirSuffix);
+            lb.initializeZookeeper(zkHost, zkPort);
+            bkTmpDirs = lb.runBookies(dirSuffix);
 
             try {
                 while (true) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java
index 8039b0c..54e07bd 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java
@@ -26,6 +26,7 @@ import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory;
+import org.apache.commons.configuration.ConfigurationException;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -76,7 +77,7 @@ public class AbstractConfigurationTest {
     }
 
     @SuppressWarnings({ "unchecked" })
-    @Test(expected = UnsupportedOperationException.class)
+    @Test(expected = ConfigurationException.class)
     public void testUnsupportedLedgerManagerFactory() throws Exception {
         LedgerManagerFactory mockFactory = mock(LedgerManagerFactory.class, CALLS_REAL_METHODS);
         conf.setLedgerManagerFactoryClass(mockFactory.getClass());