You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by zh...@apache.org on 2017/10/11 01:44:48 UTC

[bookkeeper] branch master updated: ISSUE #547: Allow load extra lifecycle components when running a bookie server.

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

zhaijia 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 7d3d02e  ISSUE #547: Allow load extra lifecycle components when running a bookie server.
7d3d02e is described below

commit 7d3d02e588c1a06f195533c48c5af8149b813978
Author: Sijie Guo <si...@apache.org>
AuthorDate: Wed Oct 11 09:44:34 2017 +0800

    ISSUE #547: Allow load extra lifecycle components when running a bookie server.
    
    Descriptions of the changes in this PR:
    
    - make a common `ServerLifecycleComponet` for the components loaded by a bookie server
    - add a flag to allow start extra components.
    - add test cases
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Jia Zhai <None>, Enrico Olivelli <eo...@gmail.com>
    
    This closes #548 from sijie/loadable_components, closes #547
---
 .../common/component/LifecycleComponentStack.java  | 11 +++
 bookkeeper-server/conf/bk_server.conf              |  9 ++
 bookkeeper-server/pom.xml                          | 34 ++++++++
 .../bookkeeper/conf/ServerConfiguration.java       | 28 ++++++
 .../java/org/apache/bookkeeper/server/Main.java    | 85 ++++++++++++-------
 .../server/component/ServerLifecycleComponent.java | 69 +++++++++++++++
 .../bookkeeper/server/component/package-info.java  | 21 +++++
 .../server/service/AutoRecoveryService.java        |  6 +-
 .../bookkeeper/server/service/BookieService.java   |  6 +-
 .../bookkeeper/server/service/HttpService.java     |  6 +-
 .../server/service/StatsProviderService.java       |  4 +-
 .../apache/bookkeeper/util/ReflectionUtils.java    | 25 +++++-
 .../bookkeeper/conf/TestServerConfiguration.java   | 34 ++++++++
 .../org/apache/bookkeeper/server/TestMain.java     | 99 ++++++++++++++++++++++
 .../component/TestServerLifecycleComponent.java    | 95 +++++++++++++++++++++
 .../bookkeeper/util/TestReflectionUtils.java       | 80 +++++++++++++++++
 .../src/test/resources/bk_server.conf              | 21 +++++
 .../resources/bookkeeper/server-suppressions.xml   |  5 ++
 pom.xml                                            | 13 +++
 19 files changed, 607 insertions(+), 44 deletions(-)

diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java
index 2dcd271..761cff1 100644
--- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java
+++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java
@@ -21,6 +21,7 @@ package org.apache.bookkeeper.common.component;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import java.util.List;
@@ -76,6 +77,16 @@ public class LifecycleComponentStack implements LifecycleComponent {
         this.components = components;
     }
 
