Repository: zookeeper
Updated Branches:
refs/heads/branch-3.4 b26eaf349 -> 835377f0e
ZOOKEEPER-2693: DOS attack on wchp/wchc four letter words (4lw)
Similar as pull request 179, this PR introduces new property zookeeper.4lw.commands.whitelist
to branch-3.4.
Unlike branch-3.5 where all 4lw (with few exceptions) is disabled by default, for branch-3.4
only "wchp" and "wchc" are disabled by default - since 4lw is widely used and there is no
alternatives in branch-3.4 so we just disable the exploitable ones.
Author: Michael Han <hanm@apache.org>
Reviewers: Rakesh Radhakrishnan <rakeshr@apache.org>
Closes #183 from hanm/ZOOKEEPER-2693-br-3.4 and squashes the following commits:
d060ddc [Michael Han] update doc.
2ce4ebd [Michael Han] ZOOKEEPER-2693: DOS attack on wchp/wchc four letter words (4lw). Initial
commit for branch-3.4.
Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/835377f0
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/835377f0
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/835377f0
Branch: refs/heads/branch-3.4
Commit: 835377f0e1cd215e791ed29c0bcff95e625f299c
Parents: b26eaf3
Author: Michael Han <hanm@apache.org>
Authored: Tue Mar 7 17:34:34 2017 +0530
Committer: Rakesh Radhakrishnan <rakeshr@apache.org>
Committed: Tue Mar 7 17:34:34 2017 +0530
----------------------------------------------------------------------
.../content/xdocs/zookeeperAdmin.xml | 44 ++++
.../apache/zookeeper/server/NIOServerCnxn.java | 33 ++-
.../zookeeper/server/NettyServerCnxn.java | 32 ++-
.../org/apache/zookeeper/server/ServerCnxn.java | 94 ++++++-
.../test/org/apache/zookeeper/ZKTestCase.java | 4 +
.../test/FourLetterWordsWhiteListTest.java | 252 +++++++++++++++++++
6 files changed, 449 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/835377f0/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 5aefa9a..fb00fae 100644
--- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
+++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
@@ -1042,6 +1042,40 @@ server.3=zoo3:2888:3888</programlisting>
</note>
</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.4.10:</emphasis>
+ This property contains a list of comma separated
+ <ulink url="#sc_4lw">Four Letter Words</ulink> commands. It is
introduced
+ to provide fine grained control over the set of commands ZooKeeper can execute,
+ so users can turn off certain commands if necessary.
+ By default it contains all supported four letter word commands except "wchp"
and "wchc",
+ if the property is not specified. If the property is specified, then only
commands listed
+ in the whitelist are enabled.
+ </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>Users can also use asterisk option so they 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>
@@ -1667,6 +1701,16 @@ imok
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/835377f0/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 4ea7fb2..456d4c2 100644
--- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
+++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
@@ -825,18 +825,30 @@ public class NIOServerCnxn extends ServerCnxn {
}
}
+ private class NopCommand extends CommandThread {
+ private String msg;
+
+ public NopCommand(PrintWriter pw, String msg) {
+ super(pw);
+ this.msg = msg;
+ }
+
+ @Override
+ public void commandRun() {
+ pw.println(msg);
+ }
+ }
+
/** Return if four letter word found and responded to, otw false **/
private boolean checkFourLetterWord(final SelectionKey k, final int len)
throws IOException
{
// 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 = cmd2String.get(len);
- if (cmd == null) {
+ if (!ServerCnxn.isKnown(len)) {
return false;
}
- LOG.info("Processing " + cmd + " command from "
- + sock.socket().getRemoteSocketAddress());
+
packetReceived();
/** cancel the selection key to remove the socket handling
@@ -858,6 +870,19 @@ public class NIOServerCnxn extends ServerCnxn {
final PrintWriter pwriter = new PrintWriter(
new BufferedWriter(new SendBufferWriter()));
+
+ String cmd = ServerCnxn.getCommandString(len);
+ // ZOOKEEPER-2693: don't execute 4lw if it's not enabled.
+ if (!ServerCnxn.isEnabled(cmd)) {
+ LOG.debug("Command {} is not executed because it is not in the whitelist.", cmd);
+ NopCommand nopCmd = new NopCommand(pwriter, 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 == ruokCmd) {
RuokCommand ruok = new RuokCommand(pwriter);
ruok.start();
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/835377f0/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 32fc371..203f0e6 100644
--- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
+++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
@@ -618,23 +618,47 @@ public class NettyServerCnxn extends ServerCnxn {
}
}
+ private class NopCommand extends CommandThread {
+ private String msg;
+
+ public NopCommand(PrintWriter pw, String msg) {
+ super(pw);
+ this.msg = msg;
+ }
+
+ @Override
+ public void commandRun() {
+ pw.println(msg);
+ }
+ }
+
/** Return if four letter word found and responded to, otw false **/
private boolean checkFourLetterWord(final Channel channel,
ChannelBuffer message, final int len) throws IOException
{
// 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 = cmd2String.get(len);
- if (cmd == null) {
+ if (!ServerCnxn.isKnown(len)) {
return false;
}
+
channel.setInterestOps(0).awaitUninterruptibly();
- LOG.info("Processing " + cmd + " command from "
- + channel.getRemoteAddress());
packetReceived();
final PrintWriter pwriter = new PrintWriter(
new BufferedWriter(new SendBufferWriter()));
+
+ String cmd = ServerCnxn.getCommandString(len);
+ // ZOOKEEPER-2693: don't execute 4lw if it's not enabled.
+ if (!ServerCnxn.isEnabled(cmd)) {
+ LOG.debug("Command {} is not executed because it is not in the whitelist.", cmd);
+ NopCommand nopCmd = new NopCommand(pwriter, 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 == ruokCmd) {
RuokCommand ruok = new RuokCommand(pwriter);
ruok.start();
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/835377f0/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
index 6dd509b..6b93e93 100644
--- a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
+++ b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
@@ -26,10 +26,17 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
+import java.util.Map;
import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.jute.Record;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
@@ -227,8 +234,91 @@ public abstract class ServerCnxn implements Stats, Watcher {
protected final static int isroCmd = ByteBuffer.wrap("isro".getBytes())
.getInt();
- protected final static HashMap<Integer, String> cmd2String =
- new HashMap<Integer, String>();
+ protected final static Map<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(ServerCnxn.class);
+
+ 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());
+ }
+ }
+ } else {
+ for (Map.Entry<Integer, String> entry : cmd2String.entrySet()) {
+ String cmd = entry.getValue();
+ if (cmd.equals("wchc") || cmd.equals("wchp")) {
+ // ZOOKEEPER-2693 / disable these exploitable commands by default.
+ continue;
+ }
+ whiteListedCommands.add(cmd);
+ }
+ }
+
+ // Readonly mode depends on "isro".
+ 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
static {
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/835377f0/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 97e2db6..9098fc4 100644
--- a/src/java/test/org/apache/zookeeper/ZKTestCase.java
+++ b/src/java/test/org/apache/zookeeper/ZKTestCase.java
@@ -52,6 +52,10 @@ public class ZKTestCase {
@Override
public void starting(FrameworkMethod method) {
testName = method.getName();
+ // ZOOKEEPER-2693 disables all 4lw by default.
+ // Here we enable the 4lw which ZooKeeper tests depends.
+ System.setProperty("zookeeper.4lw.commands.whitelist", "*");
+
LOG.info("STARTING " + testName);
}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/835377f0/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..613346f
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsWhiteListTest.java
@@ -0,0 +1,252 @@
+/**
+ * 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.server.ServerCnxn;
+import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FourLetterWordsWhiteListTest extends ClientBase {
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(FourLetterWordsTest.class);
+
+ @Rule
+ public Timeout timeout = new Timeout(30000);
+
+ /*
+ * 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();
+ ServerCnxn.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();
+ ServerCnxn.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("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();
+ ServerCnxn.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();
+ ServerCnxn.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();
+ ServerCnxn.resetWhiteList();
+ System.setProperty("zookeeper.4lw.commands.whitelist", "*");
+ startServer();
+ verifyAllCommandsSuccess();
+ }
+
+ @Test(timeout=30000)
+ public void testFourLetterWordsEnableAllCommandsThroughExplicitList() throws Exception
{
+ stopServer();
+ ServerCnxn.resetWhiteList();
+ System.setProperty("zookeeper.4lw.commands.whitelist",
+ "ruok, envi, conf, stat, srvr, cons, dump," +
+ "wchs, wchp, wchc, srst, crst, " +
+ "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);
+
+ 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);
+ 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");
+ }
+
+ private void verifyAllCommandsFail() throws Exception {
+ verifyExactMatch("ruok", generateExpectedMessage("ruok"));
+ verifyExactMatch("conf", generateExpectedMessage("conf"));
+ verifyExactMatch("cons", generateExpectedMessage("cons"));
+ verifyExactMatch("crst", generateExpectedMessage("crst"));
+ 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 void verifyFuzzyMatch(String cmd, String expected) throws IOException {
+ 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 {
+ String resp = sendRequest(cmd);
+ LOG.info("cmd " + cmd + " expected an exact match of " + expected + "; got " + resp);
+ Assert.assertTrue(resp.trim().equals(expected));
+ }
+
+ private String sendRequest(String cmd) throws IOException {
+ HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0);
+ return send4LetterWord(hpobj.host, hpobj.port, cmd);
+ }
+
+ private String sendRequest(String cmd, int timeout) throws IOException {
+ HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0);
+ return send4LetterWord(hpobj.host, hpobj.port, cmd, timeout);
+ }
+}
|