You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2017/03/04 18:22:09 UTC

zookeeper git commit: ZOOKEEPER-2693: DOS attack on wchp/wchc four letter words (4lw)

Repository: zookeeper
Updated Branches:
  refs/heads/master d13d2f08c -> 5fe68506f


ZOOKEEPER-2693: DOS attack on wchp/wchc four letter words (4lw)

The previous patch is not valid anymore as design has changed. Update patch based on phunt 's feedback.
See ZOOKEEPER-2693 JIRA for details. This patch is still targeting for branch-3.5 / master. Patch for branch-3.4 will be created shortly.

The gist of the patch is:
* Introduced a new configuration option which encodes a white list of 4lw commands. A 4lw command that is not on the white list will not be enabled on server side.
* For branch-3.5, the default value of the configuration option is empty. We want users to use the alternative AdminServer anyway.

Author: Michael Han <ha...@apache.org>

Reviewers: Rakesh Radhakrishnan <rakeshrapache.org>, Mohammad Arshad <ar...@apache.org>, Flavio Junqueira <fp...@apache.org>, Edward Ribeiro <ed...@gmail.com>, Abraham Fine <af...@apache.org>

Closes #179 from hanm/ZOOKEEPER-2693 and squashes the following commits:

1ffd114 [Michael Han] Tweak tests.
b4c421d [Michael Han] ZOOKEEPER-2693: DOS attack on wchp/wchc four letter words (4lw). Introduce a new configuration option 4lw.commands.whitelist that disables all 4lw (except srvr) by default to avoid the exploits reported on wchp/wchc commands.


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/5fe68506
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/5fe68506
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/5fe68506

Branch: refs/heads/master
Commit: 5fe68506f217246c7ebd96803f9c78e13ec2f11a
Parents: d13d2f0
Author: Michael Han <ha...@apache.org>
Authored: Sat Mar 4 10:22:03 2017 -0800
Committer: Michael Han <ha...@apache.org>
Committed: Sat Mar 4 10:22:03 2017 -0800

----------------------------------------------------------------------
 .../content/xdocs/zookeeperAdmin.xml            |  57 ++++-
 .../apache/zookeeper/server/NIOServerCnxn.java  |  21 +-
 .../zookeeper/server/NettyServerCnxn.java       |  22 +-
 .../server/command/FourLetterCommands.java      |  86 ++++++-
 .../zookeeper/server/command/NopCommand.java    |  41 +++
 .../test/org/apache/zookeeper/ZKTestCase.java   |   3 +
 .../test/FourLetterWordsWhiteListTest.java      | 251 +++++++++++++++++++
 7 files changed, 466 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
----------------------------------------------------------------------
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
index f9dff1e..6db8e40 100644
--- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
+++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
@@ -1155,6 +1155,40 @@ server.3=zoo3:2888:3888</programlisting>
             </listitem>
           </varlistentry>
 
+          <varlistentry>
+            <term>4lw.commands.whitelist</term>
+
+            <listitem>
+              <para>(Java system property: <emphasis
+                      role="bold">zookeeper.4lw.commands.whitelist</emphasis>)</para>
+
+              <para><emphasis role="bold">New in 3.5.3:</emphasis>
+                A list of comma separated <ulink url="#sc_4lw">Four Letter Words</ulink>
+                commands that user wants to use. A valid Four Letter Words
+                command must be put in this list else ZooKeeper server will
+                not enable the command.
+                By default the whitelist only contains "srvr" command
+                which zkServer.sh uses. The rest of four letter word commands are disabled
+                by default.
+              </para>
+
+              <para>Here's an example of the configuration that enables stat, ruok, conf, and isro
+              command while disabling the rest of Four Letter Words command:</para>
+              <programlisting>
+                4lw.commands.whitelist=stat, ruok, conf, isro
+              </programlisting>
+
+              <para>If you really need enable all four letter word commands by default, you can use
+                the asterisk option so you don't have to include every command one by one in the list.
+                As an example, this will enable all four letter word commands:
+              </para>
+              <programlisting>
+                4lw.commands.whitelist=*
+              </programlisting>
+
+            </listitem>
+          </varlistentry>
+
         </variablelist>
         <para></para>
       </section>
@@ -1639,7 +1673,7 @@ server.3=zoo3:2888:3888</programlisting>
     <section id="sc_zkCommands">
       <title>ZooKeeper Commands</title>
 
-      <section>
+      <section id="sc_4lw">
         <title>The Four Letter Words</title>
         <para>ZooKeeper responds to a small set of commands. Each command is
         composed of four letters. You issue the commands to ZooKeeper via telnet
@@ -1650,7 +1684,16 @@ server.3=zoo3:2888:3888</programlisting>
         while "srvr" and "cons" give extended details on server and
         connections respectively.</para>
 
-        <variablelist>
+        <para><emphasis role="bold">New in 3.5.3:</emphasis>
+          Four Letter Words need to be explicitly white listed before using.
+          Please refer <emphasis role="bold">4lw.commands.whitelist</emphasis>
+           described in <ulink url="#sc_clusterOptions">
+            cluster configuration section</ulink> for details.
+          Moving forward, Four Letter Words will be deprecated, please use
+          <ulink url="#sc_adminserver">AdminServer</ulink> instead.
+        </para>
+
+      <variablelist>
           <varlistentry>
             <term>conf</term>
 
@@ -2125,6 +2168,16 @@ server.3=zoo3:2888:3888</programlisting>
             usage limit that would cause the system to swap.</para>
           </listitem>
         </varlistentry>
+
+        <varlistentry>
+          <term>Publicly accessible deployment</term>
+          <listitem>
+            <para>
+              A ZooKeeper ensemble is expected to operate in a trusted computing environment.
+              It is thus recommended to deploy ZooKeeper behind a firewall.
+            </para>
+          </listitem>
+        </varlistentry>
       </variablelist>
     </section>
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
index 76b9be0..6dae02d 100644
--- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
+++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
@@ -47,6 +47,7 @@ import org.apache.zookeeper.server.NIOServerCnxnFactory.SelectorThread;
 import org.apache.zookeeper.server.command.CommandExecutor;
 import org.apache.zookeeper.server.command.FourLetterCommands;
 import org.apache.zookeeper.server.command.SetTraceMaskCommand;
+import org.apache.zookeeper.server.command.NopCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -478,12 +479,11 @@ public class NIOServerCnxn extends ServerCnxn {
     {
         // We take advantage of the limited size of the length to look
         // for cmds. They are all 4-bytes which fits inside of an int
-        String cmd = FourLetterCommands.getCmdMapView().get(len);
-        if (cmd == null) {
+        if (!FourLetterCommands.isKnown(len)) {
             return false;
         }
-        LOG.info("Processing " + cmd + " command from "
-                + sock.socket().getRemoteSocketAddress());
+
+        String cmd = FourLetterCommands.getCommandString(len);
         packetReceived();
 
         /** cancel the selection key to remove the socket handling
@@ -505,6 +505,19 @@ public class NIOServerCnxn extends ServerCnxn {
 
         final PrintWriter pwriter = new PrintWriter(
                 new BufferedWriter(new SendBufferWriter()));
+
+        // ZOOKEEPER-2693: don't execute 4lw if it's not enabled.
+        if (!FourLetterCommands.isEnabled(cmd)) {
+            LOG.debug("Command {} is not executed because it is not in the whitelist.", cmd);
+            NopCommand nopCmd = new NopCommand(pwriter, this, cmd +
+                    " is not executed because it is not in the whitelist.");
+            nopCmd.start();
+            return true;
+        }
+
+        LOG.info("Processing " + cmd + " command from "
+                + sock.socket().getRemoteSocketAddress());
+
         if (len == FourLetterCommands.setTraceMaskCmd) {
             incomingBuffer = ByteBuffer.allocate(8);
             int rc = sock.read(incomingBuffer);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
index 9b6f28e..cf5bd8a 100644
--- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
+++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
@@ -41,6 +41,7 @@ import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.WatcherEvent;
 import org.apache.zookeeper.server.command.CommandExecutor;
 import org.apache.zookeeper.server.command.FourLetterCommands;
+import org.apache.zookeeper.server.command.NopCommand;
 import org.apache.zookeeper.server.command.SetTraceMaskCommand;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
@@ -267,17 +268,30 @@ public class NettyServerCnxn extends ServerCnxn {
     {
         // We take advantage of the limited size of the length to look
         // for cmds. They are all 4-bytes which fits inside of an int
-        String cmd = FourLetterCommands.getCmdMapView().get(len);
-        if (cmd == null) {
+        if (!FourLetterCommands.isKnown(len)) {
             return false;
         }
+
+        String cmd = FourLetterCommands.getCommandString(len);
+
         channel.setInterestOps(0).awaitUninterruptibly();
-        LOG.info("Processing " + cmd + " command from "
-                + channel.getRemoteAddress());
         packetReceived();
 
         final PrintWriter pwriter = new PrintWriter(
                 new BufferedWriter(new SendBufferWriter()));
+
+        // ZOOKEEPER-2693: don't execute 4lw if it's not enabled.
+        if (!FourLetterCommands.isEnabled(cmd)) {
+            LOG.debug("Command {} is not executed because it is not in the whitelist.", cmd);
+            NopCommand nopCmd = new NopCommand(pwriter, this, cmd +
+                    " is not executed because it is not in the whitelist.");
+            nopCmd.start();
+            return true;
+        }
+
+        LOG.info("Processing " + cmd + " command from "
+                + channel.getRemoteAddress());
+
        if (len == FourLetterCommands.setTraceMaskCmd) {
             ByteBuffer mask = ByteBuffer.allocate(8);
             message.readBytes(mask);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java
index 0364f44..82e09dc 100644
--- a/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java
+++ b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java
@@ -18,10 +18,15 @@
 
 package org.apache.zookeeper.server.command;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.nio.ByteBuffer;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.Arrays;
 
 /**
  * This class contains constants for all the four letter commands
@@ -153,11 +158,82 @@ public class FourLetterCommands {
      */
     public final static int telnetCloseCmd = 0xfff4fffd;
 
-    final static HashMap<Integer, String> cmd2String =
-        new HashMap<Integer, String>();
+    private static final String ZOOKEEPER_4LW_COMMANDS_WHITELIST = "zookeeper.4lw.commands.whitelist";
+
+    private static final Logger LOG = LoggerFactory.getLogger(FourLetterCommands.class);
+
+    private static final Map<Integer, String> cmd2String = new HashMap<Integer, String>();
+
+    private static final Set<String> whiteListedCommands = new HashSet<String>();
+
+    private static boolean whiteListInitialized = false;
+
+    // @VisibleForTesting
+    public static void resetWhiteList() {
+        whiteListInitialized = false;
+        whiteListedCommands.clear();
+    }
+
+    /**
+     * Return the string representation of the specified command code.
+     */
+    public static String getCommandString(int command) {
+        return cmd2String.get(command);
+    }
+
+    /**
+     * Check if the specified command code is from a known command.
+     *
+     * @param command The integer code of command.
+     * @return true if the specified command is known, false otherwise.
+     */
+    public static boolean isKnown(int command) {
+        return cmd2String.containsKey(command);
+    }
+
+    /**
+     * Check if the specified command is enabled.
+     *
+     * In ZOOKEEPER-2693 we introduce a configuration option to only
+     * allow a specific set of white listed commands to execute.
+     * A command will only be executed if it is also configured
+     * in the white list.
+     *
+     * @param command The command string.
+     * @return true if the specified command is enabled
+     */
+    public static boolean isEnabled(String command) {
+        if (whiteListInitialized) {
+            return whiteListedCommands.contains(command);
+        }
+
+        String commands = System.getProperty(ZOOKEEPER_4LW_COMMANDS_WHITELIST);
+        if (commands != null) {
+            String[] list = commands.split(",");
+            for (String cmd : list) {
+                if (cmd.trim().equals("*")) {
+                    for (Map.Entry<Integer, String> entry : cmd2String.entrySet()) {
+                        whiteListedCommands.add(entry.getValue());
+                    }
+                    break;
+                }
+                if (!cmd.trim().isEmpty()) {
+                    whiteListedCommands.add(cmd.trim());
+                }
+            }
+        }
 
-    public static Map<Integer, String> getCmdMapView() {
-        return Collections.unmodifiableMap(cmd2String);
+        // It is sad that isro and srvr are used by ZooKeeper itself. Need fix this
+        // before deprecating 4lw.
+        if (System.getProperty("readonlymode.enabled", "false").equals("true")) {
+            whiteListedCommands.add("isro");
+        }
+        // zkServer.sh depends on "srvr".
+        whiteListedCommands.add("srvr");
+        whiteListInitialized = true;
+        LOG.info("The list of known four letter word commands is : {}", Arrays.asList(cmd2String));
+        LOG.info("The list of enabled four letter word commands is : {}", Arrays.asList(whiteListedCommands));
+        return whiteListedCommands.contains(command);
     }
 
     // specify all of the commands that are available

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/main/org/apache/zookeeper/server/command/NopCommand.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/command/NopCommand.java b/src/java/main/org/apache/zookeeper/server/command/NopCommand.java
new file mode 100644
index 0000000..4924fcf
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/server/command/NopCommand.java
@@ -0,0 +1,41 @@
+/**
+ * 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.zookeeper.server.command;
+
+import java.io.PrintWriter;
+
+import org.apache.zookeeper.server.ServerCnxn;
+
+/**
+ * A command that does not do anything except reply to client with predefined message.
+ * It is used to inform clients who execute none white listed four letter word commands.
+ */
+public class NopCommand extends AbstractFourLetterCommand {
+    private String msg;
+
+    public NopCommand(PrintWriter pw, ServerCnxn serverCnxn, String msg) {
+        super(pw, serverCnxn);
+        this.msg = msg;
+    }
+
+    @Override
+    public void commandRun() {
+        pw.println(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/test/org/apache/zookeeper/ZKTestCase.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/ZKTestCase.java b/src/java/test/org/apache/zookeeper/ZKTestCase.java
index 4776f8b..54a6be5 100644
--- a/src/java/test/org/apache/zookeeper/ZKTestCase.java
+++ b/src/java/test/org/apache/zookeeper/ZKTestCase.java
@@ -51,6 +51,9 @@ public class ZKTestCase {
             // accidentally attempting to start multiple admin servers on the
             // same port.
             System.setProperty("zookeeper.admin.enableServer", "false");
+            // ZOOKEEPER-2693 disables all 4lw by default.
+            // Here we enable the 4lw which ZooKeeper tests depends.
+            System.setProperty("zookeeper.4lw.commands.whitelist", "*");
             testName = method.getName();
             LOG.info("STARTING " + testName);
         }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/5fe68506/src/java/test/org/apache/zookeeper/test/FourLetterWordsWhiteListTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsWhiteListTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsWhiteListTest.java
new file mode 100644
index 0000000..f5d6967
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsWhiteListTest.java
@@ -0,0 +1,251 @@
+/**
+ * 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.zookeeper.test;
+
+import java.io.IOException;
+
+import org.apache.zookeeper.TestableZooKeeper;
+import org.apache.zookeeper.common.X509Exception.SSLContextException;
+
+import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord;
+
+import org.apache.zookeeper.server.command.FourLetterCommands;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FourLetterWordsWhiteListTest extends ClientBase {
+    protected static final Logger LOG =
+        LoggerFactory.getLogger(FourLetterWordsWhiteListTest.class);
+
+    /*
+     * ZOOKEEPER-2693: test white list of four letter words.
+     * For 3.5.x default white list is empty. Verify that is
+     * the case (except 'stat' command which is enabled in ClientBase
+     * which other tests depend on.).
+     */
+    @Test(timeout=30000)
+    public void testFourLetterWordsAllDisabledByDefault() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist", "stat");
+        startServer();
+
+        // Default white list for 3.5.x is empty, so all command should fail.
+        verifyAllCommandsFail();
+
+        TestableZooKeeper zk = createClient();
+
+        verifyAllCommandsFail();
+
+        zk.getData("/", true, null);
+
+        verifyAllCommandsFail();
+
+        zk.close();
+
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyAllCommandsFail();
+    }
+
+    @Test(timeout=30000)
+    public void testFourLetterWordsEnableSomeCommands() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist", "stat, ruok, isro");
+        startServer();
+        // stat, ruok and isro are white listed.
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyExactMatch("ruok", "imok");
+        verifyExactMatch("isro", "rw");
+
+        // Rest of commands fail.
+        verifyExactMatch("conf", generateExpectedMessage("conf"));
+        verifyExactMatch("cons", generateExpectedMessage("cons"));
+        verifyExactMatch("crst", generateExpectedMessage("crst"));
+        verifyExactMatch("dirs", generateExpectedMessage("dirs"));
+        verifyExactMatch("dump", generateExpectedMessage("dump"));
+        verifyExactMatch("envi", generateExpectedMessage("envi"));
+        verifyExactMatch("gtmk", generateExpectedMessage("gtmk"));
+        verifyExactMatch("stmk", generateExpectedMessage("stmk"));
+        verifyExactMatch("srst", generateExpectedMessage("srst"));
+        verifyExactMatch("wchc", generateExpectedMessage("wchc"));
+        verifyExactMatch("wchp", generateExpectedMessage("wchp"));
+        verifyExactMatch("wchs", generateExpectedMessage("wchs"));
+        verifyExactMatch("mntr", generateExpectedMessage("mntr"));
+    }
+
+    @Test(timeout=30000)
+    public void testISROEnabledWhenReadOnlyModeEnabled() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist", "stat");
+        System.setProperty("readonlymode.enabled", "true");
+        startServer();
+        verifyExactMatch("isro", "rw");
+        System.clearProperty("readonlymode.enabled");
+    }
+
+    @Test(timeout=30000)
+    public void testFourLetterWordsInvalidConfiguration() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist", "foo bar" +
+                " foo,,, " +
+                "bar :.,@#$%^&*() , , , , bar, bar, stat,        ");
+        startServer();
+
+        // Just make sure we are good when admin made some mistakes in config file.
+        verifyAllCommandsFail();
+        // But still, what's valid in white list will get through.
+        verifyFuzzyMatch("stat", "Outstanding");
+    }
+
+    @Test(timeout=30000)
+    public void testFourLetterWordsEnableAllCommandsThroughAsterisk() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist", "*");
+        startServer();
+        verifyAllCommandsSuccess();
+    }
+
+    @Test(timeout=30000)
+    public void testFourLetterWordsEnableAllCommandsThroughExplicitList() throws Exception {
+        stopServer();
+        FourLetterCommands.resetWhiteList();
+        System.setProperty("zookeeper.4lw.commands.whitelist",
+                "ruok, envi, conf, stat, srvr, cons, dump," +
+                        "wchs, wchp, wchc, srst, crst, " +
+                        "dirs, mntr, gtmk, isro, stmk");
+        startServer();
+        verifyAllCommandsSuccess();
+    }
+
+
+    private void verifyAllCommandsSuccess() throws Exception {
+        verifyExactMatch("ruok", "imok");
+        verifyFuzzyMatch("envi", "java.version");
+        verifyFuzzyMatch("conf", "clientPort");
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", "queued");
+        verifyFuzzyMatch("dump", "Session");
+        verifyFuzzyMatch("wchs", "watches");
+        verifyFuzzyMatch("wchp", "");
+        verifyFuzzyMatch("wchc", "");
+
+        verifyFuzzyMatch("srst", "reset");
+        verifyFuzzyMatch("crst", "reset");
+
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", "queued");
+        verifyFuzzyMatch("gtmk", "306");
+        verifyFuzzyMatch("isro", "rw");
+
+        TestableZooKeeper zk = createClient();
+        String sid = getHexSessionId(zk.getSessionId());
+
+        verifyFuzzyMatch("stat", "queued");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", sid);
+        verifyFuzzyMatch("dump", sid);
+        verifyFuzzyMatch("dirs", "size");
+
+        zk.getData("/", true, null);
+
+        verifyFuzzyMatch("stat", "queued");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", sid);
+        verifyFuzzyMatch("dump", sid);
+
+        verifyFuzzyMatch("wchs", "watching 1");
+        verifyFuzzyMatch("wchp", sid);
+        verifyFuzzyMatch("wchc", sid);
+        verifyFuzzyMatch("dirs", "size");
+        zk.close();
+
+        verifyExactMatch("ruok", "imok");
+        verifyFuzzyMatch("envi", "java.version");
+        verifyFuzzyMatch("conf", "clientPort");
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", "queued");
+        verifyFuzzyMatch("dump", "Session");
+        verifyFuzzyMatch("wchs", "watch");
+        verifyFuzzyMatch("wchp", "");
+        verifyFuzzyMatch("wchc", "");
+
+        verifyFuzzyMatch("srst", "reset");
+        verifyFuzzyMatch("crst", "reset");
+
+        verifyFuzzyMatch("stat", "Outstanding");
+        verifyFuzzyMatch("srvr", "Outstanding");
+        verifyFuzzyMatch("cons", "queued");
+        verifyFuzzyMatch("mntr", "zk_server_state\tstandalone");
+        verifyFuzzyMatch("mntr", "num_alive_connections");
+        verifyFuzzyMatch("stat", "Connections");
+        verifyFuzzyMatch("srvr", "Connections");
+        verifyFuzzyMatch("dirs", "size");
+    }
+
+    private void verifyAllCommandsFail() throws Exception {
+        verifyExactMatch("ruok", generateExpectedMessage("ruok"));
+        verifyExactMatch("conf", generateExpectedMessage("conf"));
+        verifyExactMatch("cons", generateExpectedMessage("cons"));
+        verifyExactMatch("crst", generateExpectedMessage("crst"));
+        verifyExactMatch("dirs", generateExpectedMessage("dirs"));
+        verifyExactMatch("dump", generateExpectedMessage("dump"));
+        verifyExactMatch("envi", generateExpectedMessage("envi"));
+        verifyExactMatch("gtmk", generateExpectedMessage("gtmk"));
+        verifyExactMatch("stmk", generateExpectedMessage("stmk"));
+        verifyExactMatch("srst", generateExpectedMessage("srst"));
+        verifyExactMatch("wchc", generateExpectedMessage("wchc"));
+        verifyExactMatch("wchp", generateExpectedMessage("wchp"));
+        verifyExactMatch("wchs", generateExpectedMessage("wchs"));
+        verifyExactMatch("mntr", generateExpectedMessage("mntr"));
+        verifyExactMatch("isro", generateExpectedMessage("isro"));
+
+        // srvr is enabled by default due to the sad fact zkServer.sh uses it.
+        verifyFuzzyMatch("srvr", "Outstanding");
+    }
+
+    private String sendRequest(String cmd) throws IOException, SSLContextException {
+      HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0);
+      return send4LetterWord(hpobj.host, hpobj.port, cmd);
+    }
+
+    private void verifyFuzzyMatch(String cmd, String expected) throws IOException, SSLContextException {
+        String resp = sendRequest(cmd);
+        LOG.info("cmd " + cmd + " expected " + expected + " got " + resp);
+        Assert.assertTrue(resp.contains(expected));
+    }
+
+    private String generateExpectedMessage(String command) {
+        return command + " is not executed because it is not in the whitelist.";
+    }
+
+    private void verifyExactMatch(String cmd, String expected) throws IOException, SSLContextException {
+        String resp = sendRequest(cmd);
+        LOG.info("cmd " + cmd + " expected an exact match of " + expected + "; got " + resp);
+        Assert.assertTrue(resp.trim().equals(expected));
+    }
+}