You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/05/21 21:55:04 UTC

[GitHub] sijie closed pull request #1416: Provide zookeeper startup script

sijie closed pull request #1416: Provide zookeeper startup script
URL: https://github.com/apache/bookkeeper/pull/1416
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/bin/bookkeeper b/bin/bookkeeper
index 47c4e3f0b..ceb047020 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 dab21f6f7..3a8bbdcae 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 665db8e71..ef25654c9 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 c6300a1f0..5ae8fa1f4 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 b0ba7e9f8..7f523dbe2 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 9f9ccc648..1f437abec 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 eebc4ff18..f6d061611 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 000000000..407baeb24
--- /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 000000000..5d53861c4
--- /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 0d39b3441..5c8e6646d 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 78e6945d0..3fccab981 100755
--- a/docker/scripts/apply-config-from-env.py
+++ b/docker/scripts/apply-config-from-env.py
@@ -42,6 +42,7 @@ def mylistdir(dir):
 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 @@ def mylistdir(dir):
                 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 000000000..ece24ba1b
--- /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 7ade924bc..86911170e 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 91e13f064..47a21bf50 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 000000000..09aab1d6c
--- /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/init_generic.sh b/docker/scripts/init_generic.sh
new file mode 100755
index 000000000..d1ff3234d
--- /dev/null
+++ b/docker/scripts/init_generic.sh
@@ -0,0 +1,23 @@
+#!/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
diff --git a/docker/scripts/init_standalone.sh b/docker/scripts/init_standalone.sh
new file mode 100755
index 000000000..5007b8c14
--- /dev/null
+++ b/docker/scripts/init_standalone.sh
@@ -0,0 +1,26 @@
+#!/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.
+# */
+
+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 000000000..803ef91d7
--- /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 71701f55f..fa252479a 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 4006daf6b..892eb3594 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 6efec4c44..1288695e0 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 @@ protected void configure() {
             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 void start() {
         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 ba1d3842a..8f5fb8ac0 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 String getInternalServiceUri() {
 
     @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();


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services