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/06/04 20:52:33 UTC

[bookkeeper] branch branch-4.7 updated (a30d2fe -> 5be2e90)

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

sijie pushed a change to branch branch-4.7
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git.


    from a30d2fe  ISSUE #1339: Cleanup the directories created by DbLedgerStorageTest
     new df4b389  Issue #1396: Export metrics in same http server
     new 61ab290  Refactor bookkeeper bash scripts and move dlog script to root bin directory
     new ba51575  Add a test docker image for current version only
     new 272b8e7  Add a docker based `BookKeeperClusterTestBase` for failure related integration tests
     new 5345be3  Provide zookeeper startup script
     new 5be2e90  Provide a util method to run functions with metadata client driver

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/bookkeeper                                     | 229 ++++++------------
 bin/bookkeeper-cli                                 | 141 +++---------
 bin/bookkeeper-daemon.sh                           |   5 +
 bin/common.sh                                      | 251 ++++++++++++++++++++
 bin/dlog                                           | 131 +++++++++++
 bookkeeper-dist/all/pom.xml                        |  21 ++
 bookkeeper-dist/server/pom.xml                     |  20 ++
 .../src/main/resources/LICENSE-all.bin.txt         |   6 +
 .../src/main/resources/LICENSE-server.bin.txt      |   6 +
 .../org/apache/bookkeeper/http/HttpServer.java     |   1 +
 .../apache/bookkeeper/meta/MetadataDrivers.java    |  35 +++
 .../java/org/apache/bookkeeper/server/Main.java    |   1 +
 .../server/http/BKHttpServiceProvider.java         |  18 +-
 .../server/http/service/MetricsService.java        |  78 +++++++
 .../server/http/service/MetricsServiceTest.java    | 118 ++++++++++
 .../prometheus/PrometheusMetricsProvider.java      |   8 +-
 .../prometheus/TestPrometheusMetricsProvider.java  |  11 +
 .../org/apache/bookkeeper/stats/StatsProvider.java |  11 +
 conf/bk_cli_env.sh                                 |  20 +-
 conf/bk_server.conf                                |   7 +-
 conf/bkenv.sh                                      |  45 +++-
 stream/conf/standalone.conf => conf/nettyenv.sh    |  13 +-
 .../zookeeper.conf.template => conf/zookeeper.conf |  69 ++++--
 .../Dockerfile => conf/zookeeper.conf.dynamic      |   3 +-
 dev/check-binary-license                           |   5 +
 docker/Dockerfile                                  |  12 +-
 docker/scripts/apply-config-from-env.py            |   7 +
 docker/scripts/common.sh                           |  65 ++++++
 docker/scripts/entrypoint.sh                       | 116 +++-------
 docker/scripts/healthcheck.sh                      |   8 +-
 docker/scripts/init_bookie.sh                      | 140 +++++++++++
 .../scripts/init_generic.sh                        |   4 +-
 .../scripts/init_standalone.sh                     |   7 +-
 docker/scripts/init_zookeeper.sh                   |  74 ++++++
 pom.xml                                            |  14 ++
 stream/distributedlog/core/bin/common.sh           | 124 ----------
 stream/distributedlog/core/bin/dlog                |  73 ------
 .../distributedlog/core/conf/bookie.conf.template  | 183 ---------------
 .../distributedlog/core/conf/distributedlog.conf   | 125 ----------
 stream/distributedlog/core/conf/dlogenv.sh         |  75 ------
 stream/distributedlog/core/conf/log4j.properties   |  56 -----
 stream/distributedlog/core/conf/write_proxy.conf   | 143 ------------
 .../core/conf/zookeeper.conf.dynamic.template      |   1 -
 .../current-version-image}/Dockerfile              |  35 ++-
 .../pom.xml                                        |  42 ++--
 tests/docker-images/pom.xml                        |   1 +
 tests/integration-tests-topologies/pom.xml         |  19 +-
 .../tests/containers/BKStandaloneContainer.java    |  99 ++++++++
 .../tests/containers/BookieContainer.java          | 116 ++++++++++
 .../tests/containers/ChaosContainer.java           | 116 ++++++++++
 .../tests/containers/MetadataStoreContainer.java   |  20 +-
 .../bookkeeper/tests/containers/ZKContainer.java   |  75 ++++++
 .../tests/containers/wait/HttpWaitStrategy.java    | 215 +++++++++++++++++
 .../tests/containers/wait/ZKWaitStrategy.java      |  63 +++++
 .../bookkeeper/tests/BookKeeperClusterUtils.java   |  34 ++-
 .../org/apache/bookkeeper/tests/DockerUtils.java   |  29 ++-
 tests/integration/cluster/pom.xml                  |  70 ++++++
 .../cluster/BookKeeperClusterTestBase.java         |  94 ++++++++
 .../integration/cluster/SimpleClusterTest.java     |  78 +++++++
 .../tests/integration/topologies/BKCluster.java    | 156 +++++++++++++
 .../cluster}/src/test/resources/log4j.properties   |   0
 tests/integration/pom.xml                          |  43 +++-
 tests/integration/smoke/pom.xml                    |  21 ++
 .../tests/integration/BookieShellTestBase.java     |  79 +++++++
 .../tests/integration/TestBookieShellCluster.java  |  83 +++++++
 .../bookkeeper/tests/integration/TestCLI.java      | 102 ++++++++
 .../bookkeeper/tests/integration/TestDlogCLI.java  | 146 ++++++++++++
 .../bookkeeper/tests/integration/TestSmoke.java    |  25 +-
 tests/integration/standalone/pom.xml               |  63 +++++
 .../integration/standalone/StandaloneTest.java     |  53 +++++
 .../src/test/resources/log4j.properties            |   0
 tests/pom.xml                                      |   1 +
 tests/scripts/pom.xml                              |  75 ++++++
 tests/scripts/src/test/bash/bk_test.sh             | 141 ++++++++++++
 tests/scripts/src/test/bash/bk_test_bin_common.sh  | 256 +++++++++++++++++++++
 tests/scripts/src/test/bash/bk_test_helpers        |  94 ++++++++
 tests/scripts/src/test/bash/versions               | 173 ++++++++++++++
 77 files changed, 3844 insertions(+), 1253 deletions(-)
 create mode 100755 bin/common.sh
 create mode 100755 bin/dlog
 create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/MetricsService.java
 create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/MetricsServiceTest.java
 copy stream/conf/standalone.conf => conf/nettyenv.sh (74%)
 mode change 100755 => 100644
 rename stream/distributedlog/core/conf/zookeeper.conf.template => conf/zookeeper.conf (56%)
 copy site/docker/Dockerfile => conf/zookeeper.conf.dynamic (93%)
 create mode 100755 docker/scripts/common.sh
 create mode 100755 docker/scripts/init_bookie.sh
 copy stream/conf/standalone.conf => docker/scripts/init_generic.sh (94%)
 copy stream/conf/standalone.conf => docker/scripts/init_standalone.sh (91%)
 create mode 100755 docker/scripts/init_zookeeper.sh
 delete mode 100755 stream/distributedlog/core/bin/common.sh
 delete mode 100755 stream/distributedlog/core/bin/dlog
 delete mode 100644 stream/distributedlog/core/conf/bookie.conf.template
 delete mode 100644 stream/distributedlog/core/conf/distributedlog.conf
 delete mode 100644 stream/distributedlog/core/conf/dlogenv.sh
 delete mode 100644 stream/distributedlog/core/conf/log4j.properties
 delete mode 100644 stream/distributedlog/core/conf/write_proxy.conf
 delete mode 100644 stream/distributedlog/core/conf/zookeeper.conf.dynamic.template
 copy {docker => tests/docker-images/current-version-image}/Dockerfile (51%)
 copy tests/docker-images/{all-versions-image => current-version-image}/pom.xml (73%)
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BookieContainer.java
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ChaosContainer.java
 copy stream/api/src/main/java/org/apache/bookkeeper/api/exceptions/ApiException.java => tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/MetadataStoreContainer.java (65%)
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/HttpWaitStrategy.java
 create mode 100644 tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/ZKWaitStrategy.java
 create mode 100644 tests/integration/cluster/pom.xml
 create mode 100644 tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java
 create mode 100644 tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java
 create mode 100644 tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/topologies/BKCluster.java
 copy {circe-checksum => tests/integration/cluster}/src/test/resources/log4j.properties (100%)
 create mode 100644 tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java
 create mode 100644 tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java
 create mode 100644 tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
 create mode 100644 tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestDlogCLI.java
 create mode 100644 tests/integration/standalone/pom.xml
 create mode 100644 tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
 copy {circe-checksum => tests/integration/standalone}/src/test/resources/log4j.properties (100%)
 create mode 100644 tests/scripts/pom.xml
 create mode 100755 tests/scripts/src/test/bash/bk_test.sh
 create mode 100644 tests/scripts/src/test/bash/bk_test_bin_common.sh
 create mode 100644 tests/scripts/src/test/bash/bk_test_helpers
 create mode 100644 tests/scripts/src/test/bash/versions

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

[bookkeeper] 01/06: Issue #1396: Export metrics in same http server

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit df4b389058c6c7b11fdf76f08523f97f59c675d3
Author: Sijie Guo <si...@apache.org>
AuthorDate: Mon May 14 13:03:54 2018 -0700

    Issue #1396: Export metrics in same http server
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    Currently metrics provider and http admin endpoint are isolated due to the way how things are modularized.
    This requires two ports to be used, one is for metrics provider to expose metrics, while the other one is used by
    http admin endpoint for expose admin endpoints.
    
    When implementing a bookkeeper operator on k8s, it becomes confusing on managing those ports and do health checks.
    It would be good to allow export metrics in same http admin endpoint if it is enabled.
    
    *Solution*
    
    - Introduce a method `writeAllMetrics` in stats provider to allow stats provider exporting metrics.
    - Add a `MetricsService` in bookie http service and export stats provider's metrics under `/metrics` endpoint.
    
    This fixes #1396
    
    Master Issue: #1396
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Jia Zhai <None>
    
    This closes #1399 from sijie/same_http_metrics_port, closes #1396
---
 .../org/apache/bookkeeper/http/HttpServer.java     |   1 +
 .../java/org/apache/bookkeeper/server/Main.java    |   1 +
 .../server/http/BKHttpServiceProvider.java         |  18 +++-
 .../server/http/service/MetricsService.java        |  78 ++++++++++++++
 .../server/http/service/MetricsServiceTest.java    | 118 +++++++++++++++++++++
 .../prometheus/PrometheusMetricsProvider.java      |   8 +-
 .../prometheus/TestPrometheusMetricsProvider.java  |  11 ++
 .../org/apache/bookkeeper/stats/StatsProvider.java |  11 ++
 8 files changed, 241 insertions(+), 5 deletions(-)

diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java
index d2c4e81..30e4d05 100644
--- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java
+++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java
@@ -64,6 +64,7 @@ public interface HttpServer {
     enum ApiType {
         HEARTBEAT,
         SERVER_CONFIG,
+        METRICS,
 
         // ledger
         DELETE_LEDGER,
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 a8c28c7..4815fee 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
@@ -315,6 +315,7 @@ public class Main {
             BKHttpServiceProvider provider = new BKHttpServiceProvider.Builder()
                 .setBookieServer(bookieService.getServer())
                 .setServerConfiguration(conf.getServerConf())
+                .setStatsProvider(statsProviderService.getStatsProvider())
                 .build();
             HttpService httpService =
                 new HttpService(provider, conf, rootStatsLogger);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
index 2ffe737..052b50e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java
@@ -49,10 +49,12 @@ import org.apache.bookkeeper.server.http.service.ListDiskFilesService;
 import org.apache.bookkeeper.server.http.service.ListLedgerService;
 import org.apache.bookkeeper.server.http.service.ListUnderReplicatedLedgerService;
 import org.apache.bookkeeper.server.http.service.LostBookieRecoveryDelayService;
+import org.apache.bookkeeper.server.http.service.MetricsService;
 import org.apache.bookkeeper.server.http.service.ReadLedgerEntryService;
 import org.apache.bookkeeper.server.http.service.RecoveryBookieService;
 import org.apache.bookkeeper.server.http.service.TriggerAuditService;
 import org.apache.bookkeeper.server.http.service.WhoIsAuditorService;
+import org.apache.bookkeeper.stats.StatsProvider;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -67,6 +69,7 @@ import org.apache.zookeeper.ZooKeeper;
 @Slf4j
 public class BKHttpServiceProvider implements HttpServiceProvider {
 
+    private final StatsProvider statsProvider;
     private final BookieServer bookieServer;
     private final AutoRecoveryMain autoRecovery;
     private final ServerConfiguration serverConf;
@@ -76,11 +79,13 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
 
     private BKHttpServiceProvider(BookieServer bookieServer,
                                   AutoRecoveryMain autoRecovery,
-                                  ServerConfiguration serverConf)
+                                  ServerConfiguration serverConf,
+                                  StatsProvider statsProvider)
         throws IOException, KeeperException, InterruptedException, BKException {
         this.bookieServer = bookieServer;
         this.autoRecovery = autoRecovery;
         this.serverConf = serverConf;
+        this.statsProvider = statsProvider;
         String zkServers = ZKMetadataDriverBase.resolveZkServers(serverConf);
         this.zk = ZooKeeperClient.newBuilder()
           .connectString(zkServers)
@@ -134,6 +139,7 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
         BookieServer bookieServer = null;
         AutoRecoveryMain autoRecovery = null;
         ServerConfiguration serverConf = null;
+        StatsProvider statsProvider = null;
 
         public Builder setBookieServer(BookieServer bookieServer) {
             this.bookieServer = bookieServer;
@@ -150,12 +156,18 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
             return this;
         }
 
+        public Builder setStatsProvider(StatsProvider statsProvider) {
+            this.statsProvider = statsProvider;
+            return this;
+        }
+
         public BKHttpServiceProvider build()
             throws IOException, KeeperException, InterruptedException, BKException {
             return new BKHttpServiceProvider(
                 bookieServer,
                 autoRecovery,
-                serverConf
+                serverConf,
+                statsProvider
             );
         }
     }
@@ -172,6 +184,8 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
                 return new HeartbeatService();
             case SERVER_CONFIG:
                 return new ConfigurationService(configuration);
+            case METRICS:
+                return new MetricsService(configuration, statsProvider);
 
             // ledger
             case DELETE_LEDGER:
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/MetricsService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/MetricsService.java
new file mode 100644
index 0000000..0d62ca9
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/MetricsService.java
@@ -0,0 +1,78 @@
+/*
+ * 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.http.service;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.http.HttpServer.Method;
+import org.apache.bookkeeper.http.HttpServer.StatusCode;
+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.stats.StatsProvider;
+
+/**
+ * HttpEndpointService that handle exposing metrics.
+ *
+ * <p>The GET method will return all the emtrics collected at stats provider.
+ */
+public class MetricsService implements HttpEndpointService {
+
+    private final ServerConfiguration conf;
+    private final StatsProvider statsProvider;
+
+    public MetricsService(ServerConfiguration conf,
+                          StatsProvider statsProvider) {
+        this.conf = conf;
+        this.statsProvider = statsProvider;
+    }
+
+    @Override
+    public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
+        HttpServiceResponse response = new HttpServiceResponse();
+        if (Method.GET != request.getMethod()) {
+            response.setCode(StatusCode.FORBIDDEN);
+            response.setBody(request.getMethod() + " is forbidden. Should be GET method");
+            return response;
+        }
+
+        if (null == statsProvider) {
+            response.setCode(StatusCode.INTERNAL_ERROR);
+            response.setBody("Stats provider is not enabled. Please enable it by set statsProviderClass"
+                + " on bookie configuration");
+            return response;
+        }
+
+        // GET
+        try (StringWriter writer = new StringWriter(1024)) {
+            statsProvider.writeAllMetrics(writer);
+            writer.flush();
+            response.setCode(StatusCode.OK);
+            response.setBody(writer.getBuffer().toString());
+        } catch (UnsupportedOperationException uoe) {
+            response.setCode(StatusCode.INTERNAL_ERROR);
+            response.setBody("Currently stats provider doesn't support exporting metrics in http service");
+        } catch (IOException ioe) {
+            response.setCode(StatusCode.INTERNAL_ERROR);
+            response.setBody("Exceptions are thrown when exporting metrics : " + ioe.getMessage());
+        }
+        return response;
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/MetricsServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/MetricsServiceTest.java
new file mode 100644
index 0000000..3f5e63d
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/MetricsServiceTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.http.service;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.CALLS_REAL_METHODS;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.io.Writer;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.http.HttpServer.Method;
+import org.apache.bookkeeper.http.HttpServer.StatusCode;
+import org.apache.bookkeeper.http.service.HttpServiceRequest;
+import org.apache.bookkeeper.http.service.HttpServiceResponse;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit test {@link MetricsService}.
+ */
+public class MetricsServiceTest {
+
+    private StatsProvider mockStatsProvider;
+    private MetricsService service;
+
+    @Before
+    public void setup() {
+        this.mockStatsProvider = mock(StatsProvider.class);
+        this.service = new MetricsService(new ServerConfiguration(), mockStatsProvider);
+    }
+
+    @Test
+    public void testForbiddenMethods() throws Exception {
+        HttpServiceRequest request = new HttpServiceRequest().setMethod(Method.PUT);
+        HttpServiceResponse response = service.handle(request);
+        assertEquals(StatusCode.FORBIDDEN.getValue(), response.getStatusCode());
+        assertEquals(
+            "PUT is forbidden. Should be GET method",
+            response.getBody());
+    }
+
+    @Test
+    public void testNullStatsProvider() throws Exception {
+        service = new MetricsService(new ServerConfiguration(), null);
+        HttpServiceRequest request = new HttpServiceRequest().setMethod(Method.GET);
+        HttpServiceResponse response = service.handle(request);
+        assertEquals(StatusCode.INTERNAL_ERROR.getValue(), response.getStatusCode());
+        assertEquals(
+            "Stats provider is not enabled. Please enable it by set statsProviderClass"
+                + " on bookie configuration",
+            response.getBody());
+    }
+
+    @Test
+    public void testWriteMetrics() throws Exception {
+        String content = "test-metrics";
+
+        doAnswer(invocationOnMock -> {
+            Writer writer = invocationOnMock.getArgument(0);
+            writer.write(content);
+            return null;
+        }).when(mockStatsProvider).writeAllMetrics(any(Writer.class));
+
+        HttpServiceRequest request = new HttpServiceRequest().setMethod(Method.GET);
+        HttpServiceResponse response = service.handle(request);
+
+        assertEquals(StatusCode.OK.getValue(), response.getStatusCode());
+        assertEquals(content, response.getBody());
+    }
+
+    @Test
+    public void testWriteMetricsException() throws Exception {
+        doThrow(new IOException("write-metrics-exception"))
+            .when(mockStatsProvider).writeAllMetrics(any(Writer.class));
+
+        HttpServiceRequest request = new HttpServiceRequest().setMethod(Method.GET);
+        HttpServiceResponse response = service.handle(request);
+
+        assertEquals(StatusCode.INTERNAL_ERROR.getValue(), response.getStatusCode());
+        assertEquals("Exceptions are thrown when exporting metrics : write-metrics-exception",
+            response.getBody());
+    }
+
+    @Test
+    public void testWriteMetricsUnimplemented() throws Exception {
+        mockStatsProvider = mock(StatsProvider.class, CALLS_REAL_METHODS);
+        service = new MetricsService(new ServerConfiguration(), mockStatsProvider);
+
+        HttpServiceRequest request = new HttpServiceRequest().setMethod(Method.GET);
+        HttpServiceResponse response = service.handle(request);
+
+        assertEquals(StatusCode.INTERNAL_ERROR.getValue(), response.getStatusCode());
+        assertEquals("Currently stats provider doesn't support exporting metrics in http service",
+            response.getBody());
+    }
+
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
index ee9b541..645a686 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
@@ -107,7 +107,9 @@ public class PrometheusMetricsProvider implements StatsProvider {
     @Override
     public void start(Configuration conf) {
         boolean httpEnabled = conf.getBoolean(PROMETHEUS_STATS_HTTP_ENABLE, DEFAULT_PROMETHEUS_STATS_HTTP_ENABLE);
-        if (httpEnabled) {
+        boolean bkHttpServerEnabled = conf.getBoolean("httpServerEnabled", false);
+        // only start its own http server when prometheus http is enabled and bk http server is not enabled.
+        if (httpEnabled && !bkHttpServerEnabled) {
             int httpPort = conf.getInt(PROMETHEUS_STATS_HTTP_PORT, DEFAULT_PROMETHEUS_STATS_HTTP_PORT);
             InetSocketAddress httpEndpoint = InetSocketAddress.createUnresolved("0.0.0.0", httpPort);
             this.server = new Server(httpEndpoint);
@@ -173,8 +175,8 @@ public class PrometheusMetricsProvider implements StatsProvider {
         return this.cachingStatsProvider.getStatsLogger(scope);
     }
 
-    @VisibleForTesting
-    void writeAllMetrics(Writer writer) throws IOException {
+    @Override
+    public void writeAllMetrics(Writer writer) throws IOException {
         PrometheusTextFormatUtil.writeMetricsCollectedByPrometheusClient(writer, registry);
 
         gauges.forEach((name, gauge) -> PrometheusTextFormatUtil.writeGauge(writer, name, gauge));
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
index a69ac42..8e90e04 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 
+import lombok.Cleanup;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -65,6 +66,16 @@ public class TestPrometheusMetricsProvider {
     }
 
     @Test
+    public void testStartNoHttpWhenBkHttpEnabled() {
+        PropertiesConfiguration config = new PropertiesConfiguration();
+        config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
+        config.setProperty("httpServerEnabled", true);
+        @Cleanup("stop") PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+        provider.start(config);
+        assertNull(provider.server);
+    }
+
+    @Test
     public void testStartWithHttp() {
         PropertiesConfiguration config = new PropertiesConfiguration();
         config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
index 6449e1f..b6e3460 100644
--- a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
+++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
@@ -16,6 +16,8 @@
  */
 package org.apache.bookkeeper.stats;
 
+import java.io.IOException;
+import java.io.Writer;
 import org.apache.commons.configuration.Configuration;
 
 /**
@@ -36,6 +38,15 @@ public interface StatsProvider {
     void stop();
 
     /**
+     *
+     * @param writer
+     * @throws IOException
+     */
+    default void writeAllMetrics(Writer writer) throws IOException {
+        throw new UnsupportedOperationException("writeAllMetrics is not implemented yet");
+    }
+
+    /**
      * Return the stats logger to a given <i>scope</i>.
      * @param scope
      *          Scope for the given stats

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

[bookkeeper] 03/06: Add a test docker image for current version only

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ba51575a4962f634eab826e79c83abcd48cef86b
Author: Sijie Guo <si...@apache.org>
AuthorDate: Thu May 17 01:39:04 2018 -0700

    Add a test docker image for current version only
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    The current test images don't use any scripts that shipped with official docker image.
    We basically don't have any test coverage on the official docker image we shipped for each release.
    
    *Solution*
    
    Introduce a `bookkeeper-current` image representing an image for latest master.
    
    Use `apachebookkeeper/all-versions-image` for BC tests, use `apachebookkeeper/bookkeeper-current` as tests only needs latest master image.
    
    Add an integration test to test standalone mode using the `bookkeeper-current` image
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Jia Zhai <None>
    
    This closes #1408 from sijie/docker_dev
---
 pom.xml                                            |   7 +
 .../docker-images/current-version-image/Dockerfile |  50 +++++++
 tests/docker-images/current-version-image/pom.xml  | 125 ++++++++++++++++++
 tests/docker-images/pom.xml                        |   1 +
 tests/integration-tests-topologies/pom.xml         |  14 +-
 .../tests/containers/BKStandaloneContainer.java    | 145 +++++++++++++++++++++
 tests/integration/pom.xml                          |  42 +++++-
 tests/integration/smoke/pom.xml                    |  21 +++
 tests/integration/standalone/pom.xml               |  63 +++++++++
 .../integration/standalone/StandaloneTest.java     |  53 ++++++++
 .../standalone/src/test/resources/log4j.properties |  42 ++++++
 11 files changed, 561 insertions(+), 2 deletions(-)

diff --git a/pom.xml b/pom.xml
index 5472f2b..f145309 100644
--- a/pom.xml
+++ b/pom.xml
@@ -150,6 +150,7 @@
     <shrinkwrap.version>3.0.1</shrinkwrap.version>
     <slf4j.version>1.7.25</slf4j.version>
     <spotbugs-annotations.version>3.1.1</spotbugs-annotations.version>
+    <testcontainers.version>1.7.0</testcontainers.version>
     <twitter-server.version>1.29.0</twitter-server.version>
     <vertx.version>3.4.1</vertx.version>
     <zookeeper.version>3.5.3-beta</zookeeper.version>
@@ -160,6 +161,7 @@
     <dockerfile-maven-plugin.version>1.3.7</dockerfile-maven-plugin.version>
     <license-maven-plugin.version>1.6</license-maven-plugin.version>
     <jacoco-maven-plugin.version>0.8.0</jacoco-maven-plugin.version>
+    <maven-antrun-plugin.version>1.8</maven-antrun-plugin.version>
     <maven-assembly-plugin.version>3.1.0</maven-assembly-plugin.version>
     <maven-bundle-plugin.version>3.2.0</maven-bundle-plugin.version>
     <maven-checkstyle-plugin.version>3.0.0</maven-checkstyle-plugin.version>
@@ -590,6 +592,11 @@
         <artifactId>shrinkwrap-resolver-api</artifactId>
         <version>${shrinkwrap.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.testcontainers</groupId>
+        <artifactId>testcontainers</artifactId>
+        <version>${testcontainers.version}</version>
+      </dependency>
 
       <!-- benchmark dependencies -->
       <dependency>
diff --git a/tests/docker-images/current-version-image/Dockerfile b/tests/docker-images/current-version-image/Dockerfile
new file mode 100644
index 0000000..285ecf6
--- /dev/null
+++ b/tests/docker-images/current-version-image/Dockerfile
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+FROM centos:7
+MAINTAINER Apache BookKeeper <de...@bookkeeper.apache.org>
+
+ARG BK_VERSION=DOESNOTEXISTS
+ARG DISTRO_NAME=bookkeeper-dist-server-${BK_VERSION}-bin
+ARG PKG_NAME=bookkeeper-server-${BK_VERSION}
+
+ENV BOOKIE_PORT=3181
+EXPOSE $BOOKIE_PORT
+ENV BK_USER=bookkeeper
+
+# prepare utils
+RUN set -x \
+    && adduser "${BK_USER}" \
+    && yum install -y java-1.8.0-openjdk-headless wget bash python sudo netcat \
+    && mkdir -pv /opt \
+    && cd /opt \
+    && yum clean all
+
+# untar tarballs
+ADD target/${DISTRO_NAME}.tar.gz /opt
+RUN mv /opt/${PKG_NAME} /opt/bookkeeper
+
+WORKDIR /opt/bookkeeper
+
+COPY target/scripts/apply-config-from-env.py target/scripts/entrypoint.sh target/scripts/healthcheck.sh /opt/bookkeeper/
+
+ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
+CMD ["/opt/bookkeepr/bin/bookkeeper", "bookie"]
+
+HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/healthcheck.sh
diff --git a/tests/docker-images/current-version-image/pom.xml b/tests/docker-images/current-version-image/pom.xml
new file mode 100644
index 0000000..5d2dbaf
--- /dev/null
+++ b/tests/docker-images/current-version-image/pom.xml
@@ -0,0 +1,125 @@
+<!--
+   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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>org.apache.bookkeeper.tests</groupId>
+    <artifactId>docker-images</artifactId>
+    <version>4.8.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>org.apache.bookkeeper.tests</groupId>
+  <artifactId>current-version-image</artifactId>
+  <name>Apache BookKeeper :: Tests :: Docker Images :: Current Version</name>
+  <packaging>pom</packaging>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-dist-server</artifactId>
+      <version>${project.parent.version}</version>
+      <classifier>bin</classifier>
+      <type>tar.gz</type>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+  <profiles>
+    <profile>
+      <id>docker</id>
+      <activation>
+        <property>
+          <name>integrationTests</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>com.spotify</groupId>
+            <artifactId>dockerfile-maven-plugin</artifactId>
+            <version>${dockerfile-maven-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>default</id>
+                <goals>
+                  <goal>build</goal>
+                </goals>
+              </execution>
+              <execution>
+                <id>add-latest-tag</id>
+                <goals>
+                  <goal>tag</goal>
+                </goals>
+                <configuration>
+                  <repository>apachebookkeeper/bookkeeper-current</repository>
+                  <tag>latest</tag>
+                </configuration>
+              </execution>
+            </executions>
+            <configuration>
+              <repository>apachebookkeeper/bookkeeper-current</repository>
+              <tag>${project.version}</tag>
+              <pullNewerImage>false</pullNewerImage>
+              <noCache>true</noCache>
+              <buildArgs>
+                <BK_VERSION>${project.version}</BK_VERSION>
+              </buildArgs>
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <version>${maven-dependency-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>copy-docker-dependencies</id>
+                <goals>
+                  <goal>copy-dependencies</goal>
+                </goals>
+                <phase>generate-resources</phase>
+                <configuration>
+                  <outputDirectory>${project.build.directory}/</outputDirectory>
+                  <includeArtifactIds>bookkeeper-dist-server</includeArtifactIds>
+                  <excludeTransitive>true</excludeTransitive>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <!-- this task is used for copy docker scripts to build docker image -->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <version>${maven-antrun-plugin.version}</version>
+            <executions>
+              <execution>
+                <phase>generate-resources</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <tasks>
+                    <echo>copying docker scripts</echo>
+                    <mkdir dir="${basedir}/target/scripts"/>
+                    <copydir src="${basedir}/../../../docker/scripts" dest="${basedir}/target/scripts"/>
+                  </tasks>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml
index db18943..88162c3 100644
--- a/tests/docker-images/pom.xml
+++ b/tests/docker-images/pom.xml
@@ -29,5 +29,6 @@
   <modules>
     <module>all-released-versions-image</module>
     <module>all-versions-image</module>
+    <module>current-version-image</module>
   </modules>
 </project>
diff --git a/tests/integration-tests-topologies/pom.xml b/tests/integration-tests-topologies/pom.xml
index c6bc68e..000059f 100644
--- a/tests/integration-tests-topologies/pom.xml
+++ b/tests/integration-tests-topologies/pom.xml
@@ -27,6 +27,18 @@
   <artifactId>integration-tests-topologies</artifactId>
   <packaging>jar</packaging>
 
-  <name>Apache BookKeeper :: Tests :: Common topologies for Arquillian based integration tests</name>
+  <name>Apache BookKeeper :: Tests :: Common topologies for Docker based integration tests</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.testcontainers</groupId>
+      <artifactId>testcontainers</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
 
 </project>
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
new file mode 100644
index 0000000..6ced634
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
@@ -0,0 +1,145 @@
+/*
+ * 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.tests.containers;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.time.temporal.ChronoUnit.SECONDS;
+
+import com.github.dockerjava.api.command.LogContainerCmd;
+import com.github.dockerjava.api.model.Frame;
+import com.github.dockerjava.core.command.LogContainerResultCallback;
+import java.time.Duration;
+import java.util.Objects;
+import lombok.EqualsAndHashCode;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang.StringUtils;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
+
+/**
+ * Test Container for Bookies.
+ */
+@Slf4j
+public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> extends GenericContainer<SELF> {
+
+    private static final int ZK_PORT = 2181;
+    private static final int BOOKIE_BASE_PORT = 3181;
+
+    private static final String IMAGE_NAME = "apachebookkeeper/bookkeeper-current:latest";
+
+    private static final String STANDALONE_HOST_NAME = "standalone";
+    private static final String CONTAINER_NAME_BASE = "bk-standalone-test";
+
+    private final String containerName;
+    private final int numBookies;
+
+    public BKStandaloneContainer(String containerName, int numBookies) {
+        super(IMAGE_NAME);
+        this.containerName = containerName;
+        this.numBookies = numBookies;
+    }
+
+    @Override
+    public String getContainerName() {
+        return CONTAINER_NAME_BASE + "-" + containerName + "-" + numBookies + "-bookies-" + System.currentTimeMillis();
+    }
+
+    public String getContainerLog() {
+        StringBuilder sb = new StringBuilder();
+
+        LogContainerCmd logContainerCmd = this.dockerClient.logContainerCmd(containerId);
+        logContainerCmd.withStdOut(true).withStdErr(true);
+        try {
+            logContainerCmd.exec(new LogContainerResultCallback() {
+                @Override
+                public void onNext(Frame item) {
+                    sb.append(new String(item.getPayload(), UTF_8));
+                }
+            }).awaitCompletion();
+        } catch (InterruptedException e) {
+
+        }
+        return sb.toString();
+    }
+
+    public ExecResult execCmd(String... cmd) throws Exception {
+        String cmdString = StringUtils.join(cmd, " ");
+
+        log.info("DOCKER.exec({}:{}): Executing ...", containerId, cmdString);
+
+        ExecResult result = execInContainer(cmd);
+
+        log.info("Docker.exec({}:{}): Done", containerId, cmdString);
+        log.info("Docker.exec({}:{}): Stdout -\n{}", containerId, cmdString, result.getStdout());
+        log.info("Docker.exec({}:{}): Stderr -\n{}", containerId, cmdString, result.getStderr());
+
+        return result;
+    }
+
+    @Override
+    protected void configure() {
+        addExposedPorts(
+            ZK_PORT
+        );
+        for (int i = 0; i < numBookies; i++) {
+            addExposedPort(BOOKIE_BASE_PORT + i);
+        }
+        setCommand(
+            "/opt/bookkeeper/bin/bookkeeper",
+            "localbookie",
+            "" + numBookies
+        );
+        addEnv("JAVA_HOME", "/usr/lib/jvm/jre-1.8.0");
+    }
+
+    @Override
+    public void start() {
+        this.waitStrategy = new LogMessageWaitStrategy()
+            .withRegEx(".*ForceWrite Thread started.*\\s")
+            .withTimes(numBookies)
+            .withStartupTimeout(Duration.of(60, SECONDS));
+        this.withCreateContainerCmdModifier(createContainerCmd -> {
+            createContainerCmd.withHostName(STANDALONE_HOST_NAME);
+            createContainerCmd.withName(getContainerName());
+            createContainerCmd.withEntrypoint("/bin/bash");
+        });
+
+        super.start();
+        log.info("Start a standalone bookkeeper cluster at container {}", containerName);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof BKStandaloneContainer)) {
+            return false;
+        }
+
+        BKStandaloneContainer another = (BKStandaloneContainer) o;
+        return containerName.equals(another.containerName)
+            && numBookies == another.numBookies
+            && super.equals(another);
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * super.hashCode() + Objects.hash(
+            containerName,
+            numBookies);
+    }
+}
diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml
index 025a420..dbd061d 100644
--- a/tests/integration/pom.xml
+++ b/tests/integration/pom.xml
@@ -22,12 +22,52 @@
     <groupId>org.apache.bookkeeper.tests</groupId>
     <artifactId>integration-tests-base</artifactId>
     <version>4.7.1-SNAPSHOT</version>
-    <relativePath>../integration-tests-base</relativePath>
+    <relativePath>..</relativePath>
   </parent>
   <groupId>org.apache.bookkeeper.tests.integration</groupId>
   <artifactId>tests-parent</artifactId>
   <name>Apache BookKeeper :: Tests :: Integration</name>
   <modules>
     <module>smoke</module>
+    <module>standalone</module>
   </modules>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <!-- only run tests when -DintegrationTests is specified //-->
+          <skipTests>true</skipTests>
+          <systemPropertyVariables>
+            <currentVersion>${project.version}</currentVersion>
+            <maven.buildDirectory>${project.build.directory}</maven.buildDirectory>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>integrationTests</id>
+      <activation>
+        <property>
+          <name>integrationTests</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <skipTests>false</skipTests>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
 </project>
diff --git a/tests/integration/smoke/pom.xml b/tests/integration/smoke/pom.xml
index adab24e..2e707a4 100644
--- a/tests/integration/smoke/pom.xml
+++ b/tests/integration/smoke/pom.xml
@@ -33,6 +33,27 @@
       <groupId>org.apache.bookkeeper</groupId>
       <artifactId>bookkeeper-server</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.bookkeeper.tests</groupId>
+      <artifactId>integration-tests-utils</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.bookkeeper.tests</groupId>
+      <artifactId>integration-tests-topologies</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.jboss.arquillian.junit</groupId>
+      <artifactId>arquillian-junit-standalone</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 
diff --git a/tests/integration/standalone/pom.xml b/tests/integration/standalone/pom.xml
new file mode 100644
index 0000000..ce512a9
--- /dev/null
+++ b/tests/integration/standalone/pom.xml
@@ -0,0 +1,63 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation=" http://maven.apache.org/POM/4.0.0   http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper.tests.integration</groupId>
+    <artifactId>tests-parent</artifactId>
+    <version>4.8.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.bookkeeper.tests.integration</groupId>
+  <artifactId>standalone</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Integration :: Standalone test</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-server</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.bookkeeper.tests</groupId>
+      <artifactId>integration-tests-topologies</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <!-- smoke test should never flake //-->
+          <rerunFailingTestsCount>0</rerunFailingTestsCount>
+          <redirectTestOutputToFile>${redirectTestOutputToFile}</redirectTestOutputToFile>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
new file mode 100644
index 0000000..df082d4
--- /dev/null
+++ b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tests.integration.standalone;
+
+import static org.junit.Assert.assertTrue;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.containers.BKStandaloneContainer;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.testcontainers.containers.Container.ExecResult;
+
+/**
+ * A simple test to cover running current docker image in standalone mode.
+ */
+@Slf4j
+public class StandaloneTest {
+
+    @ClassRule
+    public static BKStandaloneContainer bkContainer = new BKStandaloneContainer<>("integrationtest", 3);
+
+    @Test
+    public void runSimpleTest() throws Exception {
+        ExecResult result = bkContainer.execCmd(
+            "/opt/bookkeeper/bin/bookkeeper",
+            "shell",
+            "simpletest",
+            "-ensemble", "3",
+            "-writeQuorum", "3",
+            "-ackQuorum", "2",
+            "-numEntries", "100"
+        );
+        assertTrue(
+            result.getStdout().contains("100 entries written to ledger"));
+    }
+
+}
diff --git a/tests/integration/standalone/src/test/resources/log4j.properties b/tests/integration/standalone/src/test/resources/log4j.properties
new file mode 100644
index 0000000..10ae6bf
--- /dev/null
+++ b/tests/integration/standalone/src/test/resources/log4j.properties
@@ -0,0 +1,42 @@
+#
+#
+# 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.
+#
+#
+
+#
+# Bookkeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only, level INFO
+bookkeeper.root.logger=INFO,CONSOLE
+log4j.rootLogger=${bookkeeper.root.logger}
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+log4j.logger.org.apache.bookkeeper.bookie=INFO
+log4j.logger.org.apache.bookkeeper.meta=INFO

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

[bookkeeper] 04/06: Add a docker based `BookKeeperClusterTestBase` for failure related integration tests

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 272b8e78303b544436fde98a4a2350137fede521
Author: Sijie Guo <si...@apache.org>
AuthorDate: Fri May 18 12:49:31 2018 -0700

    Add a docker based `BookKeeperClusterTestBase` for failure related integration tests
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    Currently we don't have any failure related testing for table service. Since we are using docker as the integration testing infrastructure,
    It is better to use container for those failure testings, rather than going down the path as what we did before.
    
    *Solution*
    
    This change provides the basic test base for bookkeeper cluster using dockers. `BookKeeperClusterTestBase` provides the similar functionalities
    to start/stop bookies as what we did in the unit test.
    
    `bookkeeper/tests/containers` in `integration-tests-topologies` provides all the basic containers used for testing.
    `tests/integration/cluster` and `tests/integration/topologies` provides the test base for writing tests using dockerized bookkeeper cluster.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>
    
    This closes #1412 from sijie/docker_cluster_dev
---
 conf/bk_server.conf                                |   7 +-
 .../docker-images/current-version-image/Dockerfile |   2 +-
 tests/integration-tests-topologies/pom.xml         |   5 +
 .../tests/containers/BKStandaloneContainer.java    |  50 +----
 .../tests/containers/BookieContainer.java          | 116 +++++++++++
 ...tandaloneContainer.java => ChaosContainer.java} |  93 +++------
 .../tests/containers/MetadataStoreContainer.java   |  33 ++++
 .../bookkeeper/tests/containers/ZKContainer.java   |  62 ++++++
 .../tests/containers/wait/HttpWaitStrategy.java    | 215 +++++++++++++++++++++
 .../tests/containers/wait/ZKWaitStrategy.java      |  63 ++++++
 .../bookkeeper/tests/BookKeeperClusterUtils.java   |   5 +-
 tests/integration/cluster/pom.xml                  |  70 +++++++
 .../cluster/BookKeeperClusterTestBase.java         |  94 +++++++++
 .../integration/cluster/SimpleClusterTest.java     |  78 ++++++++
 .../tests/integration/topologies/BKCluster.java    | 156 +++++++++++++++
 .../cluster/src/test/resources/log4j.properties    |  42 ++++
 tests/integration/pom.xml                          |   1 +
 17 files changed, 981 insertions(+), 111 deletions(-)

diff --git a/conf/bk_server.conf b/conf/bk_server.conf
index 946e6af..2080b68 100755
--- a/conf/bk_server.conf
+++ b/conf/bk_server.conf
@@ -191,10 +191,13 @@ bookiePort=3181
 #############################################################################
 
 # The flag enables/disables starting the admin http server. Default value is 'false'.
-# httpServerEnabled=false
+httpServerEnabled=false
 
 # The http server port to listen on. Default value is 8080.
-# httpServerPort=8080
+httpServerPort=8080
+
+# The http server class
+httpServerClass=org.apache.bookkeeper.http.vertx.VertxHttpServer
 
 ############################################## Security ##############################################
 
diff --git a/tests/docker-images/current-version-image/Dockerfile b/tests/docker-images/current-version-image/Dockerfile
index 285ecf6..4006daf 100644
--- a/tests/docker-images/current-version-image/Dockerfile
+++ b/tests/docker-images/current-version-image/Dockerfile
@@ -45,6 +45,6 @@ WORKDIR /opt/bookkeeper
 COPY target/scripts/apply-config-from-env.py target/scripts/entrypoint.sh target/scripts/healthcheck.sh /opt/bookkeeper/
 
 ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
-CMD ["/opt/bookkeepr/bin/bookkeeper", "bookie"]
+CMD ["/opt/bookkeeper/bin/bookkeeper", "bookie"]
 
 HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/healthcheck.sh
diff --git a/tests/integration-tests-topologies/pom.xml b/tests/integration-tests-topologies/pom.xml
index 000059f..b5353eb 100644
--- a/tests/integration-tests-topologies/pom.xml
+++ b/tests/integration-tests-topologies/pom.xml
@@ -39,6 +39,11 @@
       <artifactId>junit</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper.tests</groupId>
+      <artifactId>integration-tests-utils</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
index 6ced634..6efec4c 100644
--- a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
@@ -18,25 +18,18 @@
 
 package org.apache.bookkeeper.tests.containers;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.time.temporal.ChronoUnit.SECONDS;
 
-import com.github.dockerjava.api.command.LogContainerCmd;
-import com.github.dockerjava.api.model.Frame;
-import com.github.dockerjava.core.command.LogContainerResultCallback;
 import java.time.Duration;
 import java.util.Objects;
-import lombok.EqualsAndHashCode;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.commons.lang.StringUtils;
-import org.testcontainers.containers.GenericContainer;
 import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
 
 /**
  * Test Container for Bookies.
  */
 @Slf4j
-public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> extends GenericContainer<SELF> {
+public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> extends ChaosContainer<SELF> {
 
     private static final int ZK_PORT = 2181;
     private static final int BOOKIE_BASE_PORT = 3181;
@@ -44,52 +37,17 @@ public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> ext
     private static final String IMAGE_NAME = "apachebookkeeper/bookkeeper-current:latest";
 
     private static final String STANDALONE_HOST_NAME = "standalone";
-    private static final String CONTAINER_NAME_BASE = "bk-standalone-test";
 
-    private final String containerName;
     private final int numBookies;
 
-    public BKStandaloneContainer(String containerName, int numBookies) {
-        super(IMAGE_NAME);
-        this.containerName = containerName;
+    public BKStandaloneContainer(String clusterName, int numBookies) {
+        super(clusterName, IMAGE_NAME);
         this.numBookies = numBookies;
     }
 
     @Override
     public String getContainerName() {
-        return CONTAINER_NAME_BASE + "-" + containerName + "-" + numBookies + "-bookies-" + System.currentTimeMillis();
-    }
-
-    public String getContainerLog() {
-        StringBuilder sb = new StringBuilder();
-
-        LogContainerCmd logContainerCmd = this.dockerClient.logContainerCmd(containerId);
-        logContainerCmd.withStdOut(true).withStdErr(true);
-        try {
-            logContainerCmd.exec(new LogContainerResultCallback() {
-                @Override
-                public void onNext(Frame item) {
-                    sb.append(new String(item.getPayload(), UTF_8));
-                }
-            }).awaitCompletion();
-        } catch (InterruptedException e) {
-
-        }
-        return sb.toString();
-    }
-
-    public ExecResult execCmd(String... cmd) throws Exception {
-        String cmdString = StringUtils.join(cmd, " ");
-
-        log.info("DOCKER.exec({}:{}): Executing ...", containerId, cmdString);
-
-        ExecResult result = execInContainer(cmd);
-
-        log.info("Docker.exec({}:{}): Done", containerId, cmdString);
-        log.info("Docker.exec({}:{}): Stdout -\n{}", containerId, cmdString, result.getStdout());
-        log.info("Docker.exec({}:{}): Stderr -\n{}", containerId, cmdString, result.getStderr());
-
-        return result;
+        return clusterName + "-standalone-" + numBookies + "-bookies";
     }
 
     @Override
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BookieContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BookieContainer.java
new file mode 100644
index 0000000..874ec2a
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BookieContainer.java
@@ -0,0 +1,116 @@
+/*
+ * 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.tests.containers;
+
+import static java.time.temporal.ChronoUnit.SECONDS;
+
+import java.net.URI;
+import java.time.Duration;
+import java.util.Objects;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.containers.wait.HttpWaitStrategy;
+
+/**
+ * Test Container for Bookies.
+ */
+@Slf4j
+public class BookieContainer<SELF extends BookieContainer<SELF>> extends ChaosContainer<SELF> {
+
+    private static final int BOOKIE_PORT = 3181;
+    private static final int BOOKIE_GRPC_PORT = 4181; // stream storage grpc port
+    private static final int BOOKIE_HTTP_PORT = 8080;
+
+    private static final String IMAGE_NAME = "apachebookkeeper/bookkeeper-current:latest";
+
+    private final String hostname;
+    private final String metadataServiceUri;
+
+    public BookieContainer(String clusterName,
+                           String hostname,
+                           String metadataServiceUri) {
+        super(clusterName, IMAGE_NAME);
+        this.hostname = hostname;
+        this.metadataServiceUri = metadataServiceUri;
+    }
+
+    @Override
+    public String getContainerName() {
+        return clusterName + "-" + hostname;
+    }
+
+    @Override
+    protected void configure() {
+        addExposedPorts(
+            BOOKIE_PORT,
+            BOOKIE_GRPC_PORT,
+            BOOKIE_HTTP_PORT
+        );
+        addEnv("BK_httpServerEnabled", "true");
+        addEnv("BK_httpServerPort", "" + BOOKIE_HTTP_PORT);
+        addEnv("BK_metadataServiceUri", metadataServiceUri);
+        addEnv("BK_useHostNameAsBookieID", "true");
+        if (metadataServiceUri.toLowerCase().startsWith("zk")) {
+            URI uri = URI.create(metadataServiceUri);
+            addEnv("BK_zkServers", uri.getAuthority());
+            addEnv("BK_zkLedgersRootPath", uri.getPath());
+        }
+    }
+
+    @Override
+    public void start() {
+        this.waitStrategy = new HttpWaitStrategy()
+            .forPath("/heartbeat")
+            .forStatusCode(200)
+            .forPort(BOOKIE_HTTP_PORT)
+            .withStartupTimeout(Duration.of(60, SECONDS));
+        this.withCreateContainerCmdModifier(createContainerCmd -> {
+            createContainerCmd.withHostName(hostname);
+            createContainerCmd.withName(getContainerName());
+        });
+
+        super.start();
+        log.info("Started bookie {} at cluster {}", hostname, clusterName);
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        log.info("Stopped bookie {} at cluster {}", hostname, clusterName);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof BookieContainer)) {
+            return false;
+        }
+
+        BookieContainer another = (BookieContainer) o;
+        return hostname.equals(another.hostname)
+            && metadataServiceUri.equals(another.metadataServiceUri)
+            && super.equals(another);
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * super.hashCode() + Objects.hash(
+            hostname,
+            metadataServiceUri);
+    }
+
+}
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ChaosContainer.java
similarity index 52%
copy from tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
copy to tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ChaosContainer.java
index 6ced634..33530f1 100644
--- a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ChaosContainer.java
@@ -19,45 +19,49 @@
 package org.apache.bookkeeper.tests.containers;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.time.temporal.ChronoUnit.SECONDS;
 
 import com.github.dockerjava.api.command.LogContainerCmd;
 import com.github.dockerjava.api.model.Frame;
 import com.github.dockerjava.core.command.LogContainerResultCallback;
-import java.time.Duration;
 import java.util.Objects;
-import lombok.EqualsAndHashCode;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.lang.StringUtils;
 import org.testcontainers.containers.GenericContainer;
-import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
 
 /**
- * Test Container for Bookies.
+ * A base container provides chaos capability.
  */
 @Slf4j
-public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> extends GenericContainer<SELF> {
+public class ChaosContainer<SELF extends ChaosContainer<SELF>> extends GenericContainer<SELF> {
 
-    private static final int ZK_PORT = 2181;
-    private static final int BOOKIE_BASE_PORT = 3181;
+    protected final String clusterName;
 
-    private static final String IMAGE_NAME = "apachebookkeeper/bookkeeper-current:latest";
-
-    private static final String STANDALONE_HOST_NAME = "standalone";
-    private static final String CONTAINER_NAME_BASE = "bk-standalone-test";
-
-    private final String containerName;
-    private final int numBookies;
-
-    public BKStandaloneContainer(String containerName, int numBookies) {
-        super(IMAGE_NAME);
-        this.containerName = containerName;
-        this.numBookies = numBookies;
+    protected ChaosContainer(String clusterName, String image) {
+        super(image);
+        this.clusterName = clusterName;
     }
 
-    @Override
-    public String getContainerName() {
-        return CONTAINER_NAME_BASE + "-" + containerName + "-" + numBookies + "-bookies-" + System.currentTimeMillis();
+    public void tailContainerLog() {
+        CompletableFuture.runAsync(() -> {
+            while(null == containerId) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(100);
+                } catch (InterruptedException e) {
+                    return;
+                }
+            }
+
+            LogContainerCmd logContainerCmd = this.dockerClient.logContainerCmd(containerId);
+            logContainerCmd.withStdOut(true).withStdErr(true).withFollowStream(true);
+            logContainerCmd.exec(new LogContainerResultCallback() {
+                @Override
+                public void onNext(Frame item) {
+                    log.info(new String(item.getPayload(), UTF_8));
+                }
+            });
+        });
     }
 
     public String getContainerLog() {
@@ -93,53 +97,20 @@ public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> ext
     }
 
     @Override
-    protected void configure() {
-        addExposedPorts(
-            ZK_PORT
-        );
-        for (int i = 0; i < numBookies; i++) {
-            addExposedPort(BOOKIE_BASE_PORT + i);
-        }
-        setCommand(
-            "/opt/bookkeeper/bin/bookkeeper",
-            "localbookie",
-            "" + numBookies
-        );
-        addEnv("JAVA_HOME", "/usr/lib/jvm/jre-1.8.0");
-    }
-
-    @Override
-    public void start() {
-        this.waitStrategy = new LogMessageWaitStrategy()
-            .withRegEx(".*ForceWrite Thread started.*\\s")
-            .withTimes(numBookies)
-            .withStartupTimeout(Duration.of(60, SECONDS));
-        this.withCreateContainerCmdModifier(createContainerCmd -> {
-            createContainerCmd.withHostName(STANDALONE_HOST_NAME);
-            createContainerCmd.withName(getContainerName());
-            createContainerCmd.withEntrypoint("/bin/bash");
-        });
-
-        super.start();
-        log.info("Start a standalone bookkeeper cluster at container {}", containerName);
-    }
-
-    @Override
     public boolean equals(Object o) {
-        if (!(o instanceof BKStandaloneContainer)) {
+        if (!(o instanceof ChaosContainer)) {
             return false;
         }
 
-        BKStandaloneContainer another = (BKStandaloneContainer) o;
-        return containerName.equals(another.containerName)
-            && numBookies == another.numBookies
+        ChaosContainer another = (ChaosContainer) o;
+        return clusterName.equals(another.clusterName)
             && super.equals(another);
     }
 
     @Override
     public int hashCode() {
         return 31 * super.hashCode() + Objects.hash(
-            containerName,
-            numBookies);
+            clusterName);
     }
+
 }
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/MetadataStoreContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/MetadataStoreContainer.java
new file mode 100644
index 0000000..ec8d914
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/MetadataStoreContainer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.containers;
+
+/**
+ * An abstract class for metadata store container.
+ */
+public abstract class MetadataStoreContainer<SELF extends MetadataStoreContainer<SELF>> extends ChaosContainer<SELF> {
+    protected MetadataStoreContainer(String clusterName, String image) {
+        super(clusterName, image);
+    }
+
+    public abstract String getExternalServiceUri();
+
+    public abstract String getInternalServiceUri();
+
+}
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
new file mode 100644
index 0000000..ba1d384
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
@@ -0,0 +1,62 @@
+/*
+ * 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.tests.containers;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.containers.wait.ZKWaitStrategy;
+
+@Slf4j
+public class ZKContainer<SELF extends ZKContainer<SELF>> extends MetadataStoreContainer<SELF> {
+
+    private static final int ZK_PORT = 2181;
+
+    private static final String IMAGE_NAME = "zookeeper:3.4.11";
+    public static final String HOST_NAME = "metadata-store";
+    public static final String SERVICE_URI = "zk://" + HOST_NAME + ":" + ZK_PORT + "/ledgers";
+
+    public ZKContainer(String clusterName) {
+        super(clusterName, IMAGE_NAME);
+    }
+
+    @Override
+    public String getExternalServiceUri() {
+        return "zk://" + getContainerIpAddress() + ":" + getMappedPort(ZK_PORT) + "/ledgers";
+    }
+
+    @Override
+    public String getInternalServiceUri() {
+        return SERVICE_URI;
+    }
+
+    @Override
+    protected void configure() {
+        addExposedPort(ZK_PORT);
+    }
+
+    @Override
+    public void start() {
+        this.waitStrategy = new ZKWaitStrategy(ZK_PORT);
+        this.withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd.withHostName(HOST_NAME));
+
+        super.start();
+        log.info("Start a zookeeper server at container {} : external service uri = {}, internal service uri = {}",
+            containerName, getExternalServiceUri(), getInternalServiceUri());
+    }
+
+}
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/HttpWaitStrategy.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/HttpWaitStrategy.java
new file mode 100644
index 0000000..e70140a
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/HttpWaitStrategy.java
@@ -0,0 +1,215 @@
+/*
+ * 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.tests.containers.wait;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.rnorth.ducttape.unreliables.Unreliables.retryUntilSuccess;
+
+import com.google.common.base.Strings;
+import com.google.common.io.BaseEncoding;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import org.rnorth.ducttape.TimeoutException;
+import org.testcontainers.containers.ContainerLaunchException;
+import org.testcontainers.containers.wait.strategy.AbstractWaitStrategy;
+
+/**
+ * Http Strategy
+ */
+public class HttpWaitStrategy extends AbstractWaitStrategy {
+    @java.lang.SuppressWarnings("all")
+    private static final org.slf4j.Logger log = org.slf4j.LoggerFactory.getLogger(HttpWaitStrategy.class);
+    /**
+     * Authorization HTTP header.
+     */
+    private static final String HEADER_AUTHORIZATION = "Authorization";
+    /**
+     * Basic Authorization scheme prefix.
+     */
+    private static final String AUTH_BASIC = "Basic ";
+    private String path = "/";
+    private int statusCode = HttpURLConnection.HTTP_OK;
+    private boolean tlsEnabled;
+    private String username;
+    private String password;
+    private Predicate<String> responsePredicate;
+    private int port = 80;
+
+    /**
+     * Waits for the given status code.
+     *
+     * @param statusCode the expected status code
+     * @return this
+     */
+    public HttpWaitStrategy forStatusCode(int statusCode) {
+        this.statusCode = statusCode;
+        return this;
+    }
+
+    /**
+     * Waits for the given path.
+     *
+     * @param path the path to check
+     * @return this
+     */
+    public HttpWaitStrategy forPath(String path) {
+        this.path = path;
+        return this;
+    }
+
+    /**
+     * Wait for the given port.
+     *
+     * @param port the given port
+     * @return this
+     */
+    public HttpWaitStrategy forPort(int port) {
+        this.port = port;
+        return this;
+    }
+
+    /**
+     * Indicates that the status check should use HTTPS.
+     *
+     * @return this
+     */
+    public HttpWaitStrategy usingTls() {
+        this.tlsEnabled = true;
+        return this;
+    }
+
+    /**
+     * Authenticate with HTTP Basic Authorization credentials.
+     *
+     * @param username the username
+     * @param password the password
+     * @return this
+     */
+    public HttpWaitStrategy withBasicCredentials(String username, String password) {
+        this.username = username;
+        this.password = password;
+        return this;
+    }
+
+    /**
+     * Waits for the response to pass the given predicate
+     * @param responsePredicate The predicate to test the response against
+     * @return this
+     */
+    public HttpWaitStrategy forResponsePredicate(Predicate<String> responsePredicate) {
+        this.responsePredicate = responsePredicate;
+        return this;
+    }
+
+    @Override
+    protected void waitUntilReady() {
+        final String containerName = waitStrategyTarget.getContainerInfo().getName();
+        final int livenessCheckPort = waitStrategyTarget.getMappedPort(port);
+        final String uri = buildLivenessUri(livenessCheckPort).toString();
+        log.info("{}: Waiting for {} seconds for URL: {}", containerName, startupTimeout.getSeconds(), uri);
+        // try to connect to the URL
+        try {
+            retryUntilSuccess((int) startupTimeout.getSeconds(), TimeUnit.SECONDS, () -> {
+                getRateLimiter().doWhenReady(() -> {
+                    try {
+                        final HttpURLConnection connection = (HttpURLConnection) new URL(uri).openConnection();
+                        // authenticate
+                        if (!Strings.isNullOrEmpty(username)) {
+                            connection.setRequestProperty(HEADER_AUTHORIZATION, buildAuthString(username, password));
+                            connection.setUseCaches(false);
+                        }
+                        connection.setRequestMethod("GET");
+                        connection.connect();
+                        if (statusCode != connection.getResponseCode()) {
+                            throw new RuntimeException(String.format("HTTP response code was: %s",
+                                connection.getResponseCode()));
+                        }
+                        if (responsePredicate != null) {
+                            String responseBody = getResponseBody(connection);
+                            if (!responsePredicate.test(responseBody)) {
+                                throw new RuntimeException(String.format("Response: %s did not match predicate",
+                                    responseBody));
+                            }
+                        }
+                    } catch (IOException e) {
+                        throw new RuntimeException(e);
+                    }
+                });
+                return true;
+            });
+        } catch (TimeoutException e) {
+            throw new ContainerLaunchException(String.format(
+                "Timed out waiting for URL to be accessible (%s should return HTTP %s)", uri, statusCode));
+        }
+    }
+
+    /**
+     * Build the URI on which to check if the container is ready.
+     *
+     * @param livenessCheckPort the liveness port
+     * @return the liveness URI
+     */
+    private URI buildLivenessUri(int livenessCheckPort) {
+        final String scheme = (tlsEnabled ? "https" : "http") + "://";
+        final String host = waitStrategyTarget.getContainerIpAddress();
+        final String portSuffix;
+        if ((tlsEnabled && 443 == livenessCheckPort) || (!tlsEnabled && 80 == livenessCheckPort)) {
+            portSuffix = "";
+        } else {
+            portSuffix = ":" + String.valueOf(livenessCheckPort);
+        }
+        return URI.create(scheme + host + portSuffix + path);
+    }
+
+    /**
+     * @param username the username
+     * @param password the password
+     * @return a basic authentication string for the given credentials
+     */
+    private String buildAuthString(String username, String password) {
+        return AUTH_BASIC + BaseEncoding.base64().encode((username + ":" + password).getBytes(UTF_8));
+    }
+
+    private String getResponseBody(HttpURLConnection connection) throws IOException {
+        BufferedReader reader = null;
+        try {
+            if (200 <= connection.getResponseCode() && connection.getResponseCode() <= 299) {
+                reader = new BufferedReader(new InputStreamReader((connection.getInputStream()), UTF_8));
+            } else {
+                reader = new BufferedReader(new InputStreamReader((connection.getErrorStream()), UTF_8));
+            }
+            StringBuilder builder = new StringBuilder();
+            String line;
+            while ((line = reader.readLine()) != null) {
+                builder.append(line);
+            }
+            return builder.toString();
+        } finally {
+            if (null != reader) {
+                reader.close();
+            }
+        }
+    }
+}
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/ZKWaitStrategy.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/ZKWaitStrategy.java
new file mode 100644
index 0000000..bd3d916
--- /dev/null
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/wait/ZKWaitStrategy.java
@@ -0,0 +1,63 @@
+/*
+ * 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.tests.containers.wait;
+
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.BookKeeperClusterUtils;
+import org.rnorth.ducttape.TimeoutException;
+import org.rnorth.ducttape.unreliables.Unreliables;
+import org.testcontainers.containers.ContainerLaunchException;
+import org.testcontainers.containers.wait.strategy.AbstractWaitStrategy;
+
+/**
+ * Wait Strategy until zookeeper container is up.
+ */
+@Slf4j
+public class ZKWaitStrategy extends AbstractWaitStrategy {
+
+    private final int zkPort;
+
+    public ZKWaitStrategy(int zkPort) {
+        this.zkPort = zkPort;
+    }
+
+    @Override
+    protected void waitUntilReady() {
+        String hostname = waitStrategyTarget.getContainerIpAddress();
+        int externalPort = waitStrategyTarget.getMappedPort(zkPort);
+
+        try {
+            Unreliables.retryUntilTrue(
+                (int) startupTimeout.getSeconds(),
+                TimeUnit.SECONDS,
+                () -> getRateLimiter().getWhenReady(
+                    () -> {
+                        log.info("Check if zookeeper is running at {}:{}", hostname, externalPort);
+                        return BookKeeperClusterUtils.zookeeperRunning(
+                            hostname, externalPort
+                        );
+                    }));
+        } catch (TimeoutException te) {
+            throw new ContainerLaunchException(
+                "Timed out waiting for zookeeper to be ready");
+        }
+    }
+
+}
diff --git a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
index 3cd370f..7020fff 100644
--- a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
+++ b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
@@ -60,7 +60,10 @@ public class BookKeeperClusterUtils {
 
     public static boolean zookeeperRunning(DockerClient docker, String containerId) {
         String ip = DockerUtils.getContainerIP(docker, containerId);
-        try (Socket socket = new Socket(ip, 2181)) {
+        return zookeeperRunning(ip, 2181);
+    }
+    public static boolean zookeeperRunning(String ip, int port) {
+        try (Socket socket = new Socket(ip, port)) {
             socket.setSoTimeout(1000);
             socket.getOutputStream().write("ruok".getBytes(UTF_8));
             byte[] resp = new byte[4];
diff --git a/tests/integration/cluster/pom.xml b/tests/integration/cluster/pom.xml
new file mode 100644
index 0000000..422269a
--- /dev/null
+++ b/tests/integration/cluster/pom.xml
@@ -0,0 +1,70 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation=" http://maven.apache.org/POM/4.0.0   http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper.tests.integration</groupId>
+    <artifactId>tests-parent</artifactId>
+    <version>4.8.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.bookkeeper.tests.integration</groupId>
+  <artifactId>cluster</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Integration :: Cluster test</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-server</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>stream-storage-server</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.bookkeeper.tests</groupId>
+      <artifactId>integration-tests-topologies</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <!-- smoke test should never flake //-->
+          <rerunFailingTestsCount>0</rerunFailingTestsCount>
+          <redirectTestOutputToFile>${redirectTestOutputToFile}</redirectTestOutputToFile>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java
new file mode 100644
index 0000000..36d5d86
--- /dev/null
+++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java
@@ -0,0 +1,94 @@
+/*
+ * 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.tests.integration.cluster;
+
+import com.google.common.base.Stopwatch;
+import java.net.URI;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tests.integration.topologies.BKCluster;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/**
+ * A docker container based bookkeeper cluster test base, providing similar facility like the one in unit test.
+ */
+@Slf4j
+public abstract class BookKeeperClusterTestBase {
+
+    protected static BKCluster bkCluster;
+    protected static URI metadataServiceUri;
+    protected static MetadataClientDriver metadataClientDriver;
+    protected static ScheduledExecutorService executor;
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        bkCluster = new BKCluster(RandomStringUtils.randomAlphabetic(8), 0);
+        bkCluster.start();
+
+        metadataServiceUri = URI.create(bkCluster.getExternalServiceUri());
+        ClientConfiguration conf = new ClientConfiguration()
+            .setMetadataServiceUri(metadataServiceUri.toString());
+        executor = Executors.newSingleThreadScheduledExecutor();
+        metadataClientDriver = MetadataDrivers.getClientDriver(metadataServiceUri);
+        metadataClientDriver.initialize(conf, executor, NullStatsLogger.INSTANCE, Optional.empty());
+    }
+
+    @AfterClass
+    public static void teardownCluster() {
+        if (null != metadataClientDriver) {
+            metadataClientDriver.close();
+        }
+        if (null != executor) {
+            executor.shutdown();
+        }
+        if (null != bkCluster) {
+            bkCluster.stop();
+        }
+    }
+
+    private boolean findIfBookieRegistered(String bookieName) throws Exception {
+        Set<BookieSocketAddress> bookies =
+            FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue();
+        Optional<BookieSocketAddress> registered =
+            bookies.stream().filter(addr -> addr.getHostName().equals(bookieName)).findFirst();
+        return registered.isPresent();
+    }
+
+    protected void waitUntilBookieUnregistered(String bookieName) throws Exception {
+        Stopwatch sw = Stopwatch.createStarted();
+        while (findIfBookieRegistered(bookieName)) {
+            TimeUnit.MILLISECONDS.sleep(1000);
+            log.info("Bookie {} is still registered in cluster {} after {} ms elapsed",
+                bookieName, bkCluster.getClusterName(), sw.elapsed(TimeUnit.MILLISECONDS));
+        }
+    }
+
+}
diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java
new file mode 100644
index 0000000..9d09c67
--- /dev/null
+++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.tests.integration.cluster;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.tests.containers.BookieContainer;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+
+/**
+ * A simple test on bookkeeper cluster operations, e.g. start bookies, stop bookies and list bookies.
+ */
+@Slf4j
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class SimpleClusterTest extends BookKeeperClusterTestBase {
+
+    @Test
+    public void test000_ClusterIsEmpty() throws Exception {
+        Set<BookieSocketAddress> bookies =
+            FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue();
+        assertTrue(bookies.isEmpty());
+    }
+
+    @Test
+    public void test001_StartBookie() throws Exception {
+        String bookieName = "bookie-000";
+        BookieContainer container = bkCluster.createBookie(bookieName);
+        assertNotNull("Container should be started", container);
+        assertEquals(1, bkCluster.getBookieContainers().size());
+        assertSame(container, bkCluster.getBookie(bookieName));
+
+        Set<BookieSocketAddress> bookies =
+            FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue();
+        assertEquals(1, bookies.size());
+    }
+
+    @Test
+    public void test002_StopBookie() throws Exception {
+        String bookieName = "bookie-000";
+        BookieContainer container = bkCluster.killBookie(bookieName);
+        assertNotNull("Bookie '" + bookieName + "' doesn't exist", container);
+        assertEquals(0, bkCluster.getBookieContainers().size());
+        assertNull(bkCluster.getBookie(bookieName));
+
+        waitUntilBookieUnregistered(bookieName);
+
+        Set<BookieSocketAddress> bookies =
+            FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue();
+        assertEquals(0, bookies.size());
+    }
+
+}
diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/topologies/BKCluster.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/topologies/BKCluster.java
new file mode 100644
index 0000000..c1e9e84
--- /dev/null
+++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/topologies/BKCluster.java
@@ -0,0 +1,156 @@
+/*
+ * 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.tests.integration.topologies;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.tests.containers.BookieContainer;
+import org.apache.bookkeeper.tests.containers.MetadataStoreContainer;
+import org.apache.bookkeeper.tests.containers.ZKContainer;
+import org.testcontainers.containers.Network;
+
+/**
+ * BookKeeper Cluster in containers.
+ */
+@Slf4j
+public class BKCluster {
+
+    @Getter
+    private final String clusterName;
+    private final Network network;
+    private final MetadataStoreContainer metadataContainer;
+    private final Map<String, BookieContainer> bookieContainers;
+    private final int numBookies;
+
+    public BKCluster(String clusterName, int numBookies) {
+        this.clusterName = clusterName;
+        this.network = Network.newNetwork();
+        this.metadataContainer = (MetadataStoreContainer) new ZKContainer(clusterName)
+            .withNetwork(network)
+            .withNetworkAliases(ZKContainer.HOST_NAME);
+        this.bookieContainers = Maps.newTreeMap();
+        this.numBookies = numBookies;
+    }
+
+    public String getExternalServiceUri() {
+        return metadataContainer.getExternalServiceUri();
+    }
+
+    public String getInternalServiceUri() {
+        return metadataContainer.getInternalServiceUri();
+    }
+
+    public void start() throws Exception {
+        // start the metadata store
+        this.metadataContainer.start();
+
+        // init a new cluster
+        initNewCluster(metadataContainer.getExternalServiceUri());
+
+        // create bookies
+        createBookies("bookie", numBookies);
+    }
+
+    public void stop() {
+        synchronized (this) {
+            bookieContainers.values().forEach(BookieContainer::stop);
+        }
+
+        this.metadataContainer.stop();
+        try {
+            this.network.close();
+        } catch (Exception e) {
+            log.info("Failed to shutdown network for bookkeeper cluster {}", clusterName, e);
+        }
+    }
+
+    protected void initNewCluster(String metadataServiceUri) throws Exception {
+        MetadataDrivers.runFunctionWithRegistrationManager(
+            new ServerConfiguration().setMetadataServiceUri(metadataServiceUri),
+            rm -> {
+                try {
+                    rm.initNewCluster();
+                } catch (Exception e) {
+                    throw new UncheckedExecutionException("Failed to init a new cluster", e);
+                }
+                return null;
+            }
+        );
+    }
+
+    public synchronized Map<String, BookieContainer> getBookieContainers() {
+        return bookieContainers;
+    }
+
+
+    public synchronized BookieContainer getBookie(String bookieName) {
+        return bookieContainers.get(bookieName);
+    }
+
+    public BookieContainer killBookie(String bookieName) {
+        BookieContainer container;
+        synchronized (this) {
+            container = bookieContainers.remove(bookieName);
+            if (null != container) {
+                container.stop();
+            }
+        }
+        return container;
+    }
+
+    public synchronized BookieContainer createBookie(String bookieName) {
+        BookieContainer container = getBookie(bookieName);
+        if (null == container) {
+            container = (BookieContainer) new BookieContainer(clusterName, bookieName, ZKContainer.SERVICE_URI)
+                .withNetwork(network)
+                .withNetworkAliases(bookieName);
+            container.start();
+            bookieContainers.put(bookieName, container);
+        }
+        return container;
+    }
+
+    public synchronized Map<String, BookieContainer> createBookies(String bookieNamePrefix, int numBookies)
+            throws Exception {
+        List<CompletableFuture<Void>> startFutures = Lists.newArrayListWithExpectedSize(numBookies);
+        Map<String, BookieContainer> containers = Maps.newHashMap();
+        for (int i = 0; i < numBookies; i++) {
+            final int idx = i;
+            startFutures.add(
+                CompletableFuture.runAsync(() -> {
+                    String bookieName = String.format("%s-%03d", bookieNamePrefix, idx);
+                    BookieContainer container = createBookie(bookieName);
+                    synchronized (containers) {
+                        containers.put(bookieName, container);
+                    }
+                }));
+        }
+        FutureUtils.result(FutureUtils.collect(startFutures));
+        return containers;
+    }
+}
diff --git a/tests/integration/cluster/src/test/resources/log4j.properties b/tests/integration/cluster/src/test/resources/log4j.properties
new file mode 100644
index 0000000..10ae6bf
--- /dev/null
+++ b/tests/integration/cluster/src/test/resources/log4j.properties
@@ -0,0 +1,42 @@
+#
+#
+# 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.
+#
+#
+
+#
+# Bookkeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only, level INFO
+bookkeeper.root.logger=INFO,CONSOLE
+log4j.rootLogger=${bookkeeper.root.logger}
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+log4j.logger.org.apache.bookkeeper.bookie=INFO
+log4j.logger.org.apache.bookkeeper.meta=INFO
diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml
index dbd061d..bcf3622 100644
--- a/tests/integration/pom.xml
+++ b/tests/integration/pom.xml
@@ -30,6 +30,7 @@
   <modules>
     <module>smoke</module>
     <module>standalone</module>
+    <module>cluster</module>
   </modules>
 
   <build>

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

[bookkeeper] 06/06: Provide a util method to run functions with metadata client driver

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5be2e90942536d58f50c449a729f0ba26096564f
Author: Sijie Guo <si...@apache.org>
AuthorDate: Mon May 21 22:39:40 2018 -0700

    Provide a util method to run functions with metadata client driver
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    Currently `MetadataDrivers` provides util methods to run functions with metadata bookie driver.
    It is convinient to provide a util method to run functions with metadata client driver as well.
    
    *Solution*
    
    Provide a util method `runFunctionWithMetadataClientDriver` to run functions with metadata client driver.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Jia Zhai <None>
    
    This closes #1421 from sijie/utils_to_run_client_driver
---
 .../apache/bookkeeper/meta/MetadataDrivers.java    | 35 ++++++++++++++++++++++
 1 file changed, 35 insertions(+)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
index f278205..bab6d84 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java
@@ -25,16 +25,19 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.UncheckedExecutionException;
 import java.net.URI;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.function.Function;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.NoArgsConstructor;
 import lombok.ToString;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.exceptions.Code;
@@ -319,6 +322,38 @@ public final class MetadataDrivers {
     }
 
     /**
+     * Process the provided <i>function</i> with metadata client driver resolved
+     * from the metadata service uri returned by {@link ClientConfiguration#getMetadataServiceUri()}.
+     *
+     * @param conf client configuration
+     * @param function function to apply with metadata client driver.
+     * @param executorService executor service used by the metadata client driver.
+     * @throws MetadataException when failed to access metadata store
+     * @throws ExecutionException exception thrown when processing <tt>function</tt>.
+     */
+    public static <T> T runFunctionWithMetadataClientDriver(ClientConfiguration conf,
+                                                            Function<MetadataClientDriver, T> function,
+                                                            ScheduledExecutorService executorService)
+            throws MetadataException, ExecutionException {
+        try (MetadataClientDriver driver = MetadataDrivers.getClientDriver(
+            URI.create(conf.getMetadataServiceUri())
+        )) {
+            driver.initialize(conf, executorService, NullStatsLogger.INSTANCE, Optional.empty());
+            try {
+                return function.apply(driver);
+            } catch (Exception uee) {
+                if (uee.getCause() instanceof MetadataException) {
+                    throw (MetadataException) uee.getCause();
+                } else {
+                    throw new ExecutionException(uee.getMessage(), uee.getCause());
+                }
+            }
+        } catch (ConfigurationException e) {
+            throw new MetadataException(Code.INVALID_METADATA_SERVICE_URI, e);
+        }
+    }
+
+    /**
      * Process the provided <i>function</i> with metadata bookie driver resolved
      * from the metadata service uri returned by {@link ServerConfiguration#getMetadataServiceUri()}.
      *

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

[bookkeeper] 05/06: Provide zookeeper startup script

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5345be3d6cf2067ff66c530435c6e65ce364a84a
Author: Sijie Guo <si...@apache.org>
AuthorDate: Mon May 21 14:54:52 2018 -0700

    Provide zookeeper startup script
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    ZooKeeper is a dependency of bookkeeper shipped along with bookkeeper dist package. However we never provide any script to startup zookeeper in the bookkeeper binary package and never test the zookeeper version shipped along with bookkeeper
    
    *Solution*
    
    - add zookeeper in `bin/bookeeper` scripts to start zookeeper using bookkeeper scripts
    - refactor bookkeeper docker scripts to start bookie/zookeeper/standalone
    - use zk adminPort for health check, since 4letters commands are explicitly disabled at 3.5
    
    *Tests*
    
    Existing integration tests test the docker scripts and the bash script changes to start zookeeper.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>
    
    This closes #1416 from sijie/package_zookeeper
---
 bin/bookkeeper                                     |  13 +-
 bin/bookkeeper-daemon.sh                           |   5 +
 bookkeeper-dist/all/pom.xml                        |   6 +
 bookkeeper-dist/server/pom.xml                     |   6 +
 .../src/main/resources/LICENSE-all.bin.txt         |   4 +
 .../src/main/resources/LICENSE-server.bin.txt      |   4 +
 conf/bkenv.sh                                      |   3 +
 conf/zookeeper.conf                                | 101 +++++++++++++
 conf/zookeeper.conf.dynamic                        |  19 +++
 docker/Dockerfile                                  |  12 +-
 docker/scripts/apply-config-from-env.py            |   7 +
 docker/scripts/common.sh                           |  65 +++++++++
 docker/scripts/entrypoint.sh                       | 156 +++++----------------
 docker/scripts/healthcheck.sh                      |   8 +-
 docker/scripts/init_bookie.sh                      | 140 ++++++++++++++++++
 docker/scripts/{healthcheck.sh => init_generic.sh} |   9 +-
 .../scripts/{healthcheck.sh => init_standalone.sh} |  12 +-
 docker/scripts/init_zookeeper.sh                   |  74 ++++++++++
 pom.xml                                            |   7 +
 .../docker-images/current-version-image/Dockerfile |  11 +-
 .../tests/containers/BKStandaloneContainer.java    |   8 +-
 .../bookkeeper/tests/containers/ZKContainer.java   |  21 ++-
 22 files changed, 536 insertions(+), 155 deletions(-)

diff --git a/bin/bookkeeper b/bin/bookkeeper
index 47c4e3f..ceb0470 100755
--- a/bin/bookkeeper
+++ b/bin/bookkeeper
@@ -35,6 +35,7 @@ BOOKIE_CLASSPATH=$(set_module_classpath ${BOOKIE_MODULE_PATH})
 
 # default variables
 DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf
+DEFAULT_ZK_CONF=${BK_HOME}/conf/zookeeper.conf
 
 bookkeeper_help() {
     cat <<EOF
@@ -43,8 +44,10 @@ where command is one of:
     bookie              Run a bookie server
     autorecovery        Run AutoRecovery service daemon
     localbookie <n>     Run a test ensemble of <n> bookies locally
+    standalone <n>      Run a standalone cluster of <n> bookies locally
     upgrade             Upgrade bookie filesystem
     shell               Run shell for admin commands
+    zookeeper           Run zookeeper server
     help                This help message
 
 or command is the full name of a class with a defined main() method.
@@ -52,6 +55,7 @@ or command is the full name of a class with a defined main() method.
 Environment variables:
    BOOKIE_LOG_CONF        Log4j configuration file (default ${DEFAULT_LOG_CONF})
    BOOKIE_CONF            Configuration file (default: ${DEFAULT_CONF})
+   BOOKIE_ZK_CONF         Configuration file for zookeeper (default: $DEFAULT_ZK_CONF)
    BOOKIE_EXTRA_OPTS      Extra options to be passed to the jvm
    BOOKIE_EXTRA_CLASSPATH Add extra paths to the bookkeeper classpath
    ENTRY_FORMATTER_CLASS  Entry formatter class to format entries.
@@ -87,6 +91,10 @@ if [ ${COMMAND} == "shell" ]; then
   fi
 fi
 
+if [ -z "$BOOKIE_ZK_CONF" ]; then
+    BOOKIE_ZK_CONF=$DEFAULT_ZK_CONF
+fi
+
 if [ -z "$BOOKIE_CONF" ]; then
   BOOKIE_CONF=${DEFAULT_CONF}
 fi
@@ -122,12 +130,15 @@ if [ ${COMMAND} == "bookie" ]; then
   exec ${JAVA} ${OPTS} ${JMX_ARGS} org.apache.bookkeeper.server.Main --conf ${BOOKIE_CONF} $@
 elif [ ${COMMAND} == "autorecovery" ]; then
   exec ${JAVA} ${OPTS} ${JMX_ARGS} org.apache.bookkeeper.replication.AutoRecoveryMain --conf ${BOOKIE_CONF} $@
-elif [ ${COMMAND} == "localbookie" ]; then
+elif [ ${COMMAND} == "localbookie" -o ${COMMAND} == "standalone" ]; then
   NUMBER=$1
   shift
   exec ${JAVA} ${OPTS} ${JMX_ARGS} -Dzookeeper.4lw.commands.whitelist='*' org.apache.bookkeeper.util.LocalBookKeeper ${NUMBER} ${BOOKIE_CONF} $@
 elif [ ${COMMAND} == "upgrade" ]; then
   exec ${JAVA} ${OPTS} org.apache.bookkeeper.bookie.FileSystemUpgrade --conf ${BOOKIE_CONF} $@
+elif [ $COMMAND == "zookeeper" ]; then
+    BOOKIE_LOG_FILE=${BOOKIE_LOG_FILE:-"zookeeper.log"}
+    exec $JAVA $OPTS -Dbookkeeper.log.file=$BOOKIE_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $BOOKIE_ZK_CONF $@
 elif [ ${COMMAND} == "shell" ]; then
   ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}"
   exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@
diff --git a/bin/bookkeeper-daemon.sh b/bin/bookkeeper-daemon.sh
index dab21f6..3a8bbdc 100755
--- a/bin/bookkeeper-daemon.sh
+++ b/bin/bookkeeper-daemon.sh
@@ -23,6 +23,8 @@ usage() {
 Usage: bookkeeper-daemon.sh (start|stop) <command> <args...>
 where command is one of:
     bookie           Run the bookie server
+    autorecovery     Run the AutoRecovery service daemon
+    zookeeper        Run the zookeeper server
 
 where argument is one of:
     -force (accepted only with stop command): Decides whether to stop the Bookie Server forcefully if not stopped by normal shutdown
@@ -58,6 +60,9 @@ command=$1
 shift
 
 case $command in
+  (zookeeper)
+    echo "doing $startStop $command ..."
+    ;;
   (bookie)
     echo "doing $startStop $command ..."
     ;;
diff --git a/bookkeeper-dist/all/pom.xml b/bookkeeper-dist/all/pom.xml
index 8a10382..253b356 100644
--- a/bookkeeper-dist/all/pom.xml
+++ b/bookkeeper-dist/all/pom.xml
@@ -104,6 +104,12 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
     </dependency>
+
+    <!-- jackson mapper for running zookeeper -->
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/bookkeeper-dist/server/pom.xml b/bookkeeper-dist/server/pom.xml
index e4f398a..75e70e9 100644
--- a/bookkeeper-dist/server/pom.xml
+++ b/bookkeeper-dist/server/pom.xml
@@ -81,6 +81,12 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
     </dependency>
+
+    <!-- jackson mapper for running zookeeper -->
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index b0ba7e9..7f523db 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -274,6 +274,8 @@ Apache Software License, Version 2.
 - lib/com.yahoo.datasketches-memory-0.8.3.jar [37]
 - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37]
 - lib/net.jpountz.lz4-lz4-1.3.0.jar [38]
+- lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [39]
+- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [40]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -312,6 +314,8 @@ Apache Software License, Version 2.
 [36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 [37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
 [38] Source available at https://github.com/lz4/lz4-java/tree/1.3.0
+[39] Source available at https://github.com/codehaus/jackson/tree/1.9
+[40] Source available at https://github.com/codehaus/jackson/tree/1.9
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index 9f9ccc6..1f437ab 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -239,6 +239,8 @@ Apache Software License, Version 2.
 - lib/com.yahoo.datasketches-memory-0.8.3.jar [24]
 - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [24]
 - lib/net.jpountz.lz4-lz4-1.3.0.jar [25]
+- lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [26]
+- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [27]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -265,6 +267,8 @@ Apache Software License, Version 2.
 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 [24] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
 [25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0
+[26] Source available at https://github.com/codehaus/jackson/tree/1.9
+[27] Source available at https://github.com/codehaus/jackson/tree/1.9
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-all-4.1.12.Final.jar bundles some 3rd party dependencies
diff --git a/conf/bkenv.sh b/conf/bkenv.sh
index eebc4ff..f6d0616 100644
--- a/conf/bkenv.sh
+++ b/conf/bkenv.sh
@@ -28,6 +28,9 @@
 # Configuration file of settings used in bookie server
 # BOOKIE_CONF=
 
+# Configuration file of settings used in zookeeper server
+# BOOKIE_ZK_CONF=
+
 # Extra options to be passed to the jvm
 # BOOKIE_EXTRA_OPTS=
 
diff --git a/conf/zookeeper.conf b/conf/zookeeper.conf
new file mode 100644
index 0000000..407baeb
--- /dev/null
+++ b/conf/zookeeper.conf
@@ -0,0 +1,101 @@
+#
+# 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 number of milliseconds of each tick
+tickTime=2000
+
+# The number of ticks that the initial 
+# synchronization phase can take
+initLimit=10
+
+# The number of ticks that can pass between 
+# sending a request and getting an acknowledgement
+syncLimit=30
+
+# the directory where the snapshot is stored.
+dataDir=data/zookeeper
+
+# where txlog  are written
+dataLogDir=data/zookeeper/txlog
+
+# the port at which the clients will connect
+clientPort=2181
+
+# the port at which the admin will listen
+adminPort=9990
+zookeeper.admin.enableServer=true
+
+# limit on queued clients - default: 1000
+globalOutstandingLimit=1000
+
+# number of transactions before snapshots are taken - default: 100000
+snapCount=100000
+
+# the maximum number of client connections.
+# increase this if you need to handle more clients
+#
+# - 0==unlimited
+maxClientCnxns=100
+
+# Election implementation to use. A value of "0" corresponds to the original
+# UDP-based version, "1" corresponds to the non-authenticated UDP-based
+# version of fast leader election, "2" corresponds to the authenticated
+# UDP-based version of fast leader election, and "3" corresponds to TCP-based
+# version of fast leader election. Currently, only 0 and 3 are supported,
+# 3 being the default
+electionAlg=3
+
+# Leader accepts client connections. Default value is "yes". The leader
+# machine coordinates updates. For higher update throughput at thes slight
+# expense of read throughput the leader can be configured to not accept
+# clients and focus on coordination.
+leaderServes=yes
+
+# Skips ACL checks. This results in a boost in throughput, but opens up full
+# access to the data tree to everyone.
+skipACL=no
+
+# Be sure to read the maintenance section of the 
+# administrator guide before turning on autopurge.
+#
+# http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#sc_maintenance
+#
+# The number of snapshots to retain in dataDir
+autopurge.snapRetainCount=3
+
+# Purge txn logs every hour. Before 3.4.x this was done with an external cron
+# job, now we can do it internally.
+#
+# Set to "0" to disable auto purge feature
+autopurge.purgeInterval=1
+
+# Prior to version 3.4 ZooKeeper has always used NIO directly, however in
+# versions 3.4 and later Netty is supported as an option to NIO (replaces).
+# serverCnxnFactory=org.apache.zookeeper.server.NIOServerCnxnFactory
+
+# zookeeper cluster
+
+standaloneEnabled=true
+# ZooKeeper Dynamic Reconfiguration
+# See: https://zookeeper.apache.org/doc/trunk/zookeeperReconfig.html
+#
+# standaloneEnabled=false
+# dynamicConfigFile=/path/to/zoo.cfg.dynamic
+#
+dynamicConfigFile=conf/zookeeper.conf.dynamic
diff --git a/conf/zookeeper.conf.dynamic b/conf/zookeeper.conf.dynamic
new file mode 100644
index 0000000..5d53861
--- /dev/null
+++ b/conf/zookeeper.conf.dynamic
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
diff --git a/docker/Dockerfile b/docker/Dockerfile
index 0d39b34..5c8e664 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -27,6 +27,9 @@ ARG GPG_KEY=FD74402C
 ENV BOOKIE_PORT=3181
 EXPOSE $BOOKIE_PORT
 ENV BK_USER=bookkeeper
+ENV BK_HOME=/opt/bookkeeper
+ENV JAVA_HOME=/usr/lib/jvm/jre-1.8.0
+
 
 # Download Apache Bookkeeper, untar and clean up
 RUN set -x \
@@ -48,9 +51,10 @@ RUN set -x \
 
 WORKDIR /opt/bookkeeper
 
-COPY scripts/apply-config-from-env.py scripts/entrypoint.sh scripts/healthcheck.sh /opt/bookkeeper/
+COPY scripts /opt/bookkeeper/scripts
+RUN chmod +x -R /opt/bookkeeper/scripts/
 
-ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
-CMD ["/opt/bookkeeper/bin/bookkeeper", "bookie"]
+ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/scripts/entrypoint.sh" ]
+CMD ["bookie"]
 
-HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/healthcheck.sh
+HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/scripts/healthcheck.sh
diff --git a/docker/scripts/apply-config-from-env.py b/docker/scripts/apply-config-from-env.py
index 78e6945..3fccab9 100755
--- a/docker/scripts/apply-config-from-env.py
+++ b/docker/scripts/apply-config-from-env.py
@@ -42,6 +42,7 @@ print 'conf files: '
 print conf_files
 
 bk_env_prefix = 'BK_'
+zk_env_prefix = 'ZK_'
 
 for conf_filename in conf_files:
     lines = []  # List of config file lines
@@ -77,6 +78,12 @@ for conf_filename in conf_files:
                 print '[%s] Applying config %s = %s' % (conf_filename, search_key, v)
                 idx = keys[search_key]
                 lines[idx] = '%s=%s\n' % (search_key, v)
+        if k.startswith(zk_env_prefix):
+            search_key = k[len(zk_env_prefix):]
+            if search_key in keys:
+                print '[%s] Applying config %s = %s' % (conf_filename, search_key, v)
+                idx = keys[search_key]
+                lines[idx] = '%s=%s\n' % (search_key, v)
 
     # Store back the updated config in the same file
     f = open(conf_filename, 'w')
diff --git a/docker/scripts/common.sh b/docker/scripts/common.sh
new file mode 100755
index 0000000..ece24ba
--- /dev/null
+++ b/docker/scripts/common.sh
@@ -0,0 +1,65 @@
+#!/usr/bin/env bash
+#
+#/**
+# * 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.
+# */
+
+# env var used often
+PORT0=${PORT0:-${BOOKIE_PORT}}
+PORT0=${PORT0:-3181}
+BK_DATA_DIR=${BK_DATA_DIR:-"/data/bookkeeper"}
+BK_CLUSTER_ROOT_PATH=${BK_CLUSTER_ROOT_PATH:-""}
+
+# bk env vars to replace values in config files
+export BK_HOME=/opt/bookkeeper
+export BK_bookiePort=${BK_bookiePort:-${PORT0}}
+export BK_zkServers=${BK_zkServers}
+export BK_zkLedgersRootPath=${BK_zkLedgersRootPath:-"${BK_CLUSTER_ROOT_PATH}/ledgers"}
+export BK_journalDirectory=${BK_journalDirectory:-${BK_DATA_DIR}/journal}
+export BK_ledgerDirectories=${BK_ledgerDirectories:-${BK_DATA_DIR}/ledgers}
+export BK_indexDirectories=${BK_indexDirectories:-${BK_DATA_DIR}/index}
+export BK_metadataServiceUri=${BK_metadataServiceUri:-"zk://${BK_zkServers}${BK_zkLedgersRootPath}"}
+export BK_dlogRootPath=${BK_dlogRootPath:-"${BK_CLUSTER_ROOT_PATH}/distributedlog"}
+
+echo "Environment Vars for bookie:"
+echo "  BK_bookiePort bookie service port is $BK_bookiePort"
+echo "  BK_zkServers is $BK_zkServers"
+echo "  BK_DATA_DIR is $BK_DATA_DIR"
+echo "  BK_CLUSTER_ROOT_PATH is $BK_CLUSTER_ROOT_PATH"
+echo "  BK_metadataServiceUri is $BK_metadataServiceUri"
+echo "  BK_dlogRootPath is $BK_dlogRootPath"
+
+python scripts/apply-config-from-env.py ${BK_HOME}/conf
+
+export BOOKIE_CONF=${BK_HOME}/conf/bk_server.conf
+export SERVICE_PORT=${PORT0}
+
+function create_bookie_dirs() {
+    mkdir -p "${BK_journalDirectory}" "${BK_ledgerDirectories}" "${BK_indexDirectories}"
+    echo "Created bookie dirs : "
+    echo "  journal = ${BK_journalDirectory}"
+    echo "  ledger = ${BK_ledgerDirectories}"
+    echo "  index = ${BK_indexDirectories}"
+    # -------------- #
+    # Allow the container to be started with `--user`
+    if [ "$(id -u)" = '0' ]; then
+        chown -R "${BK_USER}:${BK_USER}" "${BK_journalDirectory}" "${BK_ledgerDirectories}" "${BK_indexDirectories}"
+    fi
+    # -------------- #
+}
diff --git a/docker/scripts/entrypoint.sh b/docker/scripts/entrypoint.sh
index 7ade924..8691117 100755
--- a/docker/scripts/entrypoint.sh
+++ b/docker/scripts/entrypoint.sh
@@ -21,138 +21,52 @@
 # */
 
 export PATH=$PATH:/opt/bookkeeper/bin
-export JAVA_HOME=/usr
+export JAVA_HOME=/usr/lib/jvm/jre-1.8.0
 
-# env var used often
-PORT0=${PORT0:-${BOOKIE_PORT}}
-PORT0=${PORT0:-3181}
-BK_DATA_DIR=${BK_DATA_DIR:-"/data/bookkeeper"}
-BK_CLUSTER_ROOT_PATH=${BK_CLUSTER_ROOT_PATH:-""}
+BK_HOME=/opt/bookkeeper
+BINDIR=${BK_HOME}/bin
+BOOKKEEPER=${BINDIR}/bookkeeper
+SCRIPTS_DIR=${BK_HOME}/scripts
 
-# env vars to replace values in config files
-export BK_bookiePort=${BK_bookiePort:-${PORT0}}
-export BK_zkServers=${BK_zkServers}
-export BK_zkLedgersRootPath=${BK_zkLedgersRootPath:-"${BK_CLUSTER_ROOT_PATH}/ledgers"}
-export BK_journalDirectory=${BK_journalDirectory:-${BK_DATA_DIR}/journal}
-export BK_ledgerDirectories=${BK_ledgerDirectories:-${BK_DATA_DIR}/ledgers}
-export BK_indexDirectories=${BK_indexDirectories:-${BK_DATA_DIR}/index}
-export BK_metadataServiceUri=${BK_metadataServiceUri:-"zk://${BK_zkServers}${BK_zkLedgersRootPath}"}
-export BK_dlogRootPath=${BK_dlogRootPath:-"${BK_CLUSTER_ROOT_PATH}/distributedlog"}
-
-echo "BK_bookiePort bookie service port is $BK_bookiePort"
-echo "BK_zkServers is $BK_zkServers"
-echo "BK_DATA_DIR is $BK_DATA_DIR"
-echo "BK_CLUSTER_ROOT_PATH is $BK_CLUSTER_ROOT_PATH"
-echo "BK_metadataServiceUri is $BK_metadataServiceUri"
-echo "BK_dlogRootPath is $BK_dlogRootPath"
-
-mkdir -p "${BK_journalDirectory}" "${BK_ledgerDirectories}" "${BK_indexDirectories}"
-# -------------- #
-# Allow the container to be started with `--user`
-if [ "$1" = '/opt/bookkeeper/bin/bookkeeper' -a "$(id -u)" = '0' ]; then
-    echo "This is root, will use user $BK_USER to run it"
-    chown -R "$BK_USER:$BK_USER" "/opt/bookkeeper/" "${BK_journalDirectory}" "${BK_ledgerDirectories}" "${BK_indexDirectories}"
-    chmod -R +x "/opt/bookkeeper/"
-    sudo -s -E -u "$BK_USER" /bin/bash "$0" "$@"
-    exit
+if [ $# = 0 ]; then
+    echo "No command is found";
+    exit 1;
 fi
-# -------------- #
-
-python apply-config-from-env.py /opt/bookkeeper/conf
-
-echo "wait for zookeeper"
-until /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} ls /; do sleep 5; done
-
-echo "create the zk root dir for bookkeeper"
-/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create ${BK_CLUSTER_ROOT_PATH}
 
-export BOOKIE_CONF=/opt/bookkeeper/conf/bk_server.conf
-export SERVICE_PORT=$PORT0
+COMMAND=$1
+shift
 
-# Init the cluster if required znodes not exist in Zookeeper.
-# Use ephemeral zk node as lock to keep initialize atomic.
-/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_zkLedgersRootPath}/available/readonly
-if [ $? -eq 0 ]; then
-    echo "Metadata of cluster already exists, no need format"
-else
-    # create ephemeral zk node bkInitLock, initiator who this node, then do init; other initiators will wait.
-    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/bkInitLock
-    if [ $? -eq 0 ]; then
-        # bkInitLock created success, this is the successor to do znode init
-        echo "Bookkeeper znodes not exist in Zookeeper, do the init to create them."
-        /opt/bookkeeper/bin/bookkeeper shell initnewcluster
-        if [ $? -eq 0 ]; then
-            echo "Bookkeeper znodes init success."
-        else
-            echo "Bookkeeper znodes init failed. please check the reason."
-            exit
-        fi
+function run_command() {
+    if [ "$(id -u)" = '0' ]; then
+        chown -R "$BK_USER:$BK_USER" ${BK_HOME}
+        chmod -R +x ${BINDIR}
+        chmod -R +x ${SCRIPTS_DIR}
+        echo "This is root, will use user $BK_USER to run command '$@'"
+        sudo -s -E -u "$BK_USER" /bin/bash "$@"
+        exit
     else
-        echo "Other docker instance is doing initialize at the same time, will wait in this instance."
-        tenSeconds=1
-        while [ ${tenSeconds} -lt 10 ]
-        do
-            sleep 10
-            /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_zkLedgersRootPath}/available/readonly
-            if [ $? -eq 0 ]; then
-                echo "Waited $tenSeconds * 10 seconds, bookkeeper inited"
-                break
-            else
-                echo "Waited $tenSeconds * 10 seconds, still not init"
-                (( tenSeconds++ ))
-                continue
-            fi
-        done
+        echo "Run command '$@'"
+        $@
+    fi
+}
 
-        if [ ${tenSeconds} -eq 10 ]; then
-            echo "Waited 100 seconds for bookkeeper cluster init, something wrong, please check"
+# for BC consideration - the old scripts run `entrypoint.sh /opt/bookkeeper/bin/bookkeeper bookie`
+if [ ${COMMAND} == "${BOOKKEEPER}" ]; then
+    if [ $# -gt 0 ]; then
+        if [ $1 == "bookie" ]; then
+            source ${SCRIPTS_DIR}/init_bookie.sh
+            init_bookie
+            run_command /opt/bookkeeper/bin/bookkeeper bookie
             exit
         fi
     fi
 fi
 
-# Create default dlog namespace
-# Use ephemeral zk node as lock to keep initialize atomic.
-/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
-if [ $? -eq 0 ]; then
-    echo "Dlog namespace already created, no need to create another one"
+if [ -f ${SCRIPTS_DIR}/init_${COMMAND}.sh ]; then
+    source ${SCRIPTS_DIR}/init_${COMMAND}.sh
+    init_${COMMAND}
+    run_command ${BOOKKEEPER} ${COMMAND} $@
 else
-    # create ephemeral zk node dlogInitLock, initiator who this node, then do init; other initiators will wait.
-    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/dlogInitLock
-    if [ $? -eq 0 ]; then
-        # dlogInitLock created success, this is the successor to do znode init
-        echo "Dlog namespace not exist, do the init to create them."
-        /opt/bookkeeper/bin/dlog admin bind -l ${BK_zkLedgersRootPath} -s ${BK_zkServers} -c distributedlog://${BK_zkServers}${BK_dlogRootPath}
-        if [ $? -eq 0 ]; then
-            echo "Dlog namespace is created successfully."
-        else
-            echo "Failed to create dlog namespace ${BK_dlogRootPath}. please check the reason."
-            exit
-        fi
-    else
-        echo "Other docker instance is doing initialize at the same time, will wait in this instance."
-        tenSeconds=1
-        while [ ${tenSeconds} -lt 10 ]
-        do
-            sleep 10
-            /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
-            if [ $? -eq 0 ]; then
-                echo "Waited $tenSeconds * 10 seconds, dlog namespace created"
-                break
-            else
-                echo "Waited $tenSeconds * 10 seconds, dlog namespace still not created"
-                (( tenSeconds++ ))
-                continue
-            fi
-        done
-
-        if [ ${tenSeconds} -eq 10 ]; then
-            echo "Waited 100 seconds for creating dlog namespace, something wrong, please check"
-            exit
-        fi
-    fi
+    source ${SCRIPTS_DIR}/init_generic.sh
+    run_command ${COMMAND} $@
 fi
-
-echo "run command by exec"
-exec "$@"
-
diff --git a/docker/scripts/healthcheck.sh b/docker/scripts/healthcheck.sh
index 91e13f0..47a21bf 100755
--- a/docker/scripts/healthcheck.sh
+++ b/docker/scripts/healthcheck.sh
@@ -24,5 +24,11 @@
 
 set -x -e -u
 
+export JAVA_HOME=/usr/lib/jvm/jre-1.8.0
+
 # Sanity check that creates a ledger, writes a few entries, reads them and deletes the ledger.
-/opt/bookkeeper/bin/bookkeeper shell bookiesanity
+DEFAULT_HEALTH_CHECK_CMD="/opt/bookkeeper/bin/bookkeeper shell bookiesanity"
+
+HEALTH_CHECK_CMD=${HEALTH_CHECK_CMD:-"${DEFAULT_HEALTH_CHECK_CMD}"}
+
+exec "${HEALTH_CHECK_CMD}"
diff --git a/docker/scripts/init_bookie.sh b/docker/scripts/init_bookie.sh
new file mode 100755
index 0000000..09aab1d
--- /dev/null
+++ b/docker/scripts/init_bookie.sh
@@ -0,0 +1,140 @@
+#!/usr/bin/env bash
+#
+#/**
+# * 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.
+# */
+
+source ${SCRIPTS_DIR}/common.sh
+
+function wait_for_zookeeper() {
+    echo "wait for zookeeper"
+    until /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} ls /; do sleep 5; done
+}
+
+function create_zk_root() {
+    echo "create the zk root dir for bookkeeper"
+    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create ${BK_CLUSTER_ROOT_PATH}
+}
+
+# Init the cluster if required znodes not exist in Zookeeper.
+# Use ephemeral zk node as lock to keep initialize atomic.
+function init_cluster() {
+    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_zkLedgersRootPath}/available/readonly
+    if [ $? -eq 0 ]; then
+        echo "Metadata of cluster already exists, no need format"
+    else
+        # create ephemeral zk node bkInitLock, initiator who this node, then do init; other initiators will wait.
+        /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/bkInitLock
+        if [ $? -eq 0 ]; then
+            # bkInitLock created success, this is the successor to do znode init
+            echo "Bookkeeper znodes not exist in Zookeeper, do the init to create them."
+            /opt/bookkeeper/bin/bookkeeper shell initnewcluster
+            if [ $? -eq 0 ]; then
+                echo "Bookkeeper znodes init success."
+            else
+                echo "Bookkeeper znodes init failed. please check the reason."
+                exit
+            fi
+        else
+            echo "Other docker instance is doing initialize at the same time, will wait in this instance."
+            tenSeconds=1
+            while [ ${tenSeconds} -lt 10 ]
+            do
+                sleep 10
+                /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_zkLedgersRootPath}/available/readonly
+                if [ $? -eq 0 ]; then
+                    echo "Waited $tenSeconds * 10 seconds, bookkeeper inited"
+                    break
+                else
+                    echo "Waited $tenSeconds * 10 seconds, still not init"
+                    (( tenSeconds++ ))
+                    continue
+                fi
+            done
+
+            if [ ${tenSeconds} -eq 10 ]; then
+                echo "Waited 100 seconds for bookkeeper cluster init, something wrong, please check"
+                exit
+            fi
+        fi
+    fi
+}
+
+# Create default dlog namespace
+# Use ephemeral zk node as lock to keep initialize atomic.
+function create_dlog_namespace() {
+    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
+    if [ $? -eq 0 ]; then
+        echo "Dlog namespace already created, no need to create another one"
+    else
+        # create ephemeral zk node dlogInitLock, initiator who this node, then do init; other initiators will wait.
+        /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/dlogInitLock
+        if [ $? -eq 0 ]; then
+            # dlogInitLock created success, this is the successor to do znode init
+            echo "Dlog namespace not exist, do the init to create them."
+            /opt/bookkeeper/bin/dlog admin bind -l ${BK_zkLedgersRootPath} -s ${BK_zkServers} -c distributedlog://${BK_zkServers}${BK_dlogRootPath}
+            if [ $? -eq 0 ]; then
+                echo "Dlog namespace is created successfully."
+            else
+                echo "Failed to create dlog namespace ${BK_dlogRootPath}. please check the reason."
+                exit
+            fi
+        else
+            echo "Other docker instance is doing initialize at the same time, will wait in this instance."
+            tenSeconds=1
+            while [ ${tenSeconds} -lt 10 ]
+            do
+                sleep 10
+                /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
+                if [ $? -eq 0 ]; then
+                    echo "Waited $tenSeconds * 10 seconds, dlog namespace created"
+                    break
+                else
+                    echo "Waited $tenSeconds * 10 seconds, dlog namespace still not created"
+                    (( tenSeconds++ ))
+                    continue
+                fi
+            done
+
+            if [ ${tenSeconds} -eq 10 ]; then
+                echo "Waited 100 seconds for creating dlog namespace, something wrong, please check"
+                exit
+            fi
+        fi
+    fi
+}
+
+function init_bookie() {
+
+    # create dirs if they don't exist
+    create_bookie_dirs
+
+    # wait zookeeper to run
+    wait_for_zookeeper
+
+    # create zookeeper root
+    create_zk_root
+
+    # init the cluster
+    init_cluster
+
+    # create dlog namespace
+    create_dlog_namespace
+
+}
\ No newline at end of file
diff --git a/docker/scripts/healthcheck.sh b/docker/scripts/init_generic.sh
similarity index 82%
copy from docker/scripts/healthcheck.sh
copy to docker/scripts/init_generic.sh
index 91e13f0..d1ff323 100755
--- a/docker/scripts/healthcheck.sh
+++ b/docker/scripts/init_generic.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 #/**
 # * Copyright 2007 The Apache Software Foundation
@@ -20,9 +20,4 @@
 # * limitations under the License.
 # */
 
-#!/bin/bash
-
-set -x -e -u
-
-# Sanity check that creates a ledger, writes a few entries, reads them and deletes the ledger.
-/opt/bookkeeper/bin/bookkeeper shell bookiesanity
+source ${SCRIPTS_DIR}/common.sh
diff --git a/docker/scripts/healthcheck.sh b/docker/scripts/init_standalone.sh
similarity index 82%
copy from docker/scripts/healthcheck.sh
copy to docker/scripts/init_standalone.sh
index 91e13f0..5007b8c 100755
--- a/docker/scripts/healthcheck.sh
+++ b/docker/scripts/init_standalone.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 #/**
 # * Copyright 2007 The Apache Software Foundation
@@ -20,9 +20,7 @@
 # * limitations under the License.
 # */
 
-#!/bin/bash
-
-set -x -e -u
-
-# Sanity check that creates a ledger, writes a few entries, reads them and deletes the ledger.
-/opt/bookkeeper/bin/bookkeeper shell bookiesanity
+function init_standalone() {
+    # do nothing for now
+    return
+}
\ No newline at end of file
diff --git a/docker/scripts/init_zookeeper.sh b/docker/scripts/init_zookeeper.sh
new file mode 100755
index 0000000..803ef91
--- /dev/null
+++ b/docker/scripts/init_zookeeper.sh
@@ -0,0 +1,74 @@
+#!/usr/bin/env bash
+#
+#/**
+# * 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.
+# */
+
+# zk env vars to replace values in config files
+export ZK_dataDir=${ZK_dataDir:-"data/zookeeper/data"}
+export ZK_dataLogDir=${ZK_dataLogDir:-"data/zookeeper/txlog"}
+export ZK_standaloneEnabled=${ZK_standaloneEnabled:-"false"}
+export ZK_dynamicConfigFile=${ZK_dynamicConfigFile:-"${BK_HOME}/conf/zookeeper.conf.dynamic"}
+export ZK_SERVERS=${ZK_SERVERS:-"server.1=127.0.0.1:2888:3888:participant;0.0.0.0:2181"}
+export ZK_ID=${ZK_ID:-"1"}
+
+echo "Environment Vars for zookeeper:"
+echo "  ZK_dataDir = ${ZK_dataDir}"
+echo "  ZK_dataLogDir = ${ZK_dataLogDir}"
+echo "  ZK_ID = ${ZK_ID}"
+echo "  ZK_SERVERS = ${ZK_SERVERS}"
+echo "  ZK_standaloneEnabled = ${ZK_standaloneEnabled}"
+echo "  ZK_dynamicConfigFile = ${ZK_dynamicConfigFile}"
+
+function create_zk_dirs() {
+    mkdir -p "${ZK_dataDir}" "${ZK_dataLogDir}"
+    echo "Created zookeeper dirs : "
+    echo "  data = ${ZK_dataDir}"
+    echo "  txnlog = ${ZK_dataLogDir}"
+
+    if [[ ! -f "${ZK_dataDir}/myid" ]]; then
+        echo "${ZK_ID}" > "${ZK_dataDir}/myid"
+    fi
+
+    # -------------- #
+    # Allow the container to be started with `--user`
+    if [ "$(id -u)" = '0' ]; then
+        chown -R "${BK_USER}:${BK_USER}" "${ZK_dataDir}" "${ZK_dataLogDir}"
+    fi
+    # -------------- #
+}
+
+function create_zk_dynamic_conf() {
+    for server in ${ZK_SERVERS}; do
+        echo "$server" >> "${ZK_dynamicConfigFile}"
+    done
+}
+
+function init_zookeeper() {
+
+    # apply zookeeper envs
+    python scripts/apply-config-from-env.py ${BK_HOME}/conf
+
+    # create dirs if they don't exist
+    create_zk_dirs
+
+    # create dynamic config
+    create_zk_dynamic_conf
+
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index f145309..3c35d8d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -127,6 +127,7 @@
     <hadoop.version>2.7.3</hadoop.version>
     <hamcrest.version>1.3</hamcrest.version>
     <jackson.version>2.8.9</jackson.version>
+    <jackson-mapper-asl.version>1.9.11</jackson-mapper-asl.version>
     <jcommander.version>1.48</jcommander.version>
     <jetty.version>9.4.5.v20170502</jetty.version>
     <jmh.version>1.19</jmh.version>
@@ -306,6 +307,12 @@
         <artifactId>jackson-annotations</artifactId>
         <version>${jackson.version}</version>
       </dependency>
+      <!-- dependency needed for zookeeper jetty admin server -->
+      <dependency>
+        <groupId>org.codehaus.jackson</groupId>
+        <artifactId>jackson-mapper-asl</artifactId>
+        <version>${jackson-mapper-asl.version}</version>
+      </dependency>
 
       <!-- protobuf dependencies -->
       <dependency>
diff --git a/tests/docker-images/current-version-image/Dockerfile b/tests/docker-images/current-version-image/Dockerfile
index 4006daf..892eb35 100644
--- a/tests/docker-images/current-version-image/Dockerfile
+++ b/tests/docker-images/current-version-image/Dockerfile
@@ -27,6 +27,8 @@ ARG PKG_NAME=bookkeeper-server-${BK_VERSION}
 ENV BOOKIE_PORT=3181
 EXPOSE $BOOKIE_PORT
 ENV BK_USER=bookkeeper
+ENV BK_HOME=/opt/bookkeeper
+ENV JAVA_HOME=/usr/lib/jvm/jre-1.8.0
 
 # prepare utils
 RUN set -x \
@@ -42,9 +44,10 @@ RUN mv /opt/${PKG_NAME} /opt/bookkeeper
 
 WORKDIR /opt/bookkeeper
 
-COPY target/scripts/apply-config-from-env.py target/scripts/entrypoint.sh target/scripts/healthcheck.sh /opt/bookkeeper/
+COPY target/scripts /opt/bookkeeper/scripts
+RUN chmod +x -R /opt/bookkeeper/scripts/
 
-ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
-CMD ["/opt/bookkeeper/bin/bookkeeper", "bookie"]
+ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/scripts/entrypoint.sh" ]
+CMD ["bookie"]
 
-HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/healthcheck.sh
+HEALTHCHECK --interval=10s --timeout=60s CMD /bin/bash /opt/bookkeeper/scripts/healthcheck.sh
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
index 6efec4c..1288695 100644
--- a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/BKStandaloneContainer.java
@@ -59,11 +59,8 @@ public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> ext
             addExposedPort(BOOKIE_BASE_PORT + i);
         }
         setCommand(
-            "/opt/bookkeeper/bin/bookkeeper",
-            "localbookie",
-            "" + numBookies
-        );
-        addEnv("JAVA_HOME", "/usr/lib/jvm/jre-1.8.0");
+            "standalone",
+            "" + numBookies);
     }
 
     @Override
@@ -75,7 +72,6 @@ public class BKStandaloneContainer<SELF extends BKStandaloneContainer<SELF>> ext
         this.withCreateContainerCmdModifier(createContainerCmd -> {
             createContainerCmd.withHostName(STANDALONE_HOST_NAME);
             createContainerCmd.withName(getContainerName());
-            createContainerCmd.withEntrypoint("/bin/bash");
         });
 
         super.start();
diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
index ba1d384..8f5fb8a 100644
--- a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
+++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java
@@ -18,15 +18,19 @@
 
 package org.apache.bookkeeper.tests.containers;
 
+import static java.time.temporal.ChronoUnit.SECONDS;
+
+import java.time.Duration;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.tests.containers.wait.ZKWaitStrategy;
+import org.apache.bookkeeper.tests.containers.wait.HttpWaitStrategy;
 
 @Slf4j
 public class ZKContainer<SELF extends ZKContainer<SELF>> extends MetadataStoreContainer<SELF> {
 
     private static final int ZK_PORT = 2181;
+    private static final int ZK_HTTP_PORT = 8080;
 
-    private static final String IMAGE_NAME = "zookeeper:3.4.11";
+    private static final String IMAGE_NAME = "apachebookkeeper/bookkeeper-current:latest";
     public static final String HOST_NAME = "metadata-store";
     public static final String SERVICE_URI = "zk://" + HOST_NAME + ":" + ZK_PORT + "/ledgers";
 
@@ -46,12 +50,21 @@ public class ZKContainer<SELF extends ZKContainer<SELF>> extends MetadataStoreCo
 
     @Override
     protected void configure() {
-        addExposedPort(ZK_PORT);
+        addExposedPorts(
+            ZK_PORT,
+            ZK_HTTP_PORT);
+        setCommand("zookeeper");
+        addEnv("BK_admin.serverPort", "" + ZK_HTTP_PORT);
     }
 
     @Override
     public void start() {
-        this.waitStrategy = new ZKWaitStrategy(ZK_PORT);
+        this.waitStrategy = new HttpWaitStrategy()
+            .forPath("/commands/ruok")
+            .forStatusCode(200)
+            .forPort(ZK_HTTP_PORT)
+            .withStartupTimeout(Duration.of(60, SECONDS));
+
         this.withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd.withHostName(HOST_NAME));
 
         super.start();

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

[bookkeeper] 02/06: Refactor bookkeeper bash scripts and move dlog script to root bin directory

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 61ab2906dacec731f5a26cc766383e202d13a6d5
Author: Sijie Guo <si...@apache.org>
AuthorDate: Wed May 16 21:00:42 2018 -0700

    Refactor bookkeeper bash scripts and move dlog script to root bin directory
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    Since 4.7, we have moved bash scripts and configuration to the root directory. However the scripts and configurations for dlog and table-service modules are still in their own modules. It is inconvenient and confusing. This change mainly refactor current bookkeeper script to make it reusable for other modules.
    
    *Code Change*
    
    The main changes are:
    
    - abstract the common logic in `bin/bookkeeper` to `bin/common.sh`. These common logics include:
       * common definitions on environment variables, such jvm settings, bk conf, log4j conf, classpath and such.
       * common functions can be reusable, such as find jars, add maven dependencies.
    - simplify `bin/bookkeeper` and `bin/bookkeeper-cli` by reusing `bin/common.sh`
    - remove `stream/distributedlog/core/bin/dlog` to `bin/dlog` and simplify it by reusing `bin/common.sh`
    
    *Tests*
    
    Most of the changes in this PR are tests to ensure this script refactor is done correctly.
    
    - introduced a module `tests/scripts` for testing all the bash scripts in bookkeeper project. This module uses [shUnit2](https://github.com/kward/shunit2) for testing bash scripts. This gives a good test coverage on `bin/common.sh`.
    
    - add a few CLI smoketests under `tests/integration/smoke` to smoke test all CLI tools, including `bin/bookkeeper shell`, `bin/bookkeeper-cli` and `bin/dlog`. This makes sure all the CLI scripts work well after refactor.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Jia Zhai <None>
    
    This closes #1407 from sijie/include_dlog_library
---
 bin/bookkeeper                                     | 218 +++++-------------
 bin/bookkeeper-cli                                 | 141 +++---------
 bin/common.sh                                      | 251 ++++++++++++++++++++
 bin/dlog                                           | 131 +++++++++++
 bookkeeper-dist/all/pom.xml                        |  15 ++
 bookkeeper-dist/server/pom.xml                     |  14 ++
 .../src/main/resources/LICENSE-all.bin.txt         |   2 +
 .../src/main/resources/LICENSE-server.bin.txt      |   2 +
 conf/bk_cli_env.sh                                 |  20 +-
 conf/bkenv.sh                                      |  44 +++-
 conf/{bk_cli_env.sh => nettyenv.sh}                |  35 +--
 dev/check-binary-license                           |   5 +
 docker/scripts/entrypoint.sh                       |  44 ++++
 stream/distributedlog/core/bin/common.sh           | 124 ----------
 stream/distributedlog/core/bin/dlog                |  73 ------
 .../distributedlog/core/conf/bookie.conf.template  | 183 ---------------
 .../distributedlog/core/conf/distributedlog.conf   | 125 ----------
 stream/distributedlog/core/conf/dlogenv.sh         |  75 ------
 stream/distributedlog/core/conf/log4j.properties   |  56 -----
 stream/distributedlog/core/conf/write_proxy.conf   | 143 ------------
 .../core/conf/zookeeper.conf.dynamic.template      |   1 -
 .../core/conf/zookeeper.conf.template              |  82 -------
 .../bookkeeper/tests/BookKeeperClusterUtils.java   |  29 +++
 .../org/apache/bookkeeper/tests/DockerUtils.java   |  29 ++-
 .../tests/integration/BookieShellTestBase.java     |  79 +++++++
 .../tests/integration/TestBookieShellCluster.java  |  83 +++++++
 .../bookkeeper/tests/integration/TestCLI.java      | 102 ++++++++
 .../bookkeeper/tests/integration/TestDlogCLI.java  | 146 ++++++++++++
 .../bookkeeper/tests/integration/TestSmoke.java    |  25 +-
 tests/pom.xml                                      |   1 +
 tests/scripts/pom.xml                              |  75 ++++++
 tests/scripts/src/test/bash/bk_test.sh             | 141 ++++++++++++
 tests/scripts/src/test/bash/bk_test_bin_common.sh  | 256 +++++++++++++++++++++
 tests/scripts/src/test/bash/bk_test_helpers        |  94 ++++++++
 tests/scripts/src/test/bash/versions               | 173 ++++++++++++++
 35 files changed, 1823 insertions(+), 1194 deletions(-)

diff --git a/bin/bookkeeper b/bin/bookkeeper
index 6b5bacd..47c4e3f 100755
--- a/bin/bookkeeper
+++ b/bin/bookkeeper
@@ -18,109 +18,23 @@
 # * limitations under the License.
 # */
 
-# check if net.ipv6.bindv6only is set to 1
-bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
-if [ -n "$bindv6only" ] && [ "$bindv6only" -eq "1" ]
-then
-  echo "Error: \"net.ipv6.bindv6only\" is set to 1 - Java networking could be broken"
-  echo "For more info (the following page also applies to bookkeeper): http://wiki.apache.org/hadoop/HadoopIPv6"
-  exit 1
-fi
-
-# See the following page for extensive details on setting
-# up the JVM to accept JMX remote management:
-# http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
-# by default we allow local JMX connections
-if [ "x$JMXLOCALONLY" = "x" ]
-then
-  JMXLOCALONLY=false
-fi
-
-if [ "x$JMXDISABLE" = "x" ]
-then
-  # for some reason these two options are necessary on jdk6 on Ubuntu
-  #   accord to the docs they are not necessary, but otw jconsole cannot
-  #   do a local attach
-  JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY"
-else
-  echo "JMX disabled by user request" >&2
-fi
-
 BINDIR=`dirname "$0"`
-BK_HOME=`cd $BINDIR/..;pwd`
-
-DEFAULT_CONF=$BK_HOME/conf/bk_server.conf
-DEFAULT_LOG_CONF=$BK_HOME/conf/log4j.properties
-SERVER_HOME=${BK_HOME}/bookkeeper-server
-
-source $BK_HOME/conf/bkenv.sh
-
-LOCALBOOKIES_CONFIG_DIR="${LOCALBOOKIES_CONFIG_DIR:-/tmp/localbookies-config}"
+BK_HOME=`cd ${BINDIR}/..;pwd`
 
-# Check for the java to use
-if [[ -z $JAVA_HOME ]]; then
-  JAVA=$(which java)
-  if [ $? = 0 ]; then
-    echo "JAVA_HOME not set, using java from PATH. ($JAVA)"
-  else
-    echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
-    exit 1
-  fi
-else
-  JAVA=$JAVA_HOME/bin/java
-fi
+source ${BK_HOME}/bin/common.sh
 
-find-server-jar() {
-  DIR=$1
-  if [ -d $DIR ]; then
-    cd $DIR
-    for f in *.jar; do
-      if [[ $f =~ ^(org.apache.bookkeeper-)?bookkeeper-server-[0-9\\.]*(-SNAPSHOT)?.jar$ ]]; then
-        echo $DIR/$f
-          return
-      fi
-    done
-  fi
-}
+BOOKIE_MODULE_PATH=bookkeeper-server
+BOOKIE_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-server"
+BOOKIE_MODULE_HOME=${BK_HOME}/${BOOKIE_MODULE_PATH}
 
-RELEASE_JAR=$(find-server-jar ${BK_HOME})
-if [ -n "${RELEASE_JAR}" ]; then
-  BOOKIE_JAR=${RELEASE_JAR}
-else
-  RELEASE_JAR=$(find-server-jar ${BK_HOME}/lib)
-  if [ -n "${RELEASE_JAR}" ]; then
-    BOOKIE_JAR=${RELEASE_JAR}
-  fi
-fi
+# find the module jar
+BOOKIE_JAR=$(find_module_jar ${BOOKIE_MODULE_PATH} ${BOOKIE_MODULE_NAME})
 
-BUILT_JAR=$(find-server-jar ${SERVER_HOME}/target)
-
-if [ -z "${BUILT_JAR}" ] && [ -z "${BOOKIE_JAR}" ]; then
-  echo "Couldn't find bookkeeper jar."
-  read -p "Do you want me to run \`mvn package -DskiptTests\` for you ? " answer
-  case "${answer:0:1}" in
-    y|Y )
-      mvn package -DskipTests
-      ;;
-    * )
-      exit 1
-      ;;
-  esac
-
-  BUILT_JAR=$(find-server-jar ${SERVER_HOME}/target)
-  if [ -n "${BUILT_JAR}" ]; then
-    BOOKIE_JAR=$BUILT_JAR
-  fi
-fi
+# set up the classpath
+BOOKIE_CLASSPATH=$(set_module_classpath ${BOOKIE_MODULE_PATH})
 
-if [ -e "${BUILT_JAR}" ]; then
-  BOOKIE_JAR="${BUILT_JAR}"
-fi
-
-if [ ! -e "${BOOKIE_JAR}" ]; then
-  echo "Could not find bookkeeper jar."
-  exit 1
-fi
+# default variables
+DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf
 
 bookkeeper_help() {
     cat <<EOF
@@ -136,8 +50,8 @@ where command is one of:
 or command is the full name of a class with a defined main() method.
 
 Environment variables:
-   BOOKIE_LOG_CONF        Log4j configuration file (default $DEFAULT_LOG_CONF)
-   BOOKIE_CONF            Configuration file (default: $DEFAULT_CONF)
+   BOOKIE_LOG_CONF        Log4j configuration file (default ${DEFAULT_LOG_CONF})
+   BOOKIE_CONF            Configuration file (default: ${DEFAULT_CONF})
    BOOKIE_EXTRA_OPTS      Extra options to be passed to the jvm
    BOOKIE_EXTRA_CLASSPATH Add extra paths to the bookkeeper classpath
    ENTRY_FORMATTER_CLASS  Entry formatter class to format entries.
@@ -148,31 +62,6 @@ These variable can also be set in conf/bkenv.sh
 EOF
 }
 
-add_maven_deps_to_classpath() {
-  MVN="mvn"
-  if [ "$MAVEN_HOME" != "" ]; then
-    MVN=${MAVEN_HOME}/bin/mvn
-  fi
-
-  # Need to generate classpath from maven pom. This is costly so generate it
-  # and cache it. Save the file into our target dir so a mvn clean will get
-  # clean it up and force us create a new one.
-  f="${SERVER_HOME}/target/cached_classpath.txt"
-  if [ ! -f "${f}" ]
-  then
-    ${MVN} -f "${SERVER_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
-  fi
-  BOOKIE_CLASSPATH=${CLASSPATH}:`cat "${f}"`
-}
-
-if [ -d "$BK_HOME/lib" ]; then
-  for i in $BK_HOME/lib/*.jar; do
-    BOOKIE_CLASSPATH=$BOOKIE_CLASSPATH:$i
-  done
-else
-  add_maven_deps_to_classpath
-fi
-
 # if no args specified, show usage
 if [ $# = 0 ]; then
   bookkeeper_help;
@@ -183,67 +72,68 @@ fi
 COMMAND=$1
 shift
 
-if [ $COMMAND == "shell" ]; then
-  DEFAULT_LOG_CONF=$BK_HOME/conf/log4j.shell.properties
-    if [[ $1 == "-localbookie"  ]]; then
-        if [[ $2 == *:* ]];
-        then
-            BOOKIE_CONF=$LOCALBOOKIES_CONFIG_DIR/$2.conf
-            shift 2
-        else
-            BOOKIE_CONF=$LOCALBOOKIES_CONFIG_DIR/baseconf.conf
-            shift
-        fi
+LOCALBOOKIES_CONFIG_DIR="${LOCALBOOKIES_CONFIG_DIR:-/tmp/localbookies-config}"
+if [ ${COMMAND} == "shell" ]; then
+  DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.shell.properties
+  if [[ $1 == "-localbookie"  ]]; then
+    if [[ $2 == *:* ]];
+    then
+      BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/$2.conf
+      shift 2
+    else
+      BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/baseconf.conf
+      shift
     fi
+  fi
 fi
 
 if [ -z "$BOOKIE_CONF" ]; then
-  BOOKIE_CONF=$DEFAULT_CONF
+  BOOKIE_CONF=${DEFAULT_CONF}
 fi
 
+# Configure logging
 if [ -z "$BOOKIE_LOG_CONF" ]; then
-  BOOKIE_LOG_CONF=$DEFAULT_LOG_CONF
+  BOOKIE_LOG_CONF=${DEFAULT_LOG_CONF}
 fi
+BOOKIE_LOG_DIR=${BOOKIE_LOG_DIR:-"$BK_HOME/logs"}
+BOOKIE_LOG_FILE=${BOOKIE_LOG_FILE:-"bookkeeper-server.log"}
+BOOKIE_ROOT_LOGGER=${BOOKIE_ROOT_LOGGER:-"INFO,CONSOLE"}
 
+# Configure the classpath
 BOOKIE_CLASSPATH="$BOOKIE_JAR:$BOOKIE_CLASSPATH:$BOOKIE_EXTRA_CLASSPATH"
 BOOKIE_CLASSPATH="`dirname $BOOKIE_LOG_CONF`:$BOOKIE_CLASSPATH"
-OPTS="$OPTS -Dlog4j.configuration=`basename $BOOKIE_LOG_CONF`"
-
-OPTS="-cp $BOOKIE_CLASSPATH $OPTS"
 
-OPTS="$OPTS $BOOKIE_EXTRA_OPTS"
+# Build the OPTS
+BOOKIE_OPTS=$(build_bookie_opts)
+GC_OPTS=$(build_bookie_jvm_opts ${BOOKIE_LOG_DIR} "gc_%p.log")
+NETTY_OPTS=$(build_netty_opts)
+LOGGING_OPTS=$(build_logging_opts ${BOOKIE_LOG_CONF} ${BOOKIE_LOG_DIR} ${BOOKIE_LOG_FILE} ${BOOKIE_ROOT_LOGGER})
 
-# Disable ipv6 as it can cause issues
-OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
+OPTS="${OPTS} -cp ${BOOKIE_CLASSPATH} ${BOOKIE_OPTS} ${GC_OPTS} ${NETTY_OPTS} ${LOGGING_OPTS} ${BOOKIE_EXTRA_OPTS}"
 
-# log directory & file
-BOOKIE_ROOT_LOGGER=${BOOKIE_ROOT_LOGGER:-"INFO,CONSOLE"}
-BOOKIE_LOG_DIR=${BOOKIE_LOG_DIR:-"$BK_HOME/logs"}
-BOOKIE_LOG_FILE=${BOOKIE_LOG_FILE:-"bookkeeper-server.log"}
-
-#Configure log configuration system properties
-OPTS="$OPTS -Dbookkeeper.root.logger=$BOOKIE_ROOT_LOGGER"
-OPTS="$OPTS -Dbookkeeper.log.dir=$BOOKIE_LOG_DIR"
-OPTS="$OPTS -Dbookkeeper.log.file=$BOOKIE_LOG_FILE"
+# Create log dir if it doesn't exist
+if [ ! -d ${BOOKIE_LOG_DIR} ]; then
+    mkdir ${BOOKIE_LOG_DIR}
+fi
 
 #Change to BK_HOME to support relative paths
 cd "$BK_HOME"
-if [ $COMMAND == "bookie" ]; then
-  exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF $@
-elif [ $COMMAND == "autorecovery" ]; then
-  exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF $@
-elif [ $COMMAND == "localbookie" ]; then
+if [ ${COMMAND} == "bookie" ]; then
+  exec ${JAVA} ${OPTS} ${JMX_ARGS} org.apache.bookkeeper.server.Main --conf ${BOOKIE_CONF} $@
+elif [ ${COMMAND} == "autorecovery" ]; then
+  exec ${JAVA} ${OPTS} ${JMX_ARGS} org.apache.bookkeeper.replication.AutoRecoveryMain --conf ${BOOKIE_CONF} $@
+elif [ ${COMMAND} == "localbookie" ]; then
   NUMBER=$1
   shift
-  exec $JAVA $OPTS $JMX_ARGS -Dzookeeper.4lw.commands.whitelist='*' org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF $@
-elif [ $COMMAND == "upgrade" ]; then
-  exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF $@
-elif [ $COMMAND == "shell" ]; then
+  exec ${JAVA} ${OPTS} ${JMX_ARGS} -Dzookeeper.4lw.commands.whitelist='*' org.apache.bookkeeper.util.LocalBookKeeper ${NUMBER} ${BOOKIE_CONF} $@
+elif [ ${COMMAND} == "upgrade" ]; then
+  exec ${JAVA} ${OPTS} org.apache.bookkeeper.bookie.FileSystemUpgrade --conf ${BOOKIE_CONF} $@
+elif [ ${COMMAND} == "shell" ]; then
   ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}"
-  exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@
-elif [ $COMMAND == "help" ]; then
+  exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@
+elif [ ${COMMAND} == "help" ]; then
   bookkeeper_help;
 else
-  exec $JAVA $OPTS $COMMAND $@
+  exec ${JAVA} ${OPTS} ${COMMAND} $@
 fi
 
diff --git a/bin/bookkeeper-cli b/bin/bookkeeper-cli
index 31d9f60..2e53be0 100755
--- a/bin/bookkeeper-cli
+++ b/bin/bookkeeper-cli
@@ -21,133 +21,46 @@
 # BookKeeper CLI (experimental)
 
 BINDIR=`dirname "$0"`
-BK_HOME=`cd $BINDIR/..;pwd`
+BK_HOME=`cd ${BINDIR}/..;pwd`
 
-DEFAULT_CONF=$BK_HOME/conf/bk_server.conf
-DEFAULT_LOG_CONF=$BK_HOME/conf/log4j.cli.properties
-TOOLS_HOME=${BK_HOME}/bookkeeper-tools
+source ${BK_HOME}/bin/common.sh
+source ${BK_HOME}/conf/bk_cli_env.sh
 
-source $BK_HOME/conf/bk_cli_env.sh
+CLI_MODULE_PATH=bookkeeper-tools
+CLI_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-tools"
+CLI_MODULE_HOME=${BK_HOME}/${CLI_MODULE_PATH}
 
-# Check for the java to use
-if [[ -z $JAVA_HOME ]]; then
-  JAVA=$(which java)
-  if [ $? = 0 ]; then
-    echo "JAVA_HOME not set, using java from PATH. ($JAVA)"
-  else
-    echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
-    exit 1
-  fi
-else
-  JAVA=$JAVA_HOME/bin/java
-fi
-
-find_cli_jar() {
-  DIR=$1
-  if [ -d $DIR ]; then
-    cd $DIR
-    for f in *.jar; do
-      if [[ $f =~ ^(org.apache.bookkeeper-)?bookkeeper-tools-[0-9\\.]*(-SNAPSHOT)?.jar$ ]]; then
-        echo $DIR/$f
-          return
-      fi
-    done
-  fi
-}
-
-RELEASE_JAR=$(find_cli_jar ${BK_HOME})
-if [ -n "${RELEASE_JAR}" ]; then
-  CLI_JAR=${RELEASE_JAR}
-else
-  RELEASE_JAR=$(find_cli_jar ${BK_HOME}/lib)
-  if [ -n "${RELEASE_JAR}" ]; then
-    CLI_JAR=${RELEASE_JAR}
-  fi
-fi
-
-BUILT_JAR=$(find_cli_jar ${TOOLS_HOME}/target)
-
-if [ -z "${BUILT_JAR}" ] && [ -z "${CLI_JAR}" ]; then
-  echo "Couldn't find bookkeeper jar."
-  read -p "Do you want me to run \`mvn package -DskiptTests\` for you ? " answer
-  case "${answer:0:1}" in
-    y|Y )
-      mvn package -DskipTests
-      ;;
-    * )
-      exit 1
-      ;;
-  esac
-
-  BUILT_JAR=$(find_cli_jar ${TOOLS_HOME}/target)
-  if [ -n "${BUILT_JAR}" ]; then
-    CLI_JAR=$BUILT_JAR
-  fi
-fi
-
-if [ -e "${BUILT_JAR}" ]; then
-  CLI_JAR="${BUILT_JAR}"
-fi
-
-if [ ! -e "${CLI_JAR}" ]; then
-  echo "Could not find bookkeeper cli jar."
-  exit 1
-fi
+# find the module jar
+CLI_JAR=$(find_module_jar ${CLI_MODULE_PATH} ${CLI_MODULE_NAME})
 
-add_maven_deps_to_classpath() {
-  MVN="mvn"
-  if [ "$MAVEN_HOME" != "" ]; then
-    MVN=${MAVEN_HOME}/bin/mvn
-  fi
+# set up the classpath
+CLI_CLASSPATH=$(set_module_classpath ${CLI_MODULE_PATH})
 
-  # Need to generate classpath from maven pom. This is costly so generate it
-  # and cache it. Save the file into our target dir so a mvn clean will get
-  # clean it up and force us create a new one.
-  f="${TOOLS_HOME}/target/cached_classpath.txt"
-  if [ ! -f "${f}" ]
-  then
-    ${MVN} -f "${TOOLS_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
-  fi
-  CLI_CLASSPATH=${CLASSPATH}:`cat "${f}"`
-}
-
-if [ -d "$BK_HOME/lib" ]; then
-  for i in $BK_HOME/lib/*.jar; do
-    CLI_CLASSPATH=$CLI_CLASSPATH:$i
-  done
-else
-  add_maven_deps_to_classpath
-fi
-
-if [ -z "$CLI_CONF" ]; then
-  CLI_CONF=$DEFAULT_CONF
+DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf
+if [ -z "${CLI_CONF}" ]; then
+  CLI_CONF=${DEFAULT_CONF}
 fi
 
-if [ -z "$CLI_LOG_CONF" ]; then
-  CLI_LOG_CONF=$DEFAULT_LOG_CONF
+DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.cli.properties
+if [ -z "${CLI_LOG_CONF}" ]; then
+  CLI_LOG_CONF=${DEFAULT_LOG_CONF}
 fi
+CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"}
+CLI_LOG_FILE=${CLI_LOG_FILE:-"bookkeeper-cli.log"}
+CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"}
 
+# Configure the classpath
 CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH"
 CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH"
-OPTS="$OPTS -Dlog4j.configuration=`basename $CLI_LOG_CONF`"
 
-OPTS="-cp $CLI_CLASSPATH $OPTS"
-
-OPTS="$OPTS $CLI_EXTRA_OPTS"
-
-# Disable ipv6 as it can cause issues
-OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
-
-# log directory & file
-CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"}
-CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"}
-CLI_LOG_FILE=${CLI_LOG_FILE:-"bookkeeper-cli.log"}
+# Build the OPTs
+BOOKIE_OPTS=$(build_bookie_opts)
+GC_OPTS=$(build_cli_jvm_opts ${CLI_LOG_DIR} "bookkeeper-cli-gc.log")
+NETTY_OPTS=$(build_netty_opts)
+LOGGING_OPTS=$(build_logging_opts ${CLI_LOG_CONF} ${CLI_LOG_DIR} ${CLI_LOG_FILE} ${CLI_ROOT_LOGGER})
 
-#Configure log configuration system properties
-OPTS="$OPTS -Dbookkeeper.cli.root.logger=$CLI_ROOT_LOGGER"
-OPTS="$OPTS -Dbookkeeper.cli.log.dir=$CLI_LOG_DIR"
-OPTS="$OPTS -Dbookkeeper.cli.log.file=$CLI_LOG_FILE"
+OPTS="${OPTS} -cp ${CLI_CLASSPATH} ${BOOKIE_OPTS} ${GC_OPTS} ${NETTY_OPTS} ${LOGGING_OPTS} ${CLI_EXTRA_OPTS}"
 
 #Change to BK_HOME to support relative paths
 cd "$BK_HOME"
-exec $JAVA $OPTS org.apache.bookkeeper.tools.cli.BookieShell --conf $CLI_CONF $@
+exec ${JAVA} ${OPTS} org.apache.bookkeeper.tools.cli.BookKeeperCLI --conf ${CLI_CONF} $@
diff --git a/bin/common.sh b/bin/common.sh
new file mode 100755
index 0000000..0c048c9
--- /dev/null
+++ b/bin/common.sh
@@ -0,0 +1,251 @@
+#!/usr/bin/env bash
+#
+#/**
+# * 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.
+# */
+
+# Check net.ipv6.bindv6only
+if [ -f /sbin/sysctl ]; then
+  # check if net.ipv6.bindv6only is set to 1
+  bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
+  if [ -n "$bindv6only" ] && [ "$bindv6only" -eq "1" ]
+  then
+    echo "Error: \"net.ipv6.bindv6only\" is set to 1 - Java networking could be broken"
+    echo "For more info (the following page also applies to bookkeeper): http://wiki.apache.org/hadoop/HadoopIPv6"
+    exit 1
+  fi
+fi
+
+# See the following page for extensive details on setting
+# up the JVM to accept JMX remote management:
+# http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
+# by default we allow local JMX connections
+if [ "x$JMXLOCALONLY" = "x" ]
+then
+  JMXLOCALONLY=false
+fi
+
+if [ "x$JMXDISABLE" = "x" ]
+then
+  # for some reason these two options are necessary on jdk6 on Ubuntu
+  #   accord to the docs they are not necessary, but otw jconsole cannot
+  #   do a local attach
+  JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY"
+else
+  echo "JMX disabled by user request" >&2
+fi
+
+# Check for the java to use
+if [[ -z ${JAVA_HOME} ]]; then
+  JAVA=$(which java)
+  if [ $? = 0 ]; then
+    echo "JAVA_HOME not set, using java from PATH. ($JAVA)"
+  else
+    echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
+    exit 1
+  fi
+else
+  JAVA=${JAVA_HOME}/bin/java
+fi
+
+BINDIR=${BK_BINDIR:-"`dirname "$0"`"}
+BK_HOME=${BK_HOME:-"`cd ${BINDIR}/..;pwd`"}
+BK_CONFDIR=${BK_HOME}/conf
+DEFAULT_LOG_CONF=${BK_CONFDIR}/log4j.properties
+
+source ${BK_CONFDIR}/nettyenv.sh
+source ${BK_CONFDIR}/bkenv.sh
+source ${BK_CONFDIR}/bk_cli_env.sh
+
+# default netty settings
+NETTY_LEAK_DETECTION_LEVEL=${NETTY_LEAK_DETECTION_LEVEL:-"disabled"}
+NETTY_RECYCLER_MAXCAPACITY=${NETTY_RECYCLER_MAXCAPACITY:-"1000"}
+NETTY_RECYCLER_LINKCAPACITY=${NETTY_RECYCLER_LINKCAPACITY:-"1024"}
+
+# default bookie JVM settings
+DEFAULT_BOOKIE_GC_OPTS="-XX:+UseG1GC \
+    -XX:MaxGCPauseMillis=10 \
+    -XX:+ParallelRefProcEnabled \
+    -XX:+UnlockExperimentalVMOptions \
+    -XX:+AggressiveOpts \
+    -XX:+DoEscapeAnalysis \
+    -XX:ParallelGCThreads=32 \
+    -XX:ConcGCThreads=32 \
+    -XX:G1NewSizePercent=50 \
+    -XX:+DisableExplicitGC \
+    -XX:-ResizePLAB"
+DEFAULT_BOOKIE_GC_LOGGING_OPTS="-XX:+PrintGCDetails \
+    -XX:+PrintGCApplicationStoppedTime  \
+    -XX:+UseGCLogFileRotation \
+    -XX:NumberOfGCLogFiles=5 \
+    -XX:GCLogFileSize=64m"
+BOOKIE_MAX_HEAP_MEMORY=${BOOKIE_MAX_HEAP_MEMORY:-"1g"}
+BOOKIE_MIN_HEAP_MEMORY=${BOOKIE_MIN_HEAP_MEMORY:-"1g"}
+BOOKIE_MAX_DIRECT_MEMORY=${BOOKIE_MAX_DIRECT_MEMORY:-"2g"}
+BOOKIE_MEM_OPTS=${BOOKIE_MEM_OPTS:-"-Xms${BOOKIE_MIN_HEAP_MEMORY} -Xmx${BOOKIE_MAX_HEAP_MEMORY} -XX:MaxDirectMemorySize=${BOOKIE_MAX_DIRECT_MEMORY}"}
+BOOKIE_GC_OPTS=${BOOKIE_GC_OPTS:-"${DEFAULT_BOOKIE_GC_OPTS}"}
+BOOKIE_GC_LOGGING_OPTS=${BOOKIE_GC_LOGGING_OPTS:-"${DEFAULT_BOOKIE_GC_LOGGING_OPTS}"}
+
+# default CLI JVM settings
+DEFAULT_CLI_GC_OPTS="-XX:+UseG1GC \
+    -XX:MaxGCPauseMillis=10"
+DEFAULT_CLI_GC_LOGGING_OPTS="-XX:+PrintGCDetails \
+    -XX:+PrintGCApplicationStoppedTime  \
+    -XX:+UseGCLogFileRotation \
+    -XX:NumberOfGCLogFiles=5 \
+    -XX:GCLogFileSize=64m"
+CLI_MAX_HEAP_MEMORY=${CLI_MAX_HEAP_MEMORY:-"512M"}
+CLI_MIN_HEAP_MEMORY=${CLI_MIN_HEAP_MEMORY:-"256M"}
+CLI_MEM_OPTS=${CLI_MEM_OPTS:-"-Xms${CLI_MIN_HEAP_MEMORY} -Xmx${CLI_MAX_HEAP_MEMORY}"}
+CLI_GC_OPTS=${CLI_GC_OPTS:-"${DEFAULT_CLI_GC_OPTS}"}
+CLI_GC_LOGGING_OPTS=${CLI_GC_LOGGING_OPTS:-"${DEFAULT_CLI_GC_LOGGING_OPTS}"}
+
+find_module_jar_at() {
+  DIR=$1
+  MODULE=$2
+  REGEX="^${MODULE}-[0-9\\.]*(-SNAPSHOT)?.jar$"
+  if [ -d ${DIR} ]; then
+    cd ${DIR}
+    for f in *.jar; do
+      if [[ ${f} =~ ${REGEX} ]]; then
+        echo ${DIR}/${f}
+        return
+      fi
+    done
+  fi
+}
+
+find_module_jar() {
+  MODULE_PATH=$1
+  MODULE_NAME=$2
+  RELEASE_JAR=$(find_module_jar_at ${BK_HOME} ${MODULE_NAME})
+  if [ -n "${RELEASE_JAR}" ]; then
+    MODULE_JAR=${RELEASE_JAR}
+  else
+    RELEASE_JAR=$(find_module_jar_at ${BK_HOME}/lib ${MODULE_NAME})
+    if [ -n "${RELEASE_JAR}" ]; then
+      MODULE_JAR=${RELEASE_JAR}
+    fi
+  fi
+
+  if [ -z "${MODULE_JAR}" ]; then
+    BUILT_JAR=$(find_module_jar_at ${BK_HOME}/${MODULE_PATH}/target ${MODULE_NAME})
+    if [ -z "${BUILT_JAR}" ]; then
+      echo "Couldn't find module '${MODULE_NAME}' jar." >&2
+      read -p "Do you want me to run \`mvn package -DskiptTests\` for you ? " answer
+      case "${answer:0:1}" in
+        y|Y )
+          mvn package -DskipTests
+          ;;
+        * )
+          exit 1
+          ;;
+      esac
+
+      BUILT_JAR=$(find_module_jar_at ${BK_HOME}/${MODULE_PATH}/target ${MODULE_NAME})
+    fi
+    if [ -n "${BUILT_JAR}" ]; then
+      MODULE_JAR=${BUILT_JAR}
+    fi
+  fi
+
+  if [ ! -e "${MODULE_JAR}" ]; then
+    echo "Could not find module '${MODULE_JAR}' jar." >&2
+    exit 1
+  fi
+  echo ${MODULE_JAR}
+  return
+}
+
+add_maven_deps_to_classpath() {
+  MODULE_PATH=$1
+  MVN="mvn"
+  if [ "$MAVEN_HOME" != "" ]; then
+    MVN=${MAVEN_HOME}/bin/mvn
+  fi
+
+  # Need to generate classpath from maven pom. This is costly so generate it
+  # and cache it. Save the file into our target dir so a mvn clean will get
+  # clean it up and force us create a new one.
+  f="${MODULE_PATH}/target/cached_classpath.txt"
+  if [ ! -f ${f} ]; then
+    ${MVN} -f "${MODULE_PATH}/pom.xml" dependency:build-classpath -Dmdep.outputFile="target/cached_classpath.txt" &> /dev/null
+  fi
+}
+
+set_module_classpath() {
+  MODULE_PATH=$1
+  if [ -d "${BK_HOME}/lib" ]; then
+    BK_CLASSPATH=""
+    for i in ${BK_HOME}/lib/*.jar; do
+      BK_CLASSPATH=${BK_CLASSPATH}:${i}
+    done
+    echo ${BK_CLASSPATH}
+  else
+    add_maven_deps_to_classpath ${MODULE_PATH} >&2
+    cat ${MODULE_PATH}/target/cached_classpath.txt
+  fi
+  return
+}
+
+build_bookie_jvm_opts() {
+  LOG_DIR=$1
+  GC_LOG_FILENAME=$2
+
+  echo "$BOOKIE_MEM_OPTS $BOOKIE_GC_OPTS $BOOKIE_GC_LOGGING_OPTS $BOOKIE_PERF_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}"
+}
+
+build_cli_jvm_opts() {
+  LOG_DIR=$1
+  GC_LOG_FILENAME=$2
+
+  echo "$CLI_MEM_OPTS $CLI_GC_OPTS $CLI_GC_LOGGING_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}"
+}
+
+build_netty_opts() {
+  echo "-Dio.netty.leakDetectionLevel=${NETTY_LEAK_DETECTION_LEVEL} \
+    -Dio.netty.recycler.maxCapacity.default=${NETTY_RECYCLER_MAXCAPACITY} \
+    -Dio.netty.recycler.linkCapacity=${NETTY_RECYCLER_LINKCAPACITY}"
+}
+
+build_logging_opts() {
+  CONF_FILE=$1
+  LOG_DIR=$2
+  LOG_FILE=$3
+  LOGGER=$4
+
+  echo "-Dlog4j.configuration=`basename ${CONF_FILE}` \
+    -Dbookkeeper.root.logger=${LOGGER} \
+    -Dbookkeeper.log.dir=${LOG_DIR} \
+    -Dbookkeeper.log.file=${LOG_FILE}"
+}
+
+build_cli_logging_opts() {
+  CONF_FILE=$1
+  LOG_DIR=$2
+  LOG_FILE=$3
+  LOGGER=$4
+
+  echo "-Dlog4j.configuration=`basename ${CONF_FILE}` \
+    -Dbookkeeper.cli.root.logger=${LOGGER} \
+    -Dbookkeeper.cli.log.dir=${LOG_DIR} \
+    -Dbookkeeper.cli.log.file=${LOG_FILE}"
+}
+
+build_bookie_opts() {
+  echo "-Djava.net.preferIPv4Stack=true"
+}
diff --git a/bin/dlog b/bin/dlog
new file mode 100755
index 0000000..9a9b8f1
--- /dev/null
+++ b/bin/dlog
@@ -0,0 +1,131 @@
+#!/usr/bin/env bash
+#
+# vim:et:ft=sh:sts=2:sw=2
+#
+#/**
+# * 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.
+# */
+
+BINDIR=`dirname "$0"`
+BK_HOME=`cd ${BINDIR}/..;pwd`
+
+source ${BK_HOME}/bin/common.sh
+
+DLOG_MODULE_NAME="(org.apache.distributedlog-)?distributedlog-core"
+DLOG_MODULE_PATH=stream/distributedlog/core
+DLOG_MODULE_HOME=${BK_HOME}/${DLOG_MODULE_PATH}
+
+# find the module jar
+DLOG_JAR=$(find_module_jar ${DLOG_MODULE_PATH} ${DLOG_MODULE_NAME})
+
+# set up the classpath
+DLOG_CLASSPATH=$(set_module_classpath ${DLOG_MODULE_PATH})
+
+# default variables
+DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf
+DEFAULT_CLI_CONF=${BK_HOME}/conf/bk_server.conf
+DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.properties
+DEFAULT_CLI_LOG_CONF=${BK_HOME}/conf/log4j.cli.properties
+
+dlog_help() {
+    cat <<EOF
+Usage: dlog <command>
+where command is one of:
+    local               Run distributedlog sandbox
+    tool                Run distributedlog tool
+    admin               Run distributedlog admin tool
+    help                This help message
+
+or command is the full name of a class with a defined main() method.
+
+Environment variables:
+
+    BOOKIE_CONF            Bookie configuration file (default: ${DEFAULT_CONF})
+    BOOKIE_EXTRA_OPTS      Extra options to be passed to the bookie jvm
+    BOOKIE_EXTRA_CLASSPATH Add extra paths to the bookie classpath
+    CLI_CONF               CLI configuration file (default: ${DEFAULT_CLI_CONF})
+    CLI_EXTRA_OPTS         Extra options to be passed to the CLI jvm
+    CLI_EXTRA_CLASSPATH    Add extra paths to the CLI classpath
+
+These variable can also be set in conf/bkenv.sh & conf/bk_cli_env.sh
+EOF
+}
+
+# if no args specified, show usage
+if [ $# = 0 ]; then
+  dlog_help;
+  exit 1;
+fi
+
+# get arguments
+COMMAND=$1
+shift
+
+if [ ${COMMAND} == "local" ]; then
+  DLOG_CONF=${BOOKIE_CONF:-"${DEFAULT_CONF}"}
+  DLOG_LOG_CONF=${BOOKIE_LOG_CONF:-"${DEFAULT_LOG_CONF}"}
+  DLOG_LOG_DIR=${BOOKIE_LOG_DIR:-"${BK_HOME}/logs"}
+  DLOG_LOG_FILE=${BOOKIE_LOG_FILE:-"dlog.log"}
+  DLOG_ROOT_LOGGER=${BOOKIE_ROOT_LOGGER:-"INFO,CONSOLE"}
+  DLOG_EXTRA_CLASSPATH=${BOOKIE_EXTRA_CLASSPATH}
+  DLOG_GC_OPTS=$(build_bookie_jvm_opts ${DLOG_LOG_DIR} "dlog_gc_%p.log")
+  DLOG_EXTRA_OPTS=${BOOKIE_EXTRA_OPTS}
+  DLOG_LOGGING_OPTS=$(build_logging_opts ${DLOG_LOG_CONF} ${DLOG_LOG_DIR} ${DLOG_LOG_FILE} ${DLOG_ROOT_LOGGER})
+else
+  DLOG_CONF=${CLI_CONF:-"${DEFAULT_CLI_CONF}"}
+  DLOG_LOG_CONF=${CLI_LOG_CONF:-"${DEFAULT_CLI_LOG_CONF}"}
+  DLOG_LOG_DIR=${CLI_LOG_DIR:-"${BK_HOME}/logs"}
+  DLOG_LOG_FILE=${CLI_LOG_FILE:-"dlog-cli.log"}
+  DLOG_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"}
+  DLOG_EXTRA_CLASSPATH=${CLI_EXTRA_CLASSPATH}
+  DLOG_GC_OPTS=$(build_cli_jvm_opts ${DLOG_LOG_DIR} "dlog-cli-gc.log")
+  DLOG_EXTRA_OPTS=${CLI_EXTRA_OPTS}
+  DLOG_LOGGING_OPTS=$(build_cli_logging_opts ${DLOG_LOG_CONF} ${DLOG_LOG_DIR} ${DLOG_LOG_FILE} ${DLOG_ROOT_LOGGER})
+fi
+
+DLOG_CLASSPATH="$DLOG_JAR:$DLOG_CLASSPATH:$DLOG_EXTRA_CLASSPATH"
+DLOG_CLASSPATH="`dirname $DLOG_LOG_CONF`:$DLOG_CLASSPATH"
+
+# Build the OPTS
+BOOKIE_OPTS=$(build_bookie_opts)
+NETTY_OPTS=$(build_netty_opts)
+OPTS="${OPTS} -cp ${DLOG_CLASSPATH} ${BOOKIE_OPTS} ${DLOG_GC_OPTS} ${NETTY_OPTS} ${DLOG_LOGGING_OPTS} ${DLOG_EXTRA_OPTS}"
+
+if [ ! -d ${DLOG_LOG_DIR} ]; then
+    mkdir ${DLOG_LOG_DIR}
+fi
+
+#Change to BK_HOME to support relative paths
+cd "$BK_HOME"
+case "${COMMAND}" in
+  local)
+    exec ${JAVA} ${OPTS} ${JMX_ARGS} -Dzookeeper.4lw.commands.whitelist='*' org.apache.distributedlog.LocalDLMEmulator $@
+    ;;
+  tool)
+    exec ${JAVA} ${OPTS} org.apache.distributedlog.tools.Tool org.apache.distributedlog.tools.DistributedLogTool $@
+    ;;
+  admin)
+    exec ${JAVA} ${OPTS} org.apache.distributedlog.tools.Tool org.apache.distributedlog.admin.DistributedLogAdmin $@
+    ;;
+  help)
+    dlog_help
+    ;;
+  *)
+    exec ${JAVA} ${OPTS} ${COMMAND} $@
+    ;;
+esac
+
diff --git a/bookkeeper-dist/all/pom.xml b/bookkeeper-dist/all/pom.xml
index b8f8916..8a10382 100644
--- a/bookkeeper-dist/all/pom.xml
+++ b/bookkeeper-dist/all/pom.xml
@@ -78,6 +78,21 @@
       <version>${project.version}</version>
     </dependency>
 
+    <!-- bookkeeper.tools (new CLI) -->
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-tools</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- dlog -->
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- bookkeeper benchmark -->
     <dependency>
       <groupId>org.apache.bookkeeper</groupId>
       <artifactId>bookkeeper-benchmark</artifactId>
diff --git a/bookkeeper-dist/server/pom.xml b/bookkeeper-dist/server/pom.xml
index 287aa07..e4f398a 100644
--- a/bookkeeper-dist/server/pom.xml
+++ b/bookkeeper-dist/server/pom.xml
@@ -62,6 +62,20 @@
       <version>${project.version}</version>
     </dependency>
 
+    <!-- bookkeeper.tools (new CLI) -->
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-tools</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- dlog -->
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
     <!-- slf4j binding -->
     <dependency>
       <groupId>org.slf4j</groupId>
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index e4bb88b..b0ba7e9 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -273,6 +273,7 @@ Apache Software License, Version 2.
 - lib/com.beust-jcommander-1.48.jar [36]
 - lib/com.yahoo.datasketches-memory-0.8.3.jar [37]
 - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37]
+- lib/net.jpountz.lz4-lz4-1.3.0.jar [38]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -310,6 +311,7 @@ Apache Software License, Version 2.
 [35] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
 [36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 [37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
+[38] Source available at https://github.com/lz4/lz4-java/tree/1.3.0
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index bd0dba2..9f9ccc6 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -238,6 +238,7 @@ Apache Software License, Version 2.
 - lib/com.beust-jcommander-1.48.jar [23]
 - lib/com.yahoo.datasketches-memory-0.8.3.jar [24]
 - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [24]
+- lib/net.jpountz.lz4-lz4-1.3.0.jar [25]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -263,6 +264,7 @@ Apache Software License, Version 2.
 [22] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 [24] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
+[25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-all-4.1.12.Final.jar bundles some 3rd party dependencies
diff --git a/conf/bk_cli_env.sh b/conf/bk_cli_env.sh
index 2f57061..8faa03d 100644
--- a/conf/bk_cli_env.sh
+++ b/conf/bk_cli_env.sh
@@ -23,7 +23,9 @@
 # Set JAVA_HOME here to override the environment setting
 # JAVA_HOME=
 
+########################################
 # default settings for bookkeeper cli
+########################################
 
 # Configuration file of settings used in bookkeeper cli
 # CLI_CONF=
@@ -34,9 +36,9 @@
 # Add extra paths to the bookkeeper classpath
 # CLI_EXTRA_CLASSPATH=
 
-#
+#################################
 # CLI Logging Options
-#
+#################################
 
 # Log4j configuration file
 # CLI_LOG_CONF=
@@ -49,3 +51,17 @@
 
 # Log level & appender
 # CLI_ROOT_LOGGER="INFO,CONSOLE"
+
+#################################
+# JVM memory options
+#################################
+
+# CLI_MAX_HEAP_MEMORY=512M
+# CLI_MIN_HEAP_MEMORY=256M
+# CLI_MEM_OPTS=
+
+# JVM GC options
+# CLI_GC_OPTS=
+
+# JVM GC logging options
+# CLI_GC_LOGGING_OPTS=
diff --git a/conf/bkenv.sh b/conf/bkenv.sh
index 8900b3d..eebc4ff 100644
--- a/conf/bkenv.sh
+++ b/conf/bkenv.sh
@@ -21,17 +21,13 @@
 # Set JAVA_HOME here to override the environment setting
 # JAVA_HOME=
 
-# default settings for starting bookkeeper
+########################################
+# default settings for bookkeeper
+########################################
 
 # Configuration file of settings used in bookie server
 # BOOKIE_CONF=
 
-# Log4j configuration file
-# BOOKIE_LOG_CONF=
-
-# Logs location
-# BOOKIE_LOG_DIR=
-
 # Extra options to be passed to the jvm
 # BOOKIE_EXTRA_OPTS=
 
@@ -49,3 +45,37 @@
 
 # this default config dir should match the 'localBookiesConfigDirectory' config value in the conf file of LocalBookKeeper
 # LOCALBOOKIES_CONFIG_DIR=/tmp/localbookies-config
+
+#################################
+# BookKeeper Logging Options
+#################################
+
+# Log4j configuration file
+# BOOKIE_LOG_CONF=
+
+# Logs location
+# BOOKIE_LOG_DIR=
+
+# Log file name
+# BOOKIE_LOG_FILE="bookkeeper.log"
+
+# Log level & appender
+# BOOKIE_ROOT_LOGGER="INFO,CONSOLE"
+
+#################################
+# BookKeeper JVM memory options
+#################################
+
+# BOOKIE_MAX_HEAP_MEMORY=1g
+# BOOKIE_MIN_HEAP_MEMORY=1g
+# BOOKIE_MAX_DIRECT_MEMORY=2g
+# BOOKIE_MEM_OPTS=
+
+# JVM GC options
+# BOOKIE_GC_OPTS=
+
+# JVM GC logging options
+# BOOKIE_GC_LOGGING_OPTS=
+
+# JVM performance options
+# BOOKIE_PERF_OPTS="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch -XX:-UseBiasedLocking"
diff --git a/conf/bk_cli_env.sh b/conf/nettyenv.sh
similarity index 59%
copy from conf/bk_cli_env.sh
copy to conf/nettyenv.sh
index 2f57061..c4702b0 100644
--- a/conf/bk_cli_env.sh
+++ b/conf/nettyenv.sh
@@ -18,34 +18,13 @@
 # * limitations under the License.
 # */
 
-# Environment Settings for BookKeeper CLI (experimental)
+# Netty options
 
-# Set JAVA_HOME here to override the environment setting
-# JAVA_HOME=
+# netty buffer leak detection level - {@link http://netty.io/wiki/reference-counted-objects.html#wiki-h3-11}
+# NETTY_LEAK_DETECTION_LEVEL=
 
-# default settings for bookkeeper cli
+# netty recycler max capacity
+# NETTY_RECYCLER_MAXCAPACITY=
 
-# Configuration file of settings used in bookkeeper cli
-# CLI_CONF=
-
-# Extra options to be passed to the jvm
-# CLI_EXTRA_OPTS=
-
-# Add extra paths to the bookkeeper classpath
-# CLI_EXTRA_CLASSPATH=
-
-#
-# CLI Logging Options
-#
-
-# Log4j configuration file
-# CLI_LOG_CONF=
-
-# Logs location
-# CLI_LOG_DIR=
-
-# Log file name
-# CLI_LOG_FILE="bookkeeper-cli.log"
-
-# Log level & appender
-# CLI_ROOT_LOGGER="INFO,CONSOLE"
+# netty recycler link capacity
+# NETTY_RECYCLER_LINKCAPACITY=
diff --git a/dev/check-binary-license b/dev/check-binary-license
index a150ba8..b56cbbf 100755
--- a/dev/check-binary-license
+++ b/dev/check-binary-license
@@ -59,6 +59,11 @@ for J in $JARS; do
         continue
     fi
 
+    echo $J | grep -q "org.apache.distributedlog"
+    if [ $? == 0 ]; then
+        continue
+    fi
+
     echo "$LICENSE" | grep -q $J
     if [ $? != 0 ]; then
         echo $J unaccounted for in LICENSE
diff --git a/docker/scripts/entrypoint.sh b/docker/scripts/entrypoint.sh
index 91e63c3..7ade924 100755
--- a/docker/scripts/entrypoint.sh
+++ b/docker/scripts/entrypoint.sh
@@ -37,12 +37,14 @@ export BK_journalDirectory=${BK_journalDirectory:-${BK_DATA_DIR}/journal}
 export BK_ledgerDirectories=${BK_ledgerDirectories:-${BK_DATA_DIR}/ledgers}
 export BK_indexDirectories=${BK_indexDirectories:-${BK_DATA_DIR}/index}
 export BK_metadataServiceUri=${BK_metadataServiceUri:-"zk://${BK_zkServers}${BK_zkLedgersRootPath}"}
+export BK_dlogRootPath=${BK_dlogRootPath:-"${BK_CLUSTER_ROOT_PATH}/distributedlog"}
 
 echo "BK_bookiePort bookie service port is $BK_bookiePort"
 echo "BK_zkServers is $BK_zkServers"
 echo "BK_DATA_DIR is $BK_DATA_DIR"
 echo "BK_CLUSTER_ROOT_PATH is $BK_CLUSTER_ROOT_PATH"
 echo "BK_metadataServiceUri is $BK_metadataServiceUri"
+echo "BK_dlogRootPath is $BK_dlogRootPath"
 
 mkdir -p "${BK_journalDirectory}" "${BK_ledgerDirectories}" "${BK_indexDirectories}"
 # -------------- #
@@ -109,6 +111,48 @@ else
     fi
 fi
 
+# Create default dlog namespace
+# Use ephemeral zk node as lock to keep initialize atomic.
+/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
+if [ $? -eq 0 ]; then
+    echo "Dlog namespace already created, no need to create another one"
+else
+    # create ephemeral zk node dlogInitLock, initiator who this node, then do init; other initiators will wait.
+    /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/dlogInitLock
+    if [ $? -eq 0 ]; then
+        # dlogInitLock created success, this is the successor to do znode init
+        echo "Dlog namespace not exist, do the init to create them."
+        /opt/bookkeeper/bin/dlog admin bind -l ${BK_zkLedgersRootPath} -s ${BK_zkServers} -c distributedlog://${BK_zkServers}${BK_dlogRootPath}
+        if [ $? -eq 0 ]; then
+            echo "Dlog namespace is created successfully."
+        else
+            echo "Failed to create dlog namespace ${BK_dlogRootPath}. please check the reason."
+            exit
+        fi
+    else
+        echo "Other docker instance is doing initialize at the same time, will wait in this instance."
+        tenSeconds=1
+        while [ ${tenSeconds} -lt 10 ]
+        do
+            sleep 10
+            /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_dlogRootPath}
+            if [ $? -eq 0 ]; then
+                echo "Waited $tenSeconds * 10 seconds, dlog namespace created"
+                break
+            else
+                echo "Waited $tenSeconds * 10 seconds, dlog namespace still not created"
+                (( tenSeconds++ ))
+                continue
+            fi
+        done
+
+        if [ ${tenSeconds} -eq 10 ]; then
+            echo "Waited 100 seconds for creating dlog namespace, something wrong, please check"
+            exit
+        fi
+    fi
+fi
+
 echo "run command by exec"
 exec "$@"
 
diff --git a/stream/distributedlog/core/bin/common.sh b/stream/distributedlog/core/bin/common.sh
deleted file mode 100755
index 2b13157..0000000
--- a/stream/distributedlog/core/bin/common.sh
+++ /dev/null
@@ -1,124 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * 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.
-# */
-
-if [ $(uname) == "Linux" ]; then
-  # check if net.ipv6.bindv6only is set to 1
-  bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
-  if [ -n "${bindv6only}" ] && [ "${bindv6only}" -eq "1" ]; then
-    echo "Error: \"net.ipv6.bindv6only\" is set to 1 - Java networking could be broken"
-    echo "For more info (the following page also applies to DistributedLog): http://wiki.apache.org/hadoop/HadoopIPv6"
-    exit 1
-  fi
-fi
-
-# See the following page for extensive details on setting
-# up the JVM to accept JMX remote management:
-# http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
-# by default we allow local JMX connections
-if [ -z "${JMXLOCALONLY}" ]; then
-  JMXLOCALONLY=false
-fi
-
-if [ -z "${JMXDISABLE}" ]; then
-  echo "JMX enabled by default" >&2
-  # for some reason these two options are necessary on jdk6 on Ubuntu
-  # accord to the docs they are not necessary, but otw jconsole cannot
-  # do a local attach
-  JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=${JMXLOCALONLY}"
-else
-  echo "JMX disabled by user request" >&2
-fi
-
-echo "DLOG_HOME => ${DLOG_HOME}"
-DEFAULT_LOG_CONF="${DLOG_HOME}/conf/log4j.properties"
-
-[ -f "${DLOG_HOME}/conf/dlogenv.sh" ] && source "${DLOG_HOME}/conf/dlogenv.sh"
-
-# exclude tests jar
-RELEASE_JAR=$(ls ${DLOG_HOME}/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
-if [ $? == 0 ]; then
-  DLOG_JAR="${RELEASE_JAR}"
-fi
-
-# exclude tests jar
-BUILT_JAR=$(ls ${DLOG_HOME}/target/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
-
-if [ -e "${BUILD_JAR}" ] && [ -e "${DLOG_JAR}" ]; then
-  echo "\nCouldn't find dlog jar.";
-  echo "Make sure you've run 'mvn package'\n";
-  exit 1;
-elif [ -e "${BUILT_JAR}" ]; then
-  DLOG_JAR="${BUILT_JAR}"
-fi
-
-add_maven_deps_to_classpath() {
-  MVN="mvn"
-  if [ -n "${MAVEN_HOME}" ]; then
-    MVN="${MAVEN_HOME}/bin/mvn"
-  fi
-
-  # Need to generate classpath from maven pom. This is costly so generate it
-  # and cache it. Save the file into our target dir so a mvn clean will get
-  # clean it up and force us create a new one.
-  f="${PWD}/${DLOG_HOME}/target/cached_classpath.txt"
-  if [ ! -f "${f}" ]; then
-    "${MVN}" -f "${DLOG_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
-  fi
-  DLOG_CLASSPATH="${CLASSPATH}":$(cat "${f}")
-}
-
-if [ -d "${DLOG_HOME}/lib" ]; then
-  for i in ${DLOG_HOME}/lib/*.jar; do
-    DLOG_CLASSPATH="${DLOG_CLASSPATH}:${i}"
-  done
-else
-  add_maven_deps_to_classpath
-fi
-
-# if no args specified, exit
-if [ $# = 0 ]; then
-  exit 1
-fi
-
-if [ -z "${DLOG_LOG_CONF}" ]; then
-  DLOG_LOG_CONF="${DEFAULT_LOG_CONF}"
-fi
-
-DLOG_CLASSPATH="${DLOG_JAR}:${DLOG_CLASSPATH}:${DLOG_EXTRA_CLASSPATH}"
-if [ -n "${DLOG_LOG_CONF}" ]; then
-  DLOG_CLASSPATH="$(dirname ${DLOG_LOG_CONF}):${DLOG_CLASSPATH}"
-  OPTS="${OPTS} -Dlog4j.configuration=$(basename ${DLOG_LOG_CONF})"
-fi
-OPTS="-cp ${DLOG_CLASSPATH} ${OPTS} ${DLOG_EXTRA_OPTS}"
-
-OPTS="${OPTS} ${DLOG_EXTRA_OPTS}"
-
-# Disable ipv6 as it can cause issues
-OPTS="${OPTS} -Djava.net.preferIPv4Stack=true"
-
-# log directory & file
-DLOG_ROOT_LOGGER=${DLOG_ROOT_LOGGER:-"INFO,R"}
-DLOG_LOG_DIR=${DLOG_LOG_DIR:-"$DLOG_HOME/logs"}
-DLOG_LOG_FILE=${DLOG_LOG_FILE:-"dlog.log"}
-
-#Configure log configuration system properties
-OPTS="$OPTS -Ddlog.root.logger=${DLOG_ROOT_LOGGER}"
-OPTS="$OPTS -Ddlog.log.dir=${DLOG_LOG_DIR}"
-OPTS="$OPTS -Ddlog.log.file=${DLOG_LOG_FILE}"
diff --git a/stream/distributedlog/core/bin/dlog b/stream/distributedlog/core/bin/dlog
deleted file mode 100755
index 01f8ee3..0000000
--- a/stream/distributedlog/core/bin/dlog
+++ /dev/null
@@ -1,73 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * 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.
-# */
-
-set -e
-
-BASEDIR=$(dirname "$0")
-
-DLOG_HOME="${BASEDIR}/.."
-
-usage() {
-  cat <<EOF
-Usage: runner <command>
-where command is one of:
-  local               Run distributedlog sandbox
-  example             Run distributedlog example
-  tool                Run distributedlog tool
-  proxy_tool          Run distributedlog proxy tool to interact with proxies
-  balancer            Run distributedlog balancer
-  admin               Run distributedlog admin tool
-  help                This help message
-
-or command is the full name of a class with a defined main() method.
-
-Environment variables:
-  DLOG_LOG_CONF        Log4j configuration file (default $DEFAULT_LOG_CONF)
-  DLOG_EXTRA_OPTS      Extra options to be passed to the jvm
-  DLOG_EXTRA_CLASSPATH Add extra paths to the dlog classpath
-
-These variable can also be set in conf/dlogenv.sh
-EOF
-}
-
-source "${DLOG_HOME}"/bin/common.sh
-
-# get arguments
-COMMAND=$1
-shift
-
-case "${COMMAND}" in
-  local)
-    exec java $OPTS $JMX_ARGS -Dzookeeper.4lw.commands.whitelist='*' org.apache.distributedlog.LocalDLMEmulator $@
-    ;;
-  tool)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.tools.DistributedLogTool $@
-    ;;
-  admin)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.admin.DistributedLogAdmin $@
-    ;;
-  help)
-    usage
-    ;;
-  *)
-    exec java $OPTS $COMMAND $@
-    ;;
-esac
-
diff --git a/stream/distributedlog/core/conf/bookie.conf.template b/stream/distributedlog/core/conf/bookie.conf.template
deleted file mode 100644
index 5ca89d0..0000000
--- a/stream/distributedlog/core/conf/bookie.conf.template
+++ /dev/null
@@ -1,183 +0,0 @@
-#/**
-# * 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.
-# */
-
-## Bookie settings
-
-# Port that bookie server listen on
-bookiePort=3181
-
-# TODO: change the journal directory
-# Directory Bookkeeper outputs its write ahead log
-journalDirectory=/tmp/data/bk/journal
-
-# TODO: change the ledgers directory
-# Directory Bookkeeper outputs ledger snapshots
-ledgerDirectories=/tmp/data/bk/ledgers
-
-# TODO: change the index directory
-# Directory in which index files will be stored.
-indexDirectories=/tmp/data/bk/ledgers
-
-# Ledger Manager Class
-# What kind of ledger manager is used to manage how ledgers are stored, managed
-# and garbage collected. Try to read 'BookKeeper Internals' for detail info.
-ledgerManagerType=hierarchical
-
-# Root zookeeper path to store ledger metadata
-# This parameter is used by zookeeper-based ledger manager as a root znode to
-# store all ledgers.
-zkLedgersRootPath=/messaging/bookkeeper/ledgers
-
-# Max file size of entry logger, in bytes
-# A new entry log file will be created when the old one reaches the file size limitation
-logSizeLimit=1073741823
-
-# Max file size of journal file, in mega bytes
-# A new journal file will be created when the old one reaches the file size limitation
-#
-journalMaxSizeMB=2048
-
-# Max number of old journal file to kept
-# Keep a number of old journal files would help data recovery in specia case
-#
-journalMaxBackups=5
-
-# How long the interval to trigger next garbage collection, in milliseconds
-# Since garbage collection is running in background, too frequent gc
-# will heart performance. It is better to give a higher number of gc
-# interval if there is enough disk capacity.
-# gc per 1 hour (aligning with most DL rolling interval)
-gcInitialWaitTime=600000
-gcWaitTime=3600000
-# do minor compaction per 2 hours
-minorCompactionInterval=7200
-minorCompactionThreshold=0.2
-# disable major compaction
-majorCompactionInterval=0
-# reduce major compaction threshold to a low value to prevent bad force compaction behavior
-majorCompactionThreshold=0.3
-# Compaction Rate & Max Outstanding
-compactionRate=10737418
-compactionMaxOutstandingRequests=10737418
-
-# How long the interval to flush ledger index pages to disk, in milliseconds
-# Flushing index files will introduce much random disk I/O.
-# If separating journal dir and ledger dirs each on different devices,
-# flushing would not affect performance. But if putting journal dir
-# and ledger dirs on same device, performance degrade significantly
-# on too frequent flushing. You can consider increment flush interval
-# to get better performance, but you need to pay more time on bookie
-# server restart after failure.
-#
-flushInterval=1000
-
-# Interval to watch whether bookie is dead or not, in milliseconds
-#
-# bookieDeathWatchInterval=1000
-
-## zookeeper client settings
-
-# A list of one of more servers on which zookeeper is running.
-# The server list can be comma separated values, for example:
-# zkServers=zk1:2181,zk2:2181,zk3:2181
-zkServers=localhost:2181
-
-# ZooKeeper client session timeout in milliseconds
-# Bookie server will exit if it received SESSION_EXPIRED because it
-# was partitioned off from ZooKeeper for more than the session timeout
-# JVM garbage collection, disk I/O will cause SESSION_EXPIRED.
-# Increment this value could help avoiding this issue
-zkTimeout=30000
-
-## NIO Server settings
-
-# This settings is used to enabled/disabled Nagle's algorithm, which is a means of
-# improving the efficiency of TCP/IP networks by reducing the number of packets
-# that need to be sent over the network.
-# If you are sending many small messages, such that more than one can fit in
-# a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm
-# can provide better performance.
-# Default value is true.
-#
-serverTcpNoDelay=true
-
-## ledger cache settings
-
-# Max number of ledger index files could be opened in bookie server
-# If number of ledger index files reaches this limitation, bookie
-# server started to swap some ledgers from memory to disk.
-# Too frequent swap will affect performance. You can tune this number
-# to gain performance according your requirements.
-openFileLimit=20000
-
-# Size of a index page in ledger cache, in bytes
-# A larger index page can improve performance writing page to disk,
-# which is efficent when you have small number of ledgers and these
-# ledgers have similar number of entries.
-# If you have large number of ledgers and each ledger has fewer entries,
-# smaller index page would improve memory usage.
-pageSize=8192
-
-# How many index pages provided in ledger cache
-# If number of index pages reaches this limitation, bookie server
-# starts to swap some ledgers from memory to disk. You can increment
-# this value when you found swap became more frequent. But make sure
-# pageLimit*pageSize should not more than JVM max memory limitation,
-# otherwise you would got OutOfMemoryException.
-# In general, incrementing pageLimit, using smaller index page would
-# gain bettern performance in lager number of ledgers with fewer entries case
-# If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute
-# the limitation of number of index pages.
-pageLimit=131072
-
-#If all ledger directories configured are full, then support only read requests for clients.
-#If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted
-#to read-only mode and serve only read requests. Otherwise the bookie will be shutdown.
-readOnlyModeEnabled=true
-
-# Bookie Journal Settings
-writeBufferSizeBytes=262144
-journalFlushWhenQueueEmpty=false
-journalRemoveFromPageCache=true
-journalAdaptiveGroupWrites=true
-journalMaxGroupWaitMSec=4
-journalBufferedEntriesThreshold=180
-journalBufferedWritesThreshold=131072
-journalMaxGroupedEntriesToCommit=200
-journalPreAllocSizeMB=4
-
-# Sorted Ledger Storage Settings
-sortedLedgerStorageEnabled=true
-skipListSizeLimit=67108864
-skipListArenaChunkSize=2097152
-skipListArenaMaxAllocSize=131072
-fileInfoCacheInitialCapacity=10000
-fileInfoMaxIdleTime=3600
-
-# Bookie Threads Settings (NOTE: change this to align the cpu cores)
-numAddWorkerThreads=4
-numJournalCallbackThreads=4
-numReadWorkerThreads=4
-numLongPollWorkerThreads=4
-
-# stats
-statsProviderClass=org.apache.bookkeeper.stats.CodahaleMetricsServletProvider
-# Exporting codahale stats
-codahaleStatsHttpPort=9001
-useHostNameAsBookieID=true
-allowLoopback=true
diff --git a/stream/distributedlog/core/conf/distributedlog.conf b/stream/distributedlog/core/conf/distributedlog.conf
deleted file mode 100644
index dac71ac..0000000
--- a/stream/distributedlog/core/conf/distributedlog.conf
+++ /dev/null
@@ -1,125 +0,0 @@
-#/**
-# * 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.
-# */
-
-########################
-# ZooKeeper Client Settings
-########################
-
-# zookeeper settings
-zkSessionTimeoutSeconds=30
-zkNumRetries=0
-zkRetryStartBackoffMillis=100
-zkRetryMaxBackoffMillis=200
-# bkc zookeeper settings
-bkcZKSessionTimeoutSeconds=60
-bkcZKNumRetries=20
-bkcZKRetryStartBackoffMillis=100
-bkcZKRetryMaxBackoffMillis=200
-
-########################
-# BookKeeper Client Settings
-########################
-
-# bookkeeper client timeouts
-bkcWriteTimeoutSeconds=10
-bkcReadTimeoutSeconds=1
-bkcNumWorkerThreads=16
-# bkcNumIOThreads=16
-bkc.numChannelsPerBookie=1
-bkc.enableTaskExecutionStats=true
-bkc.connectTimeoutMillis=1000
-bkc.enablePerHostStats=true
-
-########################
-# DL Settings
-########################
-
-# lock timeout
-lockTimeoutSeconds=0
-# dl worker threads
-numWorkerThreads=16
-
-### Recovery Related Settings
-
-# recover log segments in background
-recoverLogSegmentsInBackground=true
-# disable max id in proxy
-maxIdSanityCheck=true
-# use allocator pool for proxy
-enableLedgerAllocatorPool=false
-# ledger allocator pool size
-ledgerAllocatorPoolCoreSize=20
-# check stream exists or not
-createStreamIfNotExists=true
-# encode dc id in version
-encodeDCIDInVersion=true
-# logSegmentNameVersion
-logSegmentNameVersion=1
-
-### Write Performance Related Settings
-
-# ensemble size
-ensemble-size=3
-write-quorum-size=3
-ack-quorum-size=2
-bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
-bkc.delayEnsembleChange=true
-
-# sync settings
-# buffer size is large because when we rewrite we perform a very large write to persist
-# all queue state at once (up to max queue memory size, ex. 16MB). the write will be
-# throttled if it takes too long, which can hurt performance, so important to optimize
-# for this case.
-output-buffer-size=512000
-enableImmediateFlush=false
-periodicFlushFrequencyMilliSeconds=6
-logFlushTimeoutSeconds=120
-
-### Ledger Rolling Related Settings
-
-# retention policy
-retention-size=0
-# rolling ledgers (disable time rolling/enable size rolling)
-rolling-interval=0
-
-# max logsegment bytes=2GB
-# much larger than max journal size, effectively never roll and let drpc do it
-maxLogSegmentBytes=2147483648
-
-# rolling concurrency
-logSegmentRollingConcurrency=1
-# disable sanityCheckDelete
-sanityCheckDelete=false
-ledgerAllocatorPoolName=drpc-alloc-pool
-
-### Readahead settings
-
-enableReadAhead=true
-ReadAheadBatchSize=10
-ReadAheadMaxEntries=100
-ReadAheadWaitTime=10
-
-### Rate limit
-
-rpsSoftWriteLimit=1
-rpsHardWriteLimit=5
-rpsHardServiceLimit=15
-
-### Config
-
-dynamicConfigReloadIntervalSec=5
diff --git a/stream/distributedlog/core/conf/dlogenv.sh b/stream/distributedlog/core/conf/dlogenv.sh
deleted file mode 100644
index 345e60f..0000000
--- a/stream/distributedlog/core/conf/dlogenv.sh
+++ /dev/null
@@ -1,75 +0,0 @@
-#!/bin/sh
-#
-#/**
-# * 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.
-# */
-
-##################
-# General
-##################
-
-# Log4j configuration file
-# DLOG_LOG_CONF=
-
-# Extra options to be passed to the jvm
-# DLOG_EXTRA_OPTS=
-
-# Add extra paths to the dlog classpath
-# DLOG_EXTRA_CLASSPATH=
-
-# Configure the root logger
-# DLOG_ROOT_LOGGER=
-
-# Configure the log dir
-# DLOG_LOG_DIR=
-
-# Configure the log file
-# DLOG_LOG_FILE=
-
-#################
-# ZooKeeper
-#################
-
-# Configure zookeeper root logger
-# ZK_ROOT_LOGGER=
-
-#################
-# Bookie
-#################
-
-# Configure bookie root logger
-# BK_ROOT_LOGGER=
-
-#################
-# Write Proxy
-#################
-
-# Configure write proxy root logger
-# WP_ROOT_LOGGER=
-
-# write proxy configuration file
-# WP_CONF_FILE=${DL_HOME}/conf/write_proxy.conf
-
-# port and stats port
-# WP_SERVICE_PORT=4181
-# WP_STATS_PORT=9000
-
-# shard id
-# WP_SHARD_ID=0
-
-# write proxy namespace
-# WP_NAMESPACE=distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace
diff --git a/stream/distributedlog/core/conf/log4j.properties b/stream/distributedlog/core/conf/log4j.properties
deleted file mode 100644
index af1cf5f..0000000
--- a/stream/distributedlog/core/conf/log4j.properties
+++ /dev/null
@@ -1,56 +0,0 @@
-#/**
-# * 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.
-# */
-
-#
-# DistributedLog Logging Configuration
-#
-
-# Default values
-dlog.root.logger=INFO, R
-dlog.log.dir=logs
-dlog.log.file=dlog.log
-
-log4j.rootLogger=${dlog.root.logger}
-log4j.logger.org.apache.zookeeper=INFO
-log4j.logger.org.apache.bookkeeper=INFO
-
-# redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.org.apache.bookkeeper.util.SafeRunnable=INFO, Executors
-log4j.additivity.org.apache.bookkeeper.util.SafeRunnable=false
-
-log4j.appender.Executors=org.apache.log4j.RollingFileAppender
-log4j.appender.Executors.Threshold=INFO
-log4j.appender.Executors.File=${dlog.log.dir}/executors.log
-log4j.appender.Executors.MaxFileSize=20MB
-log4j.appender.Executors.MaxBackupIndex=5
-log4j.appender.Executors.layout=org.apache.log4j.PatternLayout
-log4j.appender.Executors.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.R=org.apache.log4j.RollingFileAppender
-log4j.appender.R.Threshold=INFO
-log4j.appender.R.File=${dlog.log.dir}/${dlog.log.file}
-log4j.appender.R.MaxFileSize=20MB
-log4j.appender.R.MaxBackupIndex=50
-log4j.appender.R.layout=org.apache.log4j.PatternLayout
-log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.stderr=org.apache.log4j.ConsoleAppender
-log4j.appender.stderr.Target=System.err
-log4j.appender.stderr.Threshold=INFO
-log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
-log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
diff --git a/stream/distributedlog/core/conf/write_proxy.conf b/stream/distributedlog/core/conf/write_proxy.conf
deleted file mode 100644
index 7f5351a..0000000
--- a/stream/distributedlog/core/conf/write_proxy.conf
+++ /dev/null
@@ -1,143 +0,0 @@
-#/**
-# * 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.
-# */
-
-########################
-# ZooKeeper Client Settings
-########################
-
-# zookeeper settings
-zkSessionTimeoutSeconds=1
-zkNumRetries=0
-zkRetryStartBackoffMillis=100
-zkRetryMaxBackoffMillis=200
-# bkc zookeeper settings
-bkcZKSessionTimeoutSeconds=60
-bkcZKNumRetries=20
-bkcZKRetryStartBackoffMillis=100
-bkcZKRetryMaxBackoffMillis=200
-
-########################
-# BookKeeper Client Settings
-########################
-
-# bookkeeper client timeouts
-bkcWriteTimeoutSeconds=2
-bkcReadTimeoutSeconds=2
-bkcNumWorkerThreads=32
-bkc.numChannelsPerBookie=1
-bkc.enableTaskExecutionStats=true
-bkc.connectTimeoutMillis=200
-bkc.enableParallelRecoveryRead=true
-bkc.recoveryReadBatchSize=5
-bkc.enablePerHostStats=true
-
-########################
-# DL Settings
-########################
-
-# Metadata Settings
-
-# ledger metadata version that supports sequence id
-ledger-metadata-layout=5
-
-# lock timeout
-lockTimeoutSeconds=0
-# dl worker threads
-numWorkerThreads=32
-
-### Recovery Related Settings
-
-# recover log segments in background
-recoverLogSegmentsInBackground=false
-# disable max id in proxy
-maxIdSanityCheck=false
-# use allocator pool for proxy
-enableLedgerAllocatorPool=true
-# ledger allocator pool path
-ledgerAllocatorPoolPath=.write_proxy_allocation_pool
-# ledger allocator pool size
-ledgerAllocatorPoolCoreSize=40
-# check stream exists or not
-createStreamIfNotExists=true
-# encode dc id in version
-encodeDCIDInVersion=true
-# logSegmentNameVersion
-logSegmentNameVersion=1
-
-### Write Performance Related Settings
-
-# ensemble size
-ensemble-size=3
-write-quorum-size=3
-ack-quorum-size=2
-bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
-bkc.delayEnsembleChange=true
-bkc.writeRequestToChannelAsync=true
-
-# enable immediate flush
-enableImmediateFlush=true
-# 0k output buffer
-output-buffer-size=0
-# disable periodical flush
-periodicFlushFrequencyMilliSeconds=0
-enableTaskExecutionStats=true
-taskExecutionWarnTimeMicros=100000
-
-### Ledger Rolling Related Settings
-
-# retention policy
-retention-size=4
-# rolling ledgers (enable time rolling): 120 minutes = 2 hours
-rolling-interval=120
-# max logsegment bytes : 2GB
-maxLogSegmentBytes=2147483648
-# rolling concurrency
-logSegmentRollingConcurrency=1
-# disable sanityCheckDelete
-sanityCheckDelete=false
-# compression codec
-compressionType=lz4
-
-### Per Stream Stats
-enablePerStreamStat=true
-
-########################
-# DL Settings
-########################
-
-# proxy server settings
-server_mode=DURABLE
-serviceTimeoutMs=60000
-streamProbationTimeoutMs=120000
-server_threads=16
-server_dlsn_version=1
-server_enable_perstream_stat=true
-server_graceful_shutdown_period_ms=20000
-
-# write limits
-perWriterOutstandingWriteLimit=-1
-globalOutstandingWriteLimit=-1
-outstandingWriteLimitDarkmode=false
-
-# bytes per second limit applied at the host level (50MBps on 1Gib machines)
-bpsHardServiceLimit=52428800
-# bytes per second limit after which no new streams may be acquired (65MBps on 1Gib machines)
-bpsStreamAcquireServiceLimit=47185920
-
-# limit the maximum number of streams
-maxAcquiredPartitionsPerProxy=-1
diff --git a/stream/distributedlog/core/conf/zookeeper.conf.dynamic.template b/stream/distributedlog/core/conf/zookeeper.conf.dynamic.template
deleted file mode 100644
index b397c50..0000000
--- a/stream/distributedlog/core/conf/zookeeper.conf.dynamic.template
+++ /dev/null
@@ -1 +0,0 @@
-#/**
# * Copyright 2017 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.
# */
server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
diff --git a/stream/distributedlog/core/conf/zookeeper.conf.template b/stream/distributedlog/core/conf/zookeeper.conf.template
deleted file mode 100644
index 3c0546e..0000000
--- a/stream/distributedlog/core/conf/zookeeper.conf.template
+++ /dev/null
@@ -1,82 +0,0 @@
-# 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 number of milliseconds of each tick
-tickTime=2000
-
-# the port at which the clients will connect
-clientPort=2181
-
-# The number of ticks that the initial
-# synchronization phase can take
-initLimit=10
-
-# The number of ticks that can pass between
-# sending a request and getting an acknowledgement
-syncLimit=30
-
-# the directory where the snapshot is stored.
-dataDir=/tmp/data/zookeeper
-
-# where txlog  are written
-dataLogDir=/tmp/data/zookeeper/txlog
-
-# the port at which the admin will listen
-adminPort=9990
-zookeeper.admin.enableServer=true
-
-# limit on queued clients - default: 1000
-globalOutstandingLimit=1000
-
-# number of transactions before snapshots are taken - default: 100000
-snapCount=100000
-
-# max # of clients - 0==unlimited
-maxClientCnxns=25
-
-# Election implementation to use. A value of "0" corresponds to the original
-# UDP-based version, "1" corresponds to the non-authenticated UDP-based
-# version of fast leader election, "2" corresponds to the authenticated
-# UDP-based version of fast leader election, and "3" corresponds to TCP-based
-# version of fast leader election. Currently, only 0 and 3 are supported,
-# 3 being the default
-electionAlg=3
-
-# Leader accepts client connections. Default value is "yes". The leader
-# machine coordinates updates. For higher update throughput at thes slight
-# expense of read throughput the leader can be configured to not accept
-# clients and focus on coordination.
-leaderServes=yes
-
-# Skips ACL checks. This results in a boost in throughput, but opens up full
-# access to the data tree to everyone.
-skipACL=no
-
-# Purge txn logs every hour. Before 3.4.x this was done with an external cron
-# job, now we can do it internally.
-autopurge.purgeInterval=1
-
-# Prior to version 3.4 ZooKeeper has always used NIO directly, however in
-# versions 3.4 and later Netty is supported as an option to NIO (replaces).
-# serverCnxnFactory=org.apache.zookeeper.server.NIOServerCnxnFactory
-
-standaloneEnabled=false
-# ZooKeeper Dynamic Reconfiguration
-# See: https://zookeeper.apache.org/doc/trunk/zookeeperReconfig.html
-#
-# standaloneEnabled=false
-# dynamicConfigFile=/path/to/zoo.cfg.dynamic
-#
-server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
diff --git a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
index a7f9f8c..3cd370f 100644
--- a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
+++ b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/BookKeeperClusterUtils.java
@@ -92,6 +92,26 @@ public class BookKeeperClusterUtils {
         }
     }
 