+    @VisibleForTesting
+    public int getNumComponents() {
+        return components.size();
+    }
+
+    @VisibleForTesting
+    public LifecycleComponent getComponent(int index) {
+        return components.get(index);
+    }
+
     @Override
     public String getName() {
         return name;
diff --git a/bookkeeper-server/conf/bk_server.conf b/bookkeeper-server/conf/bk_server.conf
index bf6e5c8..98dcbcd 100755
--- a/bookkeeper-server/conf/bk_server.conf
+++ b/bookkeeper-server/conf/bk_server.conf
@@ -545,3 +545,12 @@ zkTimeout=10000
 # all the bookies and Client need to share the same user, and this is usually
 # done using Kerberos authentication. See ZooKeeper documentation
 zkEnableSecurity=false
+
+#############################################################################
+## Server parameters
+#############################################################################
+
+# Configure a list of server components to enable and load on a bookie server.
+# This provides the plugin run extra services along with a bookie server.
+#
+# extraServerComponents=
diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml
index 2042328..a7e70e9 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -387,6 +387,40 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.bookkeeper</groupId>
+            <artifactId>buildtools</artifactId>
+            <version>${project.parent.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>bookkeeper/checkstyle.xml</configLocation>
+          <suppressionsLocation>bookkeeper/server-suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
   <profiles>
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
index 21e00c2..aae3e7d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
@@ -26,6 +26,7 @@ import com.google.common.annotations.Beta;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.SortedLedgerStorage;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.stats.NullStatsProvider;
 import org.apache.bookkeeper.stats.StatsProvider;
 import org.apache.bookkeeper.util.BookKeeperConstants;
@@ -169,6 +170,9 @@ public class ServerConfiguration extends AbstractConfiguration {
     protected final static String TLS_TRUSTSTORE = "tlsTrustStore";
     protected final static String TLS_TRUSTSTORE_PASSWORD_PATH = "tlsTrustStorePasswordPath";
 
+    // Lifecycle Components
+    protected final static String EXTRA_SERVER_COMPONENTS = "extraServerComponents";
+
     /**
      * Construct a default configuration object
      */
@@ -2356,4 +2360,28 @@ public class ServerConfiguration extends AbstractConfiguration {
         return this;
     }
 
+    /**
+     * Get the extra list of server lifecycle components to enable on a bookie server.
+     *
+     * @return the extra list of server lifecycle components to enable on a bookie server.
+     */
+    public String[] getExtraServerComponents() {
+        if (!this.containsKey(EXTRA_SERVER_COMPONENTS)) {
+            return null;
+        }
+        return this.getStringArray(EXTRA_SERVER_COMPONENTS);
+    }
+
+    /**
+     * Set the extra list of server lifecycle components to enable on a bookie server.
+     *
+     * @param componentClasses
+     *          the list of server lifecycle components to enable on a bookie server.
+     * @return server configuration.
+     */
+    public ServerConfiguration setExtraServerComponents(String[] componentClasses) {
+        this.setProperty(EXTRA_SERVER_COMPONENTS, componentClasses);
+        return this;
+    }
+
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
index f3c8247..e4ac80c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
@@ -19,10 +19,12 @@
 package org.apache.bookkeeper.server;
 
 import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
 
 import java.io.File;
 import java.net.MalformedURLException;
 import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.bookie.ExitCode;
@@ -31,6 +33,7 @@ import org.apache.bookkeeper.common.component.LifecycleComponent;
 import org.apache.bookkeeper.common.component.LifecycleComponentStack;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.http.BKServiceProvider;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.server.service.AutoRecoveryService;
 import org.apache.bookkeeper.server.service.BookieService;
@@ -54,39 +57,40 @@ import org.apache.commons.configuration.ConfigurationException;
 @Slf4j
 public class Main {
 
-    static final Options bkOpts = new Options();
+    static final Options BK_OPTS = new Options();
     static {
-        bkOpts.addOption("c", "conf", true, "Configuration for Bookie Server");
-        bkOpts.addOption("withAutoRecovery", false,
+        BK_OPTS.addOption("c", "conf", true, "Configuration for Bookie Server");
+        BK_OPTS.addOption("withAutoRecovery", false,
                 "Start Autorecovery service Bookie server");
-        bkOpts.addOption("readOnly", false,
+        BK_OPTS.addOption("readOnly", false,
                 "Force Start a ReadOnly Bookie server");
-        bkOpts.addOption("z", "zkserver", true, "Zookeeper Server");
-        bkOpts.addOption("m", "zkledgerpath", true, "Zookeeper ledgers root path");
-        bkOpts.addOption("p", "bookieport", true, "bookie port exported");
-        bkOpts.addOption("j", "journal", true, "bookie journal directory");
+        BK_OPTS.addOption("z", "zkserver", true, "Zookeeper Server");
+        BK_OPTS.addOption("m", "zkledgerpath", true, "Zookeeper ledgers root path");
+        BK_OPTS.addOption("p", "bookieport", true, "bookie port exported");
+        BK_OPTS.addOption("j", "journal", true, "bookie journal directory");
         Option indexDirs = new Option ("i", "indexdirs", true, "bookie index directories");
         indexDirs.setArgs(10);
-        bkOpts.addOption(indexDirs);
+        BK_OPTS.addOption(indexDirs);
         Option ledgerDirs = new Option ("l", "ledgerdirs", true, "bookie ledgers directories");
         ledgerDirs.setArgs(10);
-        bkOpts.addOption(ledgerDirs);
-        bkOpts.addOption("h", "help", false, "Print help message");
+        BK_OPTS.addOption(ledgerDirs);
+        BK_OPTS.addOption("h", "help", false, "Print help message");
     }
 
     /**
-     * Print usage
+     * Print usage.
      */
     private static void printUsage() {
         HelpFormatter hf = new HelpFormatter();
         String header = "\n"
-                        + "BookieServer provide an interface to start a bookie with configuration file and/or arguments."
-                        + "The settings in configuration file will be overwrite by provided arguments.\n"
-                        + "Options including:\n";
-        String footer = "Here is an example:\n" +
-                        "\tBookieServer -c bookie.conf -z localhost:2181 -m /bookkeeper/ledgers " +
-                        "-p 3181 -j /mnt/journal -i \"/mnt/index1 /mnt/index2\" -l \"/mnt/ledger1 /mnt/ledger2 /mnt/ledger3\"\n";
-        hf.printHelp("BookieServer [options]\n", header,  bkOpts, footer, true);
+            + "BookieServer provide an interface to start a bookie with configuration file and/or arguments."
+            + "The settings in configuration file will be overwrite by provided arguments.\n"
+            + "Options including:\n";
+        String footer = "Here is an example:\n"
+            + "\tBookieServer -c bookie.conf -z localhost:2181 -m /bookkeeper/ledgers "
+            + "-p 3181 -j /mnt/journal -i \"/mnt/index1 /mnt/index2\""
+            + " -l \"/mnt/ledger1 /mnt/ledger2 /mnt/ledger3\"\n";
+        hf.printHelp("BookieServer [options]\n", header, BK_OPTS, footer, true);
     }
 
     private static void loadConfFile(ServerConfiguration conf, String confFile)
@@ -95,20 +99,20 @@ public class Main {
             conf.loadConf(new File(confFile).toURI().toURL());
             conf.validate();
         } catch (MalformedURLException e) {
-            log.error("Could not open configuration file: " + confFile, e);
+            log.error("Could not open configuration file: {}", confFile, e);
             throw new IllegalArgumentException();
         } catch (ConfigurationException e) {
-            log.error("Malformed configuration file: " + confFile, e);
+            log.error("Malformed configuration file: {}", confFile, e);
             throw new IllegalArgumentException();
         }
-        log.info("Using configuration file " + confFile);
+        log.info("Using configuration file {}", confFile);
     }
 
     private static ServerConfiguration parseArgs(String[] args)
         throws IllegalArgumentException {
         try {
             BasicParser parser = new BasicParser();
-            CommandLine cmdLine = parser.parse(bkOpts, args);
+            CommandLine cmdLine = parser.parse(BK_OPTS, args);
 
             if (cmdLine.hasOption('h')) {
                 throw new IllegalArgumentException();
@@ -133,34 +137,34 @@ public class Main {
             // command line arguments overwrite settings in configuration file
             if (cmdLine.hasOption('z')) {
                 String sZK = cmdLine.getOptionValue('z');
-                log.info("Get cmdline zookeeper instance: " + sZK);
+                log.info("Get cmdline zookeeper instance: {}", sZK);
                 conf.setZkServers(sZK);
             }
 
             if (cmdLine.hasOption('m')) {
                 String sZkLedgersRootPath = cmdLine.getOptionValue('m');
-                log.info("Get cmdline zookeeper ledger path: " + sZkLedgersRootPath);
+                log.info("Get cmdline zookeeper ledger path: {}", sZkLedgersRootPath);
                 conf.setZkLedgersRootPath(sZkLedgersRootPath);
             }
 
             if (cmdLine.hasOption('p')) {
                 String sPort = cmdLine.getOptionValue('p');
-                log.info("Get cmdline bookie port: " + sPort);
+                log.info("Get cmdline bookie port: {}", sPort);
                 Integer iPort = Integer.parseInt(sPort);
                 conf.setBookiePort(iPort.intValue());
             }
 
             if (cmdLine.hasOption('j')) {
                 String sJournalDir = cmdLine.getOptionValue('j');
-                log.info("Get cmdline journal dir: " + sJournalDir);
+                log.info("Get cmdline journal dir: {}", sJournalDir);
                 conf.setJournalDirName(sJournalDir);
             }
 
             if (cmdLine.hasOption('i')) {
                 String[] sIndexDirs = cmdLine.getOptionValues('i');
                 log.info("Get cmdline index dirs: ");
-                for(String index : sIndexDirs) {
-                    log.info("indexDir : " + index);
+                for (String index : sIndexDirs) {
+                    log.info("indexDir : {}", index);
                 }
                 conf.setIndexDirName(sIndexDirs);
             }
@@ -168,8 +172,8 @@ public class Main {
             if (cmdLine.hasOption('l')) {
                 String[] sLedgerDirs = cmdLine.getOptionValues('l');
                 log.info("Get cmdline ledger dirs: ");
-                for(String ledger : sLedgerDirs) {
-                    log.info("ledgerdir : " + ledger);
+                for (String ledger : sLedgerDirs) {
+                    log.info("ledgerdir : {}", ledger);
                 }
                 conf.setLedgerDirNames(sLedgerDirs);
             }
@@ -267,7 +271,7 @@ public class Main {
      * @param conf bookie server configuration
      * @return lifecycle stack
      */
-    private static LifecycleComponent buildBookieServer(BookieConfiguration conf) throws Exception {
+    static LifecycleComponentStack buildBookieServer(BookieConfiguration conf) throws Exception {
         LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack.newBuilder().withName("bookie-server");
 
         // 1. build stats provider
@@ -276,12 +280,14 @@ public class Main {
         StatsLogger rootStatsLogger = statsProviderService.getStatsProvider().getStatsLogger("");
 
         serverBuilder.addComponent(statsProviderService);
+        log.info("Load lifecycle component : {}", StatsProviderService.class.getName());
 
         // 2. build bookie server
         BookieService bookieService =
             new BookieService(conf, rootStatsLogger);
 
         serverBuilder.addComponent(bookieService);
+        log.info("Load lifecycle component : {}", BookieService.class.getName());
 
         // 3. build auto recovery
         if (conf.getServerConf().isAutoRecoveryDaemonEnabled()) {
@@ -289,6 +295,7 @@ public class Main {
                 new AutoRecoveryService(conf, rootStatsLogger.scope(REPLICATION_SCOPE));
 
             serverBuilder.addComponent(autoRecoveryService);
+            log.info("Load lifecycle component : {}", AutoRecoveryService.class.getName());
         }
 
         // 4. build http service
@@ -301,6 +308,20 @@ public class Main {
                 new HttpService(provider, conf, rootStatsLogger);
 
             serverBuilder.addComponent(httpService);
+            log.info("Load lifecycle component : {}", HttpService.class.getName());
+        }
+
+        // 5. build extra services
+        String[] extraComponents = conf.getServerConf().getExtraServerComponents();
+        if (null != extraComponents) {
+            List<ServerLifecycleComponent> components = loadServerComponents(
+                extraComponents,
+                conf,
+                rootStatsLogger);
+            for (ServerLifecycleComponent component : components) {
+                serverBuilder.addComponent(component);
+                log.info("Load lifecycle component : {}", component.getClass().getName());
+            }
         }
 
         return serverBuilder.build();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java
new file mode 100644
index 0000000..281bb7d
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.server.component;
+
+import com.google.common.collect.Lists;
+import java.lang.reflect.Constructor;
+import java.util.List;
+import lombok.experimental.PackagePrivate;
+import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
+import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
+import org.apache.bookkeeper.common.component.LifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+
+/**
+ * A {@link LifecycleComponent} that runs on a bookie server. It can be loaded via reflections.
+ */
+@PackagePrivate
+@Evolving
+public abstract class ServerLifecycleComponent extends AbstractLifecycleComponent<BookieConfiguration> {
+
+    public static List<ServerLifecycleComponent> loadServerComponents(String[] componentClassNames,
+                                                                      BookieConfiguration conf,
+                                                                      StatsLogger statsLogger) {
+        List<Class<? extends ServerLifecycleComponent>> componentClasses =
+            Lists.newArrayListWithExpectedSize(componentClassNames.length);
+        for (String componentClsName : componentClassNames) {
+            componentClasses.add(ReflectionUtils.forName(componentClsName, ServerLifecycleComponent.class));
+        }
+        return Lists.transform(componentClasses, cls -> newComponent(cls, conf, statsLogger));
+
+    }
+
+    static ServerLifecycleComponent newComponent(Class<? extends ServerLifecycleComponent> theCls,
+                                                 BookieConfiguration conf,
+                                                 StatsLogger statsLogger) {
+        try {
+            Constructor<? extends ServerLifecycleComponent> constructor =
+                theCls.getConstructor(BookieConfiguration.class, StatsLogger.class);
+            constructor.setAccessible(true);
+            return constructor.newInstance(conf, statsLogger);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    protected ServerLifecycleComponent(String componentName, BookieConfiguration conf, StatsLogger statsLogger) {
+        super(componentName, conf, statsLogger);
+    }
+
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/package-info.java
new file mode 100644
index 0000000..9e0ce52
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * The lifecycle components that run on a bookie server.
+ */
+package org.apache.bookkeeper.server.component;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java
index 46154ff..b2b8f07 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java
@@ -19,16 +19,16 @@
 package org.apache.bookkeeper.server.service;
 
 import java.io.IOException;
-import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.replication.AutoRecoveryMain;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.stats.StatsLogger;
 
 /**
- * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs autorecovery.
+ * A {@link ServerLifecycleComponent} that runs autorecovery.
  */
-public class AutoRecoveryService extends AbstractLifecycleComponent<BookieConfiguration> {
+public class AutoRecoveryService extends ServerLifecycleComponent {
 
     public static final String NAME = "autorecovery";
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
index c63388d..13a77e8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
@@ -19,16 +19,16 @@
 package org.apache.bookkeeper.server.service;
 
 import java.io.IOException;
-import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.stats.StatsLogger;
 
 /**
- * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that starts the core bookie server.
+ * A {@link ServerLifecycleComponent} that starts the core bookie server.
  */
-public class BookieService extends AbstractLifecycleComponent<BookieConfiguration> {
+public class BookieService extends ServerLifecycleComponent {
 
     public static final String NAME = "bookie-server";
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java
index 9ba00f6..78f97fe 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java
@@ -21,17 +21,17 @@ package org.apache.bookkeeper.server.service;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.io.IOException;
-import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.http.BKServiceProvider;
 import org.apache.bookkeeper.http.HttpServer;
 import org.apache.bookkeeper.http.HttpServerLoader;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.stats.StatsLogger;
 
 /**
- * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs http service.
+ * A {@link ServerLifecycleComponent} that runs http service.
  */
-public class HttpService extends AbstractLifecycleComponent<BookieConfiguration> {
+public class HttpService extends ServerLifecycleComponent {
 
     public static final String NAME = "http-service";
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java
index 88fa5bd..ab5d894 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java
@@ -19,7 +19,7 @@
 package org.apache.bookkeeper.server.service;
 
 import java.io.IOException;
-import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsProvider;
@@ -28,7 +28,7 @@ import org.apache.bookkeeper.util.ReflectionUtils;
 /**
  * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs stats provider.
  */
-public class StatsProviderService extends AbstractLifecycleComponent<BookieConfiguration> {
+public class StatsProviderService extends ServerLifecycleComponent {
 
     public static final String NAME = "stats-provider";
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java
index 4f3bdb4..6d096a1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java
@@ -23,7 +23,6 @@ package org.apache.bookkeeper.util;
 import java.lang.reflect.Constructor;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationException;
 
@@ -36,6 +35,30 @@ public class ReflectionUtils {
             new ConcurrentHashMap<Class<?>, Constructor<?>>();
 
     /**
+     * Returns the {@code Class} object object associated with the class or interface
+     * with the given string name, which is a subclass of {@code xface}.
+     *
+     * @param className class name
+     * @param xface class interface
+     * @return the class object associated with the class or interface with the given string name.
+     */
+    public static <U> Class<? extends U> forName(String className,
+                                                 Class<U> xface) {
+
+        // Construct the class
+        Class<?> theCls;
+        try {
+            theCls = Class.forName(className);
+        } catch (ClassNotFoundException cnfe) {
+            throw new RuntimeException(cnfe);
+        }
+        if (!xface.isAssignableFrom(theCls)) {
+            throw new RuntimeException(className + " not " + xface.getName());
+        }
+        return theCls.asSubclass(xface);
+    }
+
+    /**
      * Get the value of the <code>name</code> property as a <code>Class</code>.
      * If no such property is specified, then <code>defaultCls</code> is returned.
      *
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java
index 1f62258..ed29d71 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java
@@ -21,9 +21,12 @@
 
 package org.apache.bookkeeper.conf;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.configuration.ConfigurationException;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -31,6 +34,18 @@ import org.junit.Test;
  */
 public class TestServerConfiguration {
 
+    private final ServerConfiguration serverConf;
+
+    public TestServerConfiguration() {
+        serverConf = new ServerConfiguration();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        serverConf.loadConf(
+            getClass().getClassLoader().getResource("bk_server.conf"));
+    }
+
     @Test
     public void testEphemeralPortsAllowed() throws ConfigurationException {
         ServerConfiguration conf = new ServerConfiguration();
@@ -49,4 +64,23 @@ public class TestServerConfiguration {
         conf.validate();
     }
 
+    @Test
+    public void testSetExtraServerComponents() {
+        ServerConfiguration conf = new ServerConfiguration();
+        assertNull(conf.getExtraServerComponents());
+        String[] components = new String[] {
+            "test1", "test2", "test3"
+        };
+        conf.setExtraServerComponents(components);
+        assertArrayEquals(components, conf.getExtraServerComponents());
+    }
+
+    @Test
+    public void testGetExtraServerComponents() {
+        String[] components = new String[] {
+            "test1", "test2", "test3"
+        };
+        assertArrayEquals(components, serverConf.getExtraServerComponents());
+    }
+
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java
new file mode 100644
index 0000000..34708ca
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.server;
+
+import static org.apache.bookkeeper.server.Main.buildBookieServer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import java.io.IOException;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test of {@link Main}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(BookieService.class)
+public class TestMain {
+
+    static class TestComponent extends ServerLifecycleComponent {
+
+        public TestComponent(BookieConfiguration conf, StatsLogger statsLogger) {
+            super("test-component", conf, statsLogger);
+        }
+
+        @Override
+        protected void doStart() {
+        }
+
+        @Override
+        protected void doStop() {
+        }
+
+        @Override
+        protected void doClose() throws IOException {
+        }
+
+    }
+
+    @Test
+    public void testBuildBookieServer() throws Exception {
+        ServerConfiguration serverConf = new ServerConfiguration()
+            .setAutoRecoveryDaemonEnabled(false)
+            .setHttpServerEnabled(false)
+            .setExtraServerComponents(new String[] { TestComponent.class.getName() });
+        BookieConfiguration conf = new BookieConfiguration(serverConf);
+
+        BookieServer mockServer = PowerMockito.mock(BookieServer.class);
+        whenNew(BookieServer.class)
+            .withArguments(any(ServerConfiguration.class), any(StatsLogger.class))
+            .thenReturn(mockServer);
+
+        LifecycleComponentStack stack = buildBookieServer(conf);
+        assertEquals(3, stack.getNumComponents());
+        assertTrue(stack.getComponent(2) instanceof TestComponent);
+
+        stack.start();
+        verify(mockServer, times(1)).start();
+
+        stack.stop();
+
+        stack.close();
+        verify(mockServer, times(1)).shutdown();
+    }
+
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/component/TestServerLifecycleComponent.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/component/TestServerLifecycleComponent.java
new file mode 100644
index 0000000..135e39e
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/component/TestServerLifecycleComponent.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.server.component;
+
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.newComponent;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Test;
+
+public class TestServerLifecycleComponent {
+
+    static class TestComponent extends ServerLifecycleComponent {
+
+        public TestComponent(BookieConfiguration conf, StatsLogger statsLogger) {
+            super("test-component", conf, statsLogger);
+        }
+
+        @Override
+        protected void doStart() {
+            // no-op
+        }
+
+        @Override
+        protected void doStop() {
+            // no-op
+        }
+
+        @Override
+        protected void doClose() throws IOException {
+            // no-op
+        }
+    }
+
+    static class TestComponent2 extends TestComponent {
+        public TestComponent2(BookieConfiguration conf, StatsLogger statsLogger) {
+            super(conf, statsLogger);
+        }
+    }
+
+    @Test
+    public void testNewComponent() throws Exception {
+        BookieConfiguration conf = new BookieConfiguration(new ServerConfiguration());
+        StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+        ServerLifecycleComponent component = newComponent(
+            TestComponent.class,
+            conf,
+            statsLogger);
+        assertEquals("test-component", component.getName());
+        assertEquals(conf, component.getConf());
+    }
+
+    @Test
+    public void testLoadServerComponents() throws Exception {
+        BookieConfiguration conf = new BookieConfiguration(new ServerConfiguration());
+        StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+        String[] clsNames = new String[] {
+            TestComponent.class.getName(),
+            TestComponent2.class.getName()
+        };
+        List<ServerLifecycleComponent> components = loadServerComponents(
+            clsNames,
+            conf,
+            statsLogger);
+        assertEquals(2, components.size());
+        assertTrue(components.get(0) instanceof TestComponent);
+        assertTrue(components.get(1) instanceof TestComponent2);
+    }
+
+
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java
new file mode 100644
index 0000000..6171239
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.util;
+
+import static org.apache.bookkeeper.util.ReflectionUtils.forName;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.junit.Test;
+
+/**
+ * Unit test of {@link ReflectionUtils}.
+ */
+public class TestReflectionUtils {
+
+    interface InterfaceA {
+    }
+
+    interface InterfaceB {
+    }
+
+    private static class ClassA implements InterfaceA {
+    }
+
+    private static class ClassB implements InterfaceB {
+    }
+
+    @Test
+    public void testForNameClassNotFound() {
+        try {
+            forName(
+                "test.for.name.class.not.found",
+                Object.class);
+            fail("Should fail if class not found");
+        } catch (RuntimeException re) {
+            assertTrue(re.getCause() instanceof ClassNotFoundException);
+        }
+    }
+
+    @Test
+    public void testForNameUnassignable() {
+        try {
+            forName(
+                ClassA.class.getName(),
+                InterfaceB.class);
+            fail("Should fail if class is not assignable");
+        } catch (RuntimeException re) {
+            assertEquals(
+                ClassA.class.getName() + " not " + InterfaceB.class.getName(),
+                re.getMessage());
+        }
+    }
+
+    @Test
+    public void testForName() throws Exception {
+        Class<? extends InterfaceB> theCls = forName(
+            ClassB.class.getName(),
+            InterfaceB.class);
+        assertEquals(ClassB.class, theCls);
+    }
+
+}
diff --git a/bookkeeper-server/src/test/resources/bk_server.conf b/bookkeeper-server/src/test/resources/bk_server.conf
new file mode 100644
index 0000000..883b1ea
--- /dev/null
+++ b/bookkeeper-server/src/test/resources/bk_server.conf
@@ -0,0 +1,21 @@
+#/**
+# * Copyright 2007 The Apache Software Foundation
+# *
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+extraServerComponents=test1,test2,test3
diff --git a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
index ac3e999..470aff7 100644
--- a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
+++ b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
@@ -21,9 +21,11 @@
     <suppress checks="JavadocPackage" files=".*[\\/]maven-archetypes[\\/].*"/>
     <suppress checks="JavadocPackage" files=".*[\\/]examples[\\/].*"/>
     <!-- suppress packages by packages -->
+    <suppress checks=".*" files=".*[\\/]bookie[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]client[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]conf[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]feature[\\/].*"/>
+    <suppress checks=".*" files=".*[\\/]http[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]meta[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]metastore[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]net[\\/].*"/>
@@ -34,8 +36,11 @@
     <suppress checks=".*" files=".*[\\/]shims[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]stats[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]streaming[\\/].*"/>
+    <suppress checks=".*" files=".*[\\/]test[\\/].*"/>
+    <suppress checks=".*" files=".*[\\/]tls[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]tools[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]util[\\/].*"/>
+    <suppress checks=".*" files=".*[\\/]versioning[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]zookeeper[\\/].*"/>
 
     <!-- suppress all checks in the generated directories -->
diff --git a/pom.xml b/pom.xml
index d5b066f..45e7eb5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
     <junit.version>4.12</junit.version>
     <lombok.version>1.16.12</lombok.version>
     <mockito-core.version>2.8.9</mockito-core.version>
+    <powermock.version>1.7.3</powermock.version>
     <protobuf.version>3.4.0</protobuf.version>
     <netty.version>4.1.12.Final</netty.version>
     <netty-boringssl.version>2.0.3.Final</netty-boringssl.version>
@@ -170,6 +171,18 @@
       <version>${mockito-core.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-mockito2</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <co...@bookkeeper.apache.org>'].