+    public static String createDlogNamespaceIfNeeded(DockerClient docker,
+                                                     String version,
+                                                     String namespace) throws Exception {
+        String zkServers = BookKeeperClusterUtils.zookeeperConnectString(docker);
+        String dlogUri = "distributedlog://" + zkServers + namespace;
+        try (ZooKeeper zk = BookKeeperClusterUtils.zookeeperClient(docker)) {
+            if (zk.exists(namespace, false) == null) {
+                String dlog = "/opt/bookkeeper/" + version + "/bin/dlog";
+
+                runOnAnyBookie(docker, dlog,
+                    "admin",
+                    "bind",
+                    "-l", "/ledgers",
+                    "-s", zkServers,
+                    "-c", dlogUri);
+            }
+        }
+        return dlogUri;
+    }
+
     public static void formatAllBookies(DockerClient docker, String version) throws Exception {
         String bookkeeper = "/opt/bookkeeper/" + version + "/bin/bookkeeper";
         BookKeeperClusterUtils.runOnAllBookies(docker, bookkeeper, "shell", "bookieformat", "-nonInteractive");
@@ -123,6 +143,15 @@ public class BookKeeperClusterUtils {
         }
     }
 
+    public static String getAnyBookie() throws Exception {
+        Optional<String> bookie = DockerUtils.cubeIdsMatching("bookkeeper").stream().findAny();
+        if (bookie.isPresent()) {
+            return bookie.get();
+        } else {
+            throw new Exception("No bookie is available");
+        }
+    }
+
     public static void runOnAllBookies(DockerClient docker, String... cmds) throws Exception {
         for (String b : DockerUtils.cubeIdsMatching("bookkeeper")) {
             DockerUtils.runCommand(docker, b, cmds);
diff --git a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/DockerUtils.java b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/DockerUtils.java
index 11959af..53d6228 100644
--- a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/DockerUtils.java
+++ b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/DockerUtils.java
@@ -18,6 +18,8 @@
  */
 package org.apache.bookkeeper.tests;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import com.github.dockerjava.api.DockerClient;
 import com.github.dockerjava.api.async.ResultCallback;
 import com.github.dockerjava.api.command.InspectExecResponse;
@@ -131,10 +133,21 @@ public class DockerUtils {
         throw new IllegalArgumentException("Container " + containerId + " has no networks");
     }
 
-    public static void runCommand(DockerClient docker, String containerId, String... cmd) throws Exception {
+    public static String runCommand(DockerClient docker, String containerId, String... cmd) throws Exception {
+        return runCommand(docker, containerId, false, cmd);
+    }
+
+    public static String runCommand(DockerClient docker, String containerId, boolean ignoreError, String... cmd)
+            throws Exception {
         CompletableFuture<Boolean> future = new CompletableFuture<>();
-        String execid = docker.execCreateCmd(containerId).withCmd(cmd).exec().getId();
+        String execid = docker.execCreateCmd(containerId)
+            .withCmd(cmd)
+            .withAttachStderr(true)
+            .withAttachStdout(true)
+            .exec()
+            .getId();
         String cmdString = Arrays.stream(cmd).collect(Collectors.joining(" "));
+        StringBuffer output = new StringBuffer();
         docker.execStartCmd(execid).withDetach(false).exec(new ResultCallback<Frame>() {
                 @Override
                 public void close() {}
@@ -147,6 +160,7 @@ public class DockerUtils {
                 @Override
                 public void onNext(Frame object) {
                     LOG.info("DOCKER.exec({}:{}): {}", containerId, cmdString, object);
+                    output.append(new String(object.getPayload(), UTF_8));
                 }
 
                 @Override
@@ -169,10 +183,15 @@ public class DockerUtils {
         }
         int retCode = resp.getExitCode();
         if (retCode != 0) {
-            throw new Exception(
-                    String.format("cmd(%s) failed on %s with exitcode %d",
-                                  cmdString, containerId, retCode));
+            LOG.error("DOCKER.exec({}:{}): failed with {} : {}", containerId, cmdString, retCode, output);
+            if (!ignoreError) {
+                throw new Exception(String.format("cmd(%s) failed on %s with exitcode %d",
+                    cmdString, containerId, retCode));
+            }
+        } else {
+            LOG.info("DOCKER.exec({}:{}): completed with {}", containerId, cmdString, retCode);
         }
+        return output.toString();
     }
 
     public static Set<String> cubeIdsMatching(String needle) {
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java
new file mode 100644
index 0000000..f1a58ed
--- /dev/null
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java
@@ -0,0 +1,79 @@
+/*
+ * 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.tests.integration;
+
+import static org.junit.Assert.assertTrue;
+
+import lombok.extern.slf4j.Slf4j;
+import org.junit.Before;
+import org.junit.Test;
+
+@Slf4j
+public abstract class BookieShellTestBase {
+
+    private String currentVersion = System.getProperty("currentVersion");
+    private String bkScript;
+
+    @Before
+    public void setup() {
+        bkScript = "/opt/bookkeeper/" + currentVersion + "/bin/bookkeeper";
+    }
+
+    @Test
+    public abstract void test000_Setup() throws Exception;
+
+    @Test
+    public abstract void test999_Teardown();
+
+    protected abstract String runCommandInAnyContainer(String... cmd) throws Exception;
+
+    @Test
+    public void test001_SimpleTest() throws Exception {
+        assertTrue(runCommandInAnyContainer(
+            bkScript,
+            "shell",
+            "simpletest",
+            "-ensemble", "3",
+            "-writeQuorum", "3",
+            "-ackQuorum", "2",
+            "-numEntries", "100"
+        ).contains("100 entries written to ledger"));
+    }
+
+    @Test
+    public void test002_ListROBookies() throws Exception {
+        assertTrue(runCommandInAnyContainer(
+            bkScript,
+            "shell",
+            "listbookies",
+            "-ro"
+        ).contains("No bookie exists!"));
+    }
+
+    @Test
+    public void test003_ListRWBookies() throws Exception {
+        assertTrue(runCommandInAnyContainer(
+            bkScript,
+            "shell",
+            "listbookies",
+            "-rw"
+        ).contains("ReadWrite Bookies :"));
+    }
+
+}
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java
new file mode 100644
index 0000000..04b950a
--- /dev/null
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java
@@ -0,0 +1,83 @@
+/*
+ * 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.tests.integration;
+
+import static org.junit.Assert.assertTrue;
+
+import com.github.dockerjava.api.DockerClient;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.BookKeeperClusterUtils;
+import org.apache.bookkeeper.tests.DockerUtils;
+import org.jboss.arquillian.junit.Arquillian;
+import org.jboss.arquillian.test.api.ArquillianResource;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+
+@Slf4j
+@RunWith(Arquillian.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class TestBookieShellCluster extends BookieShellTestBase {
+
+    @ArquillianResource
+    private DockerClient docker;
+
+    private String currentVersion = System.getProperty("currentVersion");
+
+    @Test
+    @Override
+    public void test000_Setup() throws Exception {
+        // First test to run, formats metadata and bookies
+        if (BookKeeperClusterUtils.metadataFormatIfNeeded(docker, currentVersion)) {
+            BookKeeperClusterUtils.formatAllBookies(docker, currentVersion);
+        }
+        assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion));
+    }
+
+    @Test
+    @Override
+    public void test999_Teardown() {
+        assertTrue(BookKeeperClusterUtils.stopAllBookies(docker));
+    }
+
+    @Override
+    protected String runCommandInAnyContainer(String... cmds) throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        return DockerUtils.runCommand(docker, bookie, cmds);
+    }
+
+    @Test
+    @Override
+    public void test001_SimpleTest() throws Exception {
+        super.test001_SimpleTest();
+    }
+
+    @Test
+    @Override
+    public void test002_ListROBookies() throws Exception {
+        super.test002_ListROBookies();
+    }
+
+    @Test
+    @Override
+    public void test003_ListRWBookies() throws Exception {
+        super.test003_ListRWBookies();
+    }
+}
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
new file mode 100644
index 0000000..7761808
--- /dev/null
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
@@ -0,0 +1,102 @@
+/*
+ * 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.tests.integration;
+
+import static org.junit.Assert.assertTrue;
+
+import com.github.dockerjava.api.DockerClient;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.BookKeeperClusterUtils;
+import org.apache.bookkeeper.tests.DockerUtils;
+import org.jboss.arquillian.junit.Arquillian;
+import org.jboss.arquillian.test.api.ArquillianResource;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+
+@Slf4j
+@RunWith(Arquillian.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class TestCLI {
+
+    @ArquillianResource
+    private DockerClient docker;
+
+    private String currentVersion = System.getProperty("currentVersion");
+    private String bkCLI;
+
+    @Before
+    public void setup() {
+        bkCLI = "/opt/bookkeeper/" + currentVersion + "/bin/bookkeeper-cli";
+    }
+
+    @Test
+    public void test000_Setup() throws Exception {
+        // First test to run, formats metadata and bookies
+        if (BookKeeperClusterUtils.metadataFormatIfNeeded(docker, currentVersion)) {
+            BookKeeperClusterUtils.formatAllBookies(docker, currentVersion);
+        }
+        assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion));
+
+    }
+
+    @Test
+    public void test999_Teardown() throws Exception {
+        assertTrue(BookKeeperClusterUtils.stopAllBookies(docker));
+    }
+
+    @Test
+    public void test001_SimpleTest() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        assertTrue(DockerUtils.runCommand(docker, bookie,
+            bkCLI,
+            "client",
+            "simpletest",
+            "--ensemble-size", "3",
+            "--write-quorum-size", "3",
+            "--ack-quorum-size", "2",
+            "--num-entries", "100"
+        ).contains("100 entries written to ledger"));
+    }
+
+    @Test
+    public void test002_ListROBookies() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        assertTrue(DockerUtils.runCommand(docker, bookie,
+            bkCLI,
+            "cluster",
+            "listbookies",
+            "-ro"
+        ).contains("No bookie exists!"));
+    }
+
+    @Test
+    public void test003_ListRWBookies() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        assertTrue(DockerUtils.runCommand(docker, bookie,
+            bkCLI,
+            "cluster",
+            "listbookies",
+            "-rw"
+        ).contains("ReadWrite Bookies :"));
+    }
+
+}
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestDlogCLI.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestDlogCLI.java
new file mode 100644
index 0000000..83ecde6
--- /dev/null
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestDlogCLI.java
@@ -0,0 +1,146 @@
+/*
+ * 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.tests.integration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.github.dockerjava.api.DockerClient;
+import java.util.HashSet;
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tests.BookKeeperClusterUtils;
+import org.apache.bookkeeper.tests.DockerUtils;
+import org.jboss.arquillian.junit.Arquillian;
+import org.jboss.arquillian.test.api.ArquillianResource;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+
+@Slf4j
+@RunWith(Arquillian.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class TestDlogCLI {
+
+    private static final String DLOG_STREAM_PREFIX = "stream-";
+    private static final String STREAMS_REGEX = "0-99";
+
+    @ArquillianResource
+    private DockerClient docker;
+
+    private String currentVersion = System.getProperty("currentVersion");
+    private String dlogCLI;
+    private String dlogUri;
+
+    @Before
+    public void setup() {
+        dlogCLI = "/opt/bookkeeper/" + currentVersion + "/bin/dlog";
+        dlogUri = "distributedlog://" + BookKeeperClusterUtils.zookeeperConnectString(docker) + "/distributedlog";
+    }
+
+    @Test
+    public void test000_Setup() throws Exception {
+        // First test to run, formats metadata and bookies, then create a dlog namespace
+        if (BookKeeperClusterUtils.metadataFormatIfNeeded(docker, currentVersion)) {
+            BookKeeperClusterUtils.formatAllBookies(docker, currentVersion);
+        }
+        BookKeeperClusterUtils.createDlogNamespaceIfNeeded(docker, currentVersion, "/distributedlog");
+    }
+
+    @Test
+    public void test999_Teardown() throws Exception {
+        assertTrue(BookKeeperClusterUtils.stopAllBookies(docker));
+    }
+
+    @Test
+    public void test001_CreateStreams() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        assertTrue(DockerUtils.runCommand(docker, bookie,
+            dlogCLI,
+            "tool",
+            "create",
+            "--prefix", DLOG_STREAM_PREFIX,
+            "--expression", STREAMS_REGEX,
+            "--uri", dlogUri,
+            "-f"
+            ).isEmpty());
+    }
+
+    @Test
+    public void test002_ListStreams() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        String output = DockerUtils.runCommand(docker, bookie,
+            dlogCLI,
+            "tool",
+            "list",
+            "--uri", dlogUri,
+            "-f"
+            );
+        String[] lines = output.split("\\r?\\n");
+        Set<String> streams = new HashSet<>();
+        for (String string : lines) {
+            if (string.startsWith(DLOG_STREAM_PREFIX)) {
+                streams.add(string);
+            }
+        }
+        assertEquals(100, streams.size());
+    }
+
+    @Test
+    public void test003_ShowStream() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        String output = DockerUtils.runCommand(docker, bookie, true,
+            dlogCLI,
+            "tool",
+            "show",
+            "--uri", dlogUri,
+            "--stream", "stream-99",
+            "-f");
+        assertTrue(output.contains("Log stream-99:<default> has no records"));
+    }
+
+    @Test
+    public void test004_DeleteStream() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        String output = DockerUtils.runCommand(docker, bookie,
+            dlogCLI,
+            "tool",
+            "delete",
+            "--uri", dlogUri,
+            "--stream", "stream-99",
+            "-f");
+        assertTrue(output.isEmpty());
+    }
+
+    @Test
+    public void test005_CheckStreamDeleted() throws Exception {
+        String bookie = BookKeeperClusterUtils.getAnyBookie();
+        String output = DockerUtils.runCommand(docker, bookie, true,
+            dlogCLI,
+            "tool",
+            "show",
+            "--uri", dlogUri,
+            "--stream", "stream-99",
+            "-f");
+        assertTrue(output.contains("Log stream-99 does not exist or has been deleted"));
+    }
+
+}
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java
index f2eff85..8ecf28b 100644
--- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java
@@ -44,14 +44,15 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.tests.BookKeeperClusterUtils;
 import org.jboss.arquillian.junit.Arquillian;
 import org.jboss.arquillian.test.api.ArquillianResource;
-import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
+import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
 
 @Slf4j
 @RunWith(Arquillian.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
 public class TestSmoke {
     private static byte[] PASSWD = "foobar".getBytes();
 
@@ -60,8 +61,8 @@ public class TestSmoke {
 
     private String currentVersion = System.getProperty("currentVersion");
 
-    @Before
-    public void setup() throws Exception {
+    @Test
+    public void test000_Setup() throws Exception {
         // First test to run, formats metadata and bookies
         if (BookKeeperClusterUtils.metadataFormatIfNeeded(docker, currentVersion)) {
             BookKeeperClusterUtils.formatAllBookies(docker, currentVersion);
@@ -69,13 +70,13 @@ public class TestSmoke {
         Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion));
     }
 
-    @After
-    public void teardown() throws Exception {
+    @Test
+    public void tear999_Teardown() {
         Assert.assertTrue(BookKeeperClusterUtils.stopAllBookies(docker));
     }
 
     @Test
-    public void testReadWrite() throws Exception {
+    public void test001_ReadWrite() throws Exception {
         String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker);
         int numEntries = 100;
         try (BookKeeper bk = new BookKeeper(zookeeper)) {
@@ -92,7 +93,7 @@ public class TestSmoke {
     }
 
     @Test
-    public void testReadWriteAdv() throws Exception {
+    public void test002_ReadWriteAdv() throws Exception {
         String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker);
         int numEntries = 100;
         try (BookKeeper bk = new BookKeeper(zookeeper)) {
@@ -125,12 +126,12 @@ public class TestSmoke {
     }
 
     @Test
-    public void testTailingReadsWithoutExplicitLac() throws Exception {
+    public void test003_TailingReadsWithoutExplicitLac() throws Exception {
         testTailingReads(100, 98, 0);
     }
 
     @Test
-    public void testTailingReadsWithExplicitLac() throws Exception {
+    public void test004_TailingReadsWithExplicitLac() throws Exception {
         testTailingReads(100, 99, 100);
     }
 
@@ -229,12 +230,12 @@ public class TestSmoke {
     }
 
     @Test
-    public void testLongTailingReadsWithoutExplicitLac() throws Exception {
+    public void test005_LongTailingReadsWithoutExplicitLac() throws Exception {
         testLongPollTailingReads(100, 98, 0);
     }
 
     @Test
-    public void testLongTailingReadsWithExplicitLac() throws Exception {
+    public void test006_LongTailingReadsWithExplicitLac() throws Exception {
         testLongPollTailingReads(100, 99, 100);
     }
 
diff --git a/tests/pom.xml b/tests/pom.xml
index c1dc864..8833ca3 100644
--- a/tests/pom.xml
+++ b/tests/pom.xml
@@ -40,6 +40,7 @@
     <module>integration-tests-topologies</module>
     <module>backward-compat</module>
     <module>integration</module>
+    <module>scripts</module>
   </modules>
   <build>
     <plugins>
diff --git a/tests/scripts/pom.xml b/tests/scripts/pom.xml
new file mode 100644
index 0000000..e100ea8
--- /dev/null
+++ b/tests/scripts/pom.xml
@@ -0,0 +1,75 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation=" http://maven.apache.org/POM/4.0.0   http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper.tests</groupId>
+    <artifactId>tests-parent</artifactId>
+    <version>4.8.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.bookkeeper.tests</groupId>
+  <artifactId>scripts</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Bash Scripts Test</name>
+
+  <dependencies>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>com.googlecode.maven-download-plugin</groupId>
+        <artifactId>download-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>install-shunit2</id>
+            <phase>integration-test</phase>
+            <goals>
+              <goal>wget</goal>
+            </goals>
+            <configuration>
+              <url>https://github.com/kward/shunit2/archive/v2.1.7.zip</url>
+              <unpack>true</unpack>
+              <outputDirectory>${project.basedir}/target/lib</outputDirectory>
+              <skip>${skipTests}</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>bash-tests</id>
+            <phase>integration-test</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <skip>${skipTests}</skip>
+              <workingDirectory>${project.basedir}/src/test/bash</workingDirectory>
+              <executable>${project.basedir}/src/test/bash/bk_test.sh</executable>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/tests/scripts/src/test/bash/bk_test.sh b/tests/scripts/src/test/bash/bk_test.sh
new file mode 100755
index 0000000..3c7b02e
--- /dev/null
+++ b/tests/scripts/src/test/bash/bk_test.sh
@@ -0,0 +1,141 @@
+#!/usr/bin/env bash
+#
+# vim:et:ft=sh:sts=2:sw=2
+#
+#/**
+# * 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.
+# */
+
+ARGV0=`basename "$0"`
+PREFIX="bk_test_"
+SHELLS="/bin/sh /bin/bash"
+
+find_tests_at() {
+  DIR=$1
+  PREF=$2
+  REGEX="^${PREF}[a-z_]*.sh$"
+  RESULTS=""
+  if [ -d ${DIR} ]; then
+    cd ${DIR}
+    for f in *.sh; do
+      if [[ ${f} =~ ${REGEX} ]]; then
+        RESULTS="${RESULTS} ${f}"
+      fi
+    done
+  fi
+  echo ${RESULTS}
+}
+
+TESTS=$(find_tests_at "." ${PREFIX})
+
+# load common unit test functions
+source ./versions
+source ./bk_test_helpers
+
+usage() {
+  echo "usage: ${ARGV0} [-e key=val ...] [-s shell(s)] [-t test(s)]"
+}
+
+env=''
+
+# process command line flags
+while getopts 'e:hs:t:' opt; do
+  case ${opt} in
+    e)  # set an environment variable
+      key=`expr "${OPTARG}" : '\([^=]*\)='`
+      val=`expr "${OPTARG}" : '[^=]*=\(.*\)'`
+      if [ -z "${key}" -o -z "${val}" ]; then
+        usage
+        exit 1
+      fi
+      eval "${key}='${val}'"
+      export ${key}
+      env="${env:+${env} }${key}"
+      ;;
+    h) usage; exit 0 ;;  # output help
+    s) shells=${OPTARG} ;;  # list of shells to run
+    t) tests=${OPTARG} ;;  # list of tests to run
+    *) usage; exit 1 ;;
+  esac
+done
+shift `expr ${OPTIND} - 1`
+
+# fill shells and/or tests
+shells=${shells:-${SHELLS}}
+tests=${tests:-${TESTS}}
+
+# error checking
+if [ -z "${tests}" ]; then
+  bk_info 'no tests found to run; exiting'
+  exit 0
+fi
+
+# print run info
+cat <<EOF
+#------------------------------------------------------------------------------
+# System data
+#
+# test run info
+shells="${shells}"
+tests="${tests}"
+EOF
+for key in ${env}; do
+  eval "echo \"${key}=\$${key}\""
+done
+echo
+
+# output system data
+echo "# system info"
+echo "$ date"
+date
+
+echo "$ uname -mprsv"
+uname -mprsv
+
+#
+# run tests
+#
+
+for shell in ${shells}; do
+  echo
+
+  # check for existance of shell
+  if [ ! -x ${shell} ]; then
+    bk_warn "unable to run tests with the ${shell} shell"
+    continue
+  fi
+
+  cat <<EOF
+#------------------------------------------------------------------------------
+# Running the test suite with ${shell}
+#
+EOF
+
+  shell_name=`basename ${shell}`
+  shell_version=`versions_shellVersion "${shell}"`
+
+  echo "shell name: ${shell_name}"
+  echo "shell version: ${shell_version}"
+
+  # execute the tests
+  for suite in ${tests}; do
+    suiteName=`expr "${suite}" : "${PREFIX}\(.*\).sh"`
+    echo
+    echo "--- Executing the '${suiteName}' test suite ---"
+    ( exec ${shell} ./${suite} 2>&1; )
+  done
+done
diff --git a/tests/scripts/src/test/bash/bk_test_bin_common.sh b/tests/scripts/src/test/bash/bk_test_bin_common.sh
new file mode 100644
index 0000000..0c726ea
--- /dev/null
+++ b/tests/scripts/src/test/bash/bk_test_bin_common.sh
@@ -0,0 +1,256 @@
+#!/usr/bin/env bash
+#
+# vim:et:ft=sh:sts=2:sw=2
+#
+#/**
+# * 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.
+# */
+
+# load test helpers
+. ./bk_test_helpers
+
+#------------------------------------------------------------------------------
+# suite tests
+#
+
+testDefaultVariables() {
+  source ${BK_BINDIR}/common.sh
+  assertEquals "BINDIR is not set correctly" "${BK_BINDIR}" "${BINDIR}"
+  assertEquals "BK_HOME is not set correctly" "${BK_HOMEDIR}" "${BK_HOME}"
+  assertEquals "DEFAULT_LOG_CONF is not set correctly" "${BK_CONFDIR}/log4j.properties" "${DEFAULT_LOG_CONF}"
+  assertEquals "NETTY_LEAK_DETECTION_LEVEL is not set correctly" "disabled" "${NETTY_LEAK_DETECTION_LEVEL}"
+  assertEquals "NETTY_RECYCLER_MAXCAPACITY is not set correctly" "1000" "${NETTY_RECYCLER_MAXCAPACITY}"
+  assertEquals "NETTY_RECYCLER_LINKCAPACITY is not set correctly" "1024" "${NETTY_RECYCLER_LINKCAPACITY}"
+  assertEquals "BOOKIE_MAX_HEAP_MEMORY is not set correctly" "1g" "${BOOKIE_MAX_HEAP_MEMORY}"
+  assertEquals "BOOKIE_MIN_HEAP_MEMORY is not set correctly" "1g" "${BOOKIE_MIN_HEAP_MEMORY}"
+  assertEquals "BOOKIE_MAX_DIRECT_MEMORY is not set correctly" "2g" "${BOOKIE_MAX_DIRECT_MEMORY}"
+  assertEquals "BOOKIE_MEM_OPTS is not set correctly" "-Xms1g -Xmx1g -XX:MaxDirectMemorySize=2g" "${BOOKIE_MEM_OPTS}"
+  assertEquals "BOOKIE_GC_OPTS is not set correctly" "${DEFAULT_BOOKIE_GC_OPTS}" "${BOOKIE_GC_OPTS}"
+  assertEquals "BOOKIE_GC_LOGGING_OPTS is not set correctly" "${DEFAULT_BOOKIE_GC_LOGGING_OPTS}" "${BOOKIE_GC_LOGGING_OPTS}"
+  assertEquals "CLI_MAX_HEAP_MEMORY is not set correctly" "512M" "${CLI_MAX_HEAP_MEMORY}"
+  assertEquals "CLI_MIN_HEAP_MEMORY is not set correctly" "256M" "${CLI_MIN_HEAP_MEMORY}"
+  assertEquals "CLI_MEM_OPTS is not set correctly" "-Xms256M -Xmx512M" "${CLI_MEM_OPTS}"
+  assertEquals "CLI_GC_OPTS is not set correctly" "${DEFAULT_CLI_GC_OPTS}" "${CLI_GC_OPTS}"
+  assertEquals "CLI_GC_LOGGING_OPTS is not set correctly" "${DEFAULT_CLI_GC_LOGGING_OPTS}" "${CLI_GC_LOGGING_OPTS}"
+}
+
+testFindModuleJarAt() {
+  source ${BK_BINDIR}/common.sh
+
+  MODULE="test-module"
+
+  # case 1: empty dir
+  TEST_DIR1=${BK_TMPDIR}/testdir1
+  mkdir -p ${TEST_DIR1}
+  MODULE_JAR1=$(find_module_jar_at ${TEST_DIR1} ${MODULE})
+  assertEquals "No module jar should be found at empty dir" "" "${MODULE_JAR1}"
+
+  # case 2: SNAPSHOT jar
+  TEST_FILES=(
+    "invalid-${MODULE}.jar"
+    "invalid-${MODULE}-4.8.0.jar"
+    "invalid-${MODULE}-4.8.0-SNAPSHOT.jar"
+    "${MODULE}.jar.invalid"
+    "${MODULE}-4.8.0.jar.invalid"
+    "${MODULE}-4.8.0-SNAPSHOT.jar.invalid"
+    "${MODULE}.jar"
+    "${MODULE}-4.8.0-SNAPSHOT.jar"
+  )
+
+  TEST_DIR2=${BK_TMPDIR}/testdir2
+  mkdir -p ${TEST_DIR2}
+  count=0
+  while [ "x${TEST_FILES[count]}" != "x" ]
+  do
+    touch ${TEST_DIR2}/${TEST_FILES[count]}
+    count=$(( $count + 1 ))
+  done
+  MODULE_JAR2=$(find_module_jar_at ${TEST_DIR2} ${MODULE})
+  assertEquals "${MODULE}-4.8.0-SNAPSHOT.jar is not found" "${TEST_DIR2}/${MODULE}-4.8.0-SNAPSHOT.jar" "${MODULE_JAR2}"
+
+  # case 3: release jar
+  TEST_FILES=(
+    "invalid-${MODULE}.jar"
+    "invalid-${MODULE}-4.8.0.jar"
+    "invalid-${MODULE}-4.8.0-SNAPSHOT.jar"
+    "${MODULE}.jar.invalid"
+    "${MODULE}-4.8.0.jar.invalid"
+    "${MODULE}-4.8.0-SNAPSHOT.jar.invalid"
+    "${MODULE}.jar"
+    "${MODULE}-4.8.0.jar"
+  )
+
+  TEST_DIR3=${BK_TMPDIR}/testdir3
+  mkdir -p ${TEST_DIR3}
+  count=0
+  while [ "x${TEST_FILES[count]}" != "x" ]
+  do
+    touch ${TEST_DIR3}/${TEST_FILES[count]}
+    count=$(( $count + 1 ))
+  done
+  MODULE_JAR3=$(find_module_jar_at ${TEST_DIR3} ${MODULE})
+  assertEquals "${MODULE}-4.8.0.jar is not found" "${TEST_DIR3}/${MODULE}-4.8.0.jar" "${MODULE_JAR3}"
+}
+
+testFindModuleJar() {
+  BK_HOME=${BK_TMPDIR}
+  # prepare the env files
+  mkdir -p ${BK_HOME}/conf
+  echo "" > ${BK_HOME}/conf/nettyenv.sh
+  echo "" > ${BK_HOME}/conf/bkenv.sh
+  echo "" > ${BK_HOME}/conf/bk_cli_env.sh
+
+  source ${BK_BINDIR}/common.sh
+
+  MODULE="test-module"
+  MODULE_PATH="testmodule"
+  VERSION="4.8.0"
+
+  TEST_FILES=(
+    "${MODULE}-${VERSION}.jar"
+    "lib/${MODULE}-${VERSION}.jar"
+    "${MODULE_PATH}/target/${MODULE}-${VERSION}.jar"
+  )
+  count=0
+  while [ "x${TEST_FILES[count]}" != "x" ]
+  do
+    DIR=`dirname ${BK_TMPDIR}/${TEST_FILES[count]}`
+    mkdir -p ${DIR}
+    touch ${BK_TMPDIR}/${TEST_FILES[count]}
+    count=$(( $count + 1 ))
+  done
+
+  count=0
+  while [ "x${TEST_FILES[count]}" != "x" ]
+  do
+    FILE="${BK_TMPDIR}/${TEST_FILES[count]}"
+    ACTUAL_FILE=$(find_module_jar ${MODULE_PATH} ${MODULE})
+
+    assertEquals "Module file is not found" "${FILE}" "${ACTUAL_FILE}"
+
+    # delete the file
+    rm ${FILE}
+    count=$(( $count + 1 ))
+  done
+
+  unset BK_HOME
+}
+
+testLoadEnvfiles() {
+  BK_HOME=${BK_TMPDIR}
+
+  # prepare the env files
+  mkdir -p ${BK_HOME}/conf
+  echo "NETTY_LEAK_DETECTION_LEVEL=enabled" > ${BK_HOME}/conf/nettyenv.sh
+  echo "BOOKIE_MAX_HEAP_MEMORY=2048M" > ${BK_HOME}/conf/bkenv.sh
+  echo "CLI_MAX_HEAP_MEMORY=2048M" > ${BK_HOME}/conf/bk_cli_env.sh
+
+  # load the common.sh
+  source ${BK_BINDIR}/common.sh
+
+  assertEquals "NETTY_LEAK_DETECTION_LEVEL is not set correctly" "enabled" "${NETTY_LEAK_DETECTION_LEVEL}"
+  assertEquals "BOOKIE_MAX_HEAP_MEMORY is not set correctly" "2048M" "${BOOKIE_MAX_HEAP_MEMORY}"
+  assertEquals "CLI_MAX_HEAP_MEMORY is not set correctly" "2048M" "${CLI_MAX_HEAP_MEMORY}"
+
+  unset NETTY_LEAK_DETECTION_LEVEL
+  unset BOOKIE_MAX_HEAP_MEMORY
+  unset CLI_MAX_HEAP_MEMORY
+  unset BK_HOME
+}
+
+testBuildBookieJVMOpts() {
+  source ${BK_BINDIR}/common.sh
+
+  TEST_LOG_DIR=${BK_TMPDIR}/logdir
+  TEST_GC_LOG_FILENAME="test-gc.log"
+  ACTUAL_JVM_OPTS=$(build_bookie_jvm_opts ${TEST_LOG_DIR} ${TEST_GC_LOG_FILENAME})
+  EXPECTED_JVM_OPTS="-Xms1g -Xmx1g -XX:MaxDirectMemorySize=2g ${DEFAULT_BOOKIE_GC_OPTS} ${DEFAULT_BOOKIE_GC_LOGGING_OPTS}  -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}"
+
+  assertEquals "JVM OPTS is not set correctly" "${EXPECTED_JVM_OPTS}" "${ACTUAL_JVM_OPTS}"
+}
+
+testBuildCLIJVMOpts() {
+  source ${BK_BINDIR}/common.sh
+
+  TEST_LOG_DIR=${BK_TMPDIR}/logdir
+  TEST_GC_LOG_FILENAME="test-gc.log"
+  ACTUAL_JVM_OPTS=$(build_cli_jvm_opts ${TEST_LOG_DIR} ${TEST_GC_LOG_FILENAME})
+  EXPECTED_JVM_OPTS="-Xms256M -Xmx512M ${DEFAULT_CLI_GC_OPTS} ${DEFAULT_CLI_GC_LOGGING_OPTS} -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}"
+
+  assertEquals "JVM OPTS is not set correctly" "${EXPECTED_JVM_OPTS}" "${ACTUAL_JVM_OPTS}"
+}
+
+testBuildNettyOpts() {
+  source ${BK_BINDIR}/common.sh
+
+  ACTUAL_NETTY_OPTS=$(build_netty_opts)
+  EXPECTED_NETTY_OPTS="-Dio.netty.leakDetectionLevel=disabled \
+    -Dio.netty.recycler.maxCapacity.default=1000 \
+    -Dio.netty.recycler.linkCapacity=1024"
+
+    assertEquals "Netty OPTS is not set correctly" "${EXPECTED_NETTY_OPTS}" "${ACTUAL_NETTY_OPTS}"
+}
+
+testBuildBookieOpts() {
+  source ${BK_BINDIR}/common.sh
+
+  ACTUAL_OPTS=$(build_bookie_opts)
+  EXPECTED_OPTS="-Djava.net.preferIPv4Stack=true"
+
+  assertEquals "Bookie OPTS is not set correctly" "${EXPECTED_OPTS}" "${ACTUAL_OPTS}"
+}
+
+testBuildLoggingOpts() {
+  TEST_CONF_FILE="test.conf"
+  TEST_LOG_DIR="test_log_dir"
+  TEST_LOG_FILE="test_log_file"
+  TEST_LOGGER="INFO,TEST"
+
+  EXPECTED_OPTS="-Dlog4j.configuration=${TEST_CONF_FILE} \
+    -Dbookkeeper.root.logger=${TEST_LOGGER} \
+    -Dbookkeeper.log.dir=${TEST_LOG_DIR} \
+    -Dbookkeeper.log.file=${TEST_LOG_FILE}"
+  ACTUAL_OPTS=$(build_logging_opts ${TEST_CONF_FILE} ${TEST_LOG_DIR} ${TEST_LOG_FILE} ${TEST_LOGGER})
+
+  assertEquals "Logging OPTS is not set correctly" "${EXPECTED_OPTS}" "${ACTUAL_OPTS}"
+}
+
+testBuildCLILoggingOpts() {
+  TEST_CONF_FILE="test.conf"
+  TEST_LOG_DIR="test_log_dir"
+  TEST_LOG_FILE="test_log_file"
+  TEST_LOGGER="INFO,TEST"
+
+  EXPECTED_OPTS="-Dlog4j.configuration=${TEST_CONF_FILE} \
+    -Dbookkeeper.cli.root.logger=${TEST_LOGGER} \
+    -Dbookkeeper.cli.log.dir=${TEST_LOG_DIR} \
+    -Dbookkeeper.cli.log.file=${TEST_LOG_FILE}"
+  ACTUAL_OPTS=$(build_cli_logging_opts ${TEST_CONF_FILE} ${TEST_LOG_DIR} ${TEST_LOG_FILE} ${TEST_LOGGER})
+
+  assertEquals "Logging OPTS is not set correctly" "${EXPECTED_OPTS}" "${ACTUAL_OPTS}"
+}
+
+#------------------------------------------------------------------------------
+# suite functions
+#
+
+oneTimeSetUp() {
+  bk_oneTimeSetUp
+}
+
+# load and run shUnit2
+. ${BK_SHUNIT}
diff --git a/tests/scripts/src/test/bash/bk_test_helpers b/tests/scripts/src/test/bash/bk_test_helpers
new file mode 100644
index 0000000..6f43e79
--- /dev/null
+++ b/tests/scripts/src/test/bash/bk_test_helpers
@@ -0,0 +1,94 @@
+#!/usr/bin/env bash
+#
+# vim:et:ft=sh:sts=2:sw=2
+#
+#/**
+# * 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.
+# */
+
+# name of script
+BK_ARGV0=`basename "$0"`
+
+# path to shUnit2 library. can be overridden by setting BK_SHUNIT_INC.
+BK_SHUNIT=${BK_SHUNIT_INC:-../../../target/lib/shunit2-2.1.7/shunit2}
+
+# path to bk bin directory.
+TESTDIR=`dirname "$0"`
+BK_BINDIR=`cd ${TESTDIR}/../../../../../bin;pwd`
+BK_HOMEDIR=`cd ${TESTDIR}/../../../../..;pwd`
+BK_CONFDIR=`cd ${TESTDIR}/../../../../../conf;pwd`
+
+#
+# test helper functions
+#
+
+# message functions
+bk_trace() { echo "bk_test:TRACE $@" >&2; }
+bk_debug() { echo "bk_test:DEBUG $@" >&2; }
+bk_info() { echo "bk_test:INFO $@" >&2; }
+bk_warn() { echo "bk_test:WARN $@" >&2; }
+bk_error() { echo "bk_test:ERROR $@" >&2; }
+bk_fatal() { echo "bk_test:FATAL $@" >&2; }
+
+bk_oneTimeSetUp() {
+  # these will be cleaned up automatically by shunit2
+  BK_TMPDIR=${SHUNIT_TMPDIR}
+  stdoutF="${BK_TMPDIR}/stdout"
+  stderrF="${BK_TMPDIR}/stderr"
+  expectedF="${BK_TMPDIR}/expected"
+}
+
+# Assert the success of an operation.
+#
+# If an operation is not successful (i.e. it returns a non-zero return code)
+# dump the output of the stderrF to the screen.
+#
+# Args:
+#   message: string: message to output [optional]
+#   result: integer: operation result
+assertSuccess() {
+  if [ $# -eq 2 ]; then
+    bk_message_=$1
+    shift
+  else
+    bk_message_=''
+  fi
+  bk_result_=$1
+
+  assertEquals "${bk_message_}" ${SHUNIT_TRUE} ${bk_result_}
+  [ ${bk_result_} -eq ${SHUNIT_TRUE} ] || cat "${stderrF}"
+
+  unset bk_message_ bk_result_
+}
+
+assertError() {
+  if [ $# -eq 2 ]; then
+    bk_message_="$1: "
+    shift
+  else
+    bk_message_=''
+  fi
+  bk_error_=$1
+
+  bk_file_=${stderrF}
+  grep "^bk_test:ERROR.*${bk_error_}" "${bk_file_}" >/dev/null
+  bk_result_=$?
+  assertTrue "${bk_message_}missing '${bk_error_}' error" ${bk_result_}
+  [ ${bk_result_} -eq 0 ] || cat "${bk_file_}"
+
+  unset bk_file_ bk_error_ bk_message_ bk_result_
+}
diff --git a/tests/scripts/src/test/bash/versions b/tests/scripts/src/test/bash/versions
new file mode 100644
index 0000000..9b868f5
--- /dev/null
+++ b/tests/scripts/src/test/bash/versions
@@ -0,0 +1,173 @@
+#!/usr/bin/env bash
+#
+# vim:et:ft=sh:sts=2:sw=2
+#
+#/**
+# * 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.
+# */
+
+ARGV0=`basename "$0"`
+LSB_RELEASE="/etc/lsb-release"
+VERSIONS_SHELLS="/bin/bash /bin/sh"
+
+true; TRUE=$?
+false; FALSE=$?
+ERROR=2
+
+UNAME_R=`uname -r`
+UNAME_S=`uname -s`
+
+__versions_haveStrings=${ERROR}
+
+versions_osName() {
+  os_name_='unrecognized'
+  os_system_=${UNAME_S}
+  os_release_=${UNAME_R}
+  case ${os_system_} in
+    CYGWIN_NT-*) os_name_='Cygwin' ;;
+    Darwin)
+      os_name_=`/usr/bin/sw_vers -productName`
+      os_version_=`versions_osVersion`
+      case ${os_version_} in
+        10.4|10.4.[0-9]*) os_name_='Mac OS X Tiger' ;;
+        10.5|10.5.[0-9]*) os_name_='Mac OS X Leopard' ;;
+        10.6|10.6.[0-9]*) os_name_='Mac OS X Snow Leopard' ;;
+        10.7|10.7.[0-9]*) os_name_='Mac OS X Lion' ;;
+        10.8|10.8.[0-9]*) os_name_='Mac OS X Mountain Lion' ;;
+        10.9|10.9.[0-9]*) os_name_='Mac OS X Mavericks' ;;
+        10.10|10.10.[0-9]*) os_name_='Mac OS X Yosemite' ;;
+        10.11|10.11.[0-9]*) os_name_='Mac OS X El Capitan' ;;
+        10.12|10.12.[0-9]*) os_name_='macOS Sierra' ;;
+        10.13|10.13.[0-9]*) os_name_='macOS High Sierra' ;;
+        *) os_name_='macOS' ;;
+      esac
+      ;;
+    FreeBSD) os_name_='FreeBSD' ;;
+    Linux) os_name_='Linux' ;;
+    SunOS)
+      os_name_='SunOS'
+      if [ -r '/etc/release' ]; then
+        if grep 'OpenSolaris' /etc/release >/dev/null; then
+          os_name_='OpenSolaris'
+        else
+          os_name_='Solaris'
+        fi
+      fi
+      ;;
+  esac
+
+  echo ${os_name_}
+  unset os_name_ os_system_ os_release_ os_version_
+}
+
+versions_osVersion() {
+  os_version_='unrecognized'
+  os_system_=${UNAME_S}
+  os_release_=${UNAME_R}
+  case ${os_system_} in
+    CYGWIN_NT-*)
+      os_version_=`expr "${os_release_}" : '\([0-9]*\.[0-9]\.[0-9]*\).*'`
+      ;;
+    Darwin)
+      os_version_=`/usr/bin/sw_vers -productVersion`
+      ;;
+    FreeBSD)
+      os_version_=`expr "${os_release_}" : '\([0-9]*\.[0-9]*\)-.*'`
+      ;;
+    Linux)
+      if [ -r '/etc/os-release' ]; then
+          os_version_=`awk -F= '$1~/PRETTY_NAME/{print $2}' /etc/os-release \
+            |sed 's/"//g'`
+      elif [ -r '/etc/redhat-release' ]; then
+        os_version_=`cat /etc/redhat-release`
+      elif [ -r '/etc/SuSE-release' ]; then
+        os_version_=`head -n 1 /etc/SuSE-release`
+      elif [ -r "${LSB_RELEASE}" ]; then
+        if grep -q 'DISTRIB_ID=Ubuntu' "${LSB_RELEASE}"; then
+          # shellcheck disable=SC2002
+          os_version_=`cat "${LSB_RELEASE}" \
+            |awk -F= '$1~/DISTRIB_DESCRIPTION/{print $2}' \
+            |sed 's/"//g;s/ /-/g'`
+        fi
+      fi
+      ;;
+    SunOS)
+      if [ -r '/etc/release' ]; then
+        if grep 'OpenSolaris' /etc/release >/dev/null; then  # OpenSolaris
+          os_version_=`grep 'OpenSolaris' /etc/release |awk '{print $2"("$3")"}'`
+        else  # Solaris
+          major_=`echo "${os_release_}" |sed 's/[0-9]*\.\([0-9]*\)/\1/'`
+          minor_=`grep Solaris /etc/release |sed 's/[^u]*\(u[0-9]*\).*/\1/'`
+          os_version_="${major_}${minor_}"
+        fi
+      fi
+      ;;
+  esac
+
+  echo "${os_version_}"
+  unset os_release_ os_system_ os_version_ major_ minor_
+}
+
+versions_shellVersion() {
+  shell_=$1
+
+  shell_present_=${FALSE}
+  case "${shell_}" in
+    ash) [ -x '/bin/busybox' ] && shell_present_=${TRUE} ;;
+    *) [ -x "${shell_}" ] && shell_present_=${TRUE} ;;
+  esac
+  if [ ${shell_present_} -eq ${FALSE} ]; then
+    echo 'not installed'
+    return ${FALSE}
+  fi
+
+  version_=''
+  case ${shell_} in
+    */sh)
+      # This could be one of any number of shells. Try until one fits.
+      version_=''
+      [ -z "${version_}" ] && version_=`versions_shell_bash "${shell_}"`
+      ;;
+    */bash) version_=`versions_shell_bash "${shell_}"` ;;
+    *) version_='invalid'
+  esac
+
+  echo "${version_:-unknown}"
+  unset shell_ version_
+}
+
+versions_shell_bash() {
+  $1 --version : 2>&1 |grep 'GNU bash' |sed 's/.*version \([^ ]*\).*/\1/'
+}
+
+versions_main() {
+  # Treat unset variables as an error.
+  set -u
+
+  os_name=`versions_osName`
+  os_version=`versions_osVersion`
+  echo "os: ${os_name} version: ${os_version}"
+
+  for shell in ${VERSIONS_SHELLS}; do
+    shell_version=`versions_shellVersion "${shell}"`
+    echo "shell: ${shell} version: ${shell_version}"
+  done
+}
+
+if [ "${ARGV0}" = 'versions' ]; then
+  versions_main "$@"
+fi

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