You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by fp...@apache.org on 2016/11/13 19:59:49 UTC

[2/2] zookeeper git commit: ZOOKEEPER-2014: Only admin should be allowed to reconfig a cluster.

ZOOKEEPER-2014: Only admin should be allowed to reconfig a cluster.

This PR implements ZOOKEEPER-2014. For details, please refer to

JIRA: https://issues.apache.org/jira/browse/ZOOKEEPER-2014
Review board: https://reviews.apache.org/r/51546/

Author: Michael Han <ha...@cloudera.com>

Reviewers: fpj <fp...@apache.org>, breed <br...@apache.org>, rgs <rg...@itevenworks.net>

Closes #96 from hanm/ZOOKEEPER-2014


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

Branch: refs/heads/master
Commit: 73e102a58d01b27bc6208bbfbde2d12f0deba1f4
Parents: 881256e
Author: Michael Han <ha...@cloudera.com>
Authored: Sun Nov 13 11:59:29 2016 -0800
Committer: fpj <fp...@apache.org>
Committed: Sun Nov 13 11:59:29 2016 -0800

----------------------------------------------------------------------
 build.xml                                       |   1 +
 src/c/include/zookeeper.h                       |   3 +-
 src/c/tests/TestReconfigServer.cc               | 131 ++++++++--
 src/c/tests/ZooKeeperQuorumServer.cc            |  46 +++-
 src/c/tests/ZooKeeperQuorumServer.h             |  11 +-
 .../content/xdocs/zookeeperAdmin.xml            |  37 +++
 .../content/xdocs/zookeeperReconfig.xml         | 129 ++++++++++
 .../main/org/apache/zookeeper/ClientCnxn.java   |   4 +-
 .../org/apache/zookeeper/KeeperException.java   |  20 +-
 .../main/org/apache/zookeeper/ZooKeeper.java    |  95 +------
 .../org/apache/zookeeper/ZooKeeperMain.java     |   8 +-
 .../apache/zookeeper/admin/ZooKeeperAdmin.java  | 250 +++++++++++++++++++
 .../org/apache/zookeeper/cli/CliCommand.java    |   3 +-
 .../apache/zookeeper/cli/ReconfigCommand.java   |  14 +-
 .../org/apache/zookeeper/server/DataTree.java   |  26 +-
 .../zookeeper/server/PrepRequestProcessor.java  |   9 +
 .../zookeeper/server/ZooKeeperServer.java       |   3 +-
 .../zookeeper/server/ZooKeeperServerMain.java   |   3 +-
 .../server/quorum/QuorumPeerConfig.java         |  19 +-
 .../zookeeper/server/quorum/QuorumPeerMain.java |   4 +-
 .../zookeeper/test/system/BaseSysTest.java      |   1 -
 .../org/apache/zookeeper/TestableZooKeeper.java |   3 +-
 .../apache/zookeeper/server/DataTreeTest.java   |  45 ++--
 .../zookeeper/server/quorum/LearnerTest.java    |   3 +-
 .../server/quorum/RaceConditionTest.java        |   6 +-
 .../server/quorum/ReconfigBackupTest.java       |  14 +-
 .../quorum/ReconfigDuringLeaderSyncTest.java    |  20 +-
 .../server/quorum/ReconfigFailureCasesTest.java |  43 +++-
 .../server/quorum/ReconfigLegacyTest.java       |  11 +-
 .../server/quorum/StandaloneDisabledTest.java   |  17 +-
 .../zookeeper/server/quorum/Zab1_0Test.java     |   4 +-
 .../test/org/apache/zookeeper/test/ACLTest.java |   9 +-
 .../zookeeper/test/ReconfigExceptionTest.java   | 220 ++++++++++++++++
 .../zookeeper/test/ReconfigMisconfigTest.java   | 130 ++++++++++
 .../org/apache/zookeeper/test/ReconfigTest.java | 151 ++++++++---
 .../apache/zookeeper/test/StandaloneTest.java   |  24 +-
 36 files changed, 1273 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 4bb5450..aacf55a 100644
--- a/build.xml
+++ b/build.xml
@@ -528,6 +528,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
           <include name="org/apache/zookeeper/WatchedEvent.java"/>
           <include name="org/apache/zookeeper/ZooDefs.java"/>
           <include name="org/apache/zookeeper/ZooKeeper.java"/>
+          <include name="org/apache/zookeeper/admin/ZooKeeperAdmin.java"/>
           <include name="org/apache/zookeeper/server/LogFormatter.java"/>
           <include name="org/apache/zookeeper/server/SnapshotFormatter.java"/>
           <include name="org/apache/zookeeper/server/PurgeTxnLog.java"/>

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/c/include/zookeeper.h
----------------------------------------------------------------------
diff --git a/src/c/include/zookeeper.h b/src/c/include/zookeeper.h
index 18a203d..ec19e5a 100644
--- a/src/c/include/zookeeper.h
+++ b/src/c/include/zookeeper.h
@@ -124,7 +124,8 @@ enum ZOO_ERRORS {
   ZNOTREADONLY = -119, /*!< state-changing request is passed to read-only server */
   ZEPHEMERALONLOCALSESSION = -120, /*!< Attempt to create ephemeral node on a local session */
   ZNOWATCHER = -121, /*!< The watcher couldn't be found */
-  ZRWSERVERFOUND = -122 /*!< r/w server found while in r/o mode */
+  ZRWSERVERFOUND = -122, /*!< r/w server found while in r/o mode */
+  ZRECONFIGDISABLED = -123 /*!< Attempts to perform a reconfiguration operation when reconfiguration feature is disabled */
 };
 
 #ifdef __cplusplus

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/c/tests/TestReconfigServer.cc
----------------------------------------------------------------------
diff --git a/src/c/tests/TestReconfigServer.cc b/src/c/tests/TestReconfigServer.cc
index 6a429ac..b86b33d 100644
--- a/src/c/tests/TestReconfigServer.cc
+++ b/src/c/tests/TestReconfigServer.cc
@@ -15,6 +15,9 @@
  * the License.
  */
 #include <algorithm>
+#include <sstream>
+#include <vector>
+#include <utility>
 #include <cppunit/extensions/HelperMacros.h>
 #include <unistd.h>
 #include "zookeeper.h"
@@ -28,6 +31,8 @@ class TestReconfigServer : public CPPUNIT_NS::TestFixture {
     CPPUNIT_TEST(testNonIncremental);
     CPPUNIT_TEST(testRemoveConnectedFollower);
     CPPUNIT_TEST(testRemoveFollower);
+    CPPUNIT_TEST(testReconfigFailureWithoutAuth);
+    CPPUNIT_TEST(testReconfigFailureWithoutServerSuperuserPasswordConfigured);
 #endif
     CPPUNIT_TEST_SUITE_END();
 
@@ -39,7 +44,8 @@ class TestReconfigServer : public CPPUNIT_NS::TestFixture {
     void testNonIncremental();
     void testRemoveConnectedFollower();
     void testRemoveFollower();
-
+    void testReconfigFailureWithoutAuth();
+    void testReconfigFailureWithoutServerSuperuserPasswordConfigured();
   private:
     static const uint32_t NUM_SERVERS;
     FILE* logfile_;
@@ -49,6 +55,7 @@ class TestReconfigServer : public CPPUNIT_NS::TestFixture {
     void parseConfig(char* buf, int len, std::vector<std::string>& servers,
                      std::string& version);
     bool waitForConnected(zhandle_t* zh, uint32_t timeout_sec);
+    zhandle_t* connectFollowers(std::vector<int32_t> &followers);
 };
 
 const uint32_t TestReconfigServer::NUM_SERVERS = 3;
@@ -70,7 +77,10 @@ TestReconfigServer::
 
 void TestReconfigServer::
 setUp() {
-    cluster_ = ZooKeeperQuorumServer::getCluster(NUM_SERVERS);
+    ZooKeeperQuorumServer::tConfigPairs configs;
+    configs.push_back(std::make_pair("reconfigEnabled", "true"));
+    cluster_ = ZooKeeperQuorumServer::getCluster(NUM_SERVERS, configs,
+        "SERVER_JVMFLAGS=-Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is test */);
 }
 
 void TestReconfigServer::
@@ -151,7 +161,7 @@ testRemoveFollower() {
     zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0);
     CPPUNIT_ASSERT_EQUAL(true, waitForConnected(zk, 10));
     CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat));
-
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK));
     // check if all the servers are listed in the config.
     parseConfig(buf, len, servers, version);
     // initially should be 1<<32, which is 0x100000000. This is the zxid
@@ -219,6 +229,7 @@ testNonIncremental() {
     zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0);
     CPPUNIT_ASSERT_EQUAL(true, waitForConnected(zk, 10));
     CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat));
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK));
 
     // check if all the servers are listed in the config.
     parseConfig(buf, len, servers, version);
@@ -274,37 +285,46 @@ testNonIncremental() {
     zookeeper_close(zk);
 }
 
-/**
- * 1. Connect to a follower.
- * 2. Remove the follower the client is connected to.
- */
-void TestReconfigServer::
-testRemoveConnectedFollower() {
-    std::vector<std::string> servers;
-    std::string version;
-    struct Stat stat;
-    int len = 1024;
-    char buf[len];
-
-    // connect to a follower.
+zhandle_t* TestReconfigServer::
+connectFollowers(std::vector<int32_t> &followers) {
+    std::stringstream ss;
     int32_t leader = getLeader();
-    std::vector<int32_t> followers = getFollowers();
     CPPUNIT_ASSERT(leader >= 0);
     CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(followers.size()));
-    std::stringstream ss;
     for (int i = 0; i < followers.size(); i++) {
-      ss << cluster_[followers[i]]->getHostPort() << ",";
+        ss << cluster_[followers[i]]->getHostPort() << ",";
     }
     ss << cluster_[leader]->getHostPort();
     std::string hosts = ss.str().c_str();
     zoo_deterministic_conn_order(true);
     zhandle_t* zk = zookeeper_init(hosts.c_str(), NULL, 10000, NULL, NULL, 0);
     CPPUNIT_ASSERT_EQUAL(true, waitForConnected(zk, 10));
+
     std::string connectedHost(zoo_get_current_server(zk));
     std::string portString = connectedHost.substr(connectedHost.find(":") + 1);
     uint32_t port;
     std::istringstream (portString) >> port;
     CPPUNIT_ASSERT_EQUAL(cluster_[followers[0]]->getClientPort(), port);
+    return zk;
+}
+
+/**
+ * 1. Connect to a follower.
+ * 2. Remove the follower the client is connected to.
+ */
+void TestReconfigServer::
+testRemoveConnectedFollower() {
+    std::vector<std::string> servers;
+    std::string version;
+    struct Stat stat;
+    int len = 1024;
+    char buf[len];
+
+    // connect to a follower.
+    std::stringstream ss;
+    std::vector<int32_t> followers = getFollowers();
+    zhandle_t* zk = connectFollowers(followers);
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK));
 
     // remove the follower.
     len = 1024;
@@ -324,4 +344,77 @@ testRemoveConnectedFollower() {
     zookeeper_close(zk);
 }
 
+/**
+ * ZOOKEEPER-2014: only admin or users who are explicitly granted permission can do reconfig.
+ */
+void TestReconfigServer::
+testReconfigFailureWithoutAuth() {
+    std::vector<std::string> servers;
+    std::string version;
+    struct Stat stat;
+    int len = 1024;
+    char buf[len];
+
+    // connect to a follower.
+    std::stringstream ss;
+    std::vector<int32_t> followers = getFollowers();
+    zhandle_t* zk = connectFollowers(followers);
+
+    // remove the follower.
+    len = 1024;
+    ss.str("");
+    ss << followers[0];
+    // No auth, should fail.
+    CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    // Wrong auth, should fail.
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:wrong", 11, NULL,(void*)ZOK));
+    CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    // Right auth, should pass.
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK));
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat));
+    parseConfig(buf, len, servers, version);
+    CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size()));
+    for (int i = 0; i < cluster_.size(); i++) {
+        if (i == followers[0]) {
+            continue;
+        }
+        CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(),
+                       cluster_[i]->getServerString()) != servers.end());
+    }
+    zookeeper_close(zk);
+}
+
+void TestReconfigServer::
+testReconfigFailureWithoutServerSuperuserPasswordConfigured() {
+    std::vector<std::string> servers;
+    std::string version;
+    struct Stat stat;
+    int len = 1024;
+    char buf[len];
+
+    // Create a new quorum with the super user's password not configured.
+    tearDown();
+    ZooKeeperQuorumServer::tConfigPairs configs;
+    configs.push_back(std::make_pair("reconfigEnabled", "true"));
+    cluster_ = ZooKeeperQuorumServer::getCluster(NUM_SERVERS, configs, "");
+
+    // connect to a follower.
+    std::stringstream ss;
+    std::vector<int32_t> followers = getFollowers();
+    zhandle_t* zk = connectFollowers(followers);
+
+    // remove the follower.
+    len = 1024;
+    ss.str("");
+    ss << followers[0];
+    // All cases should fail as server ensemble was not configured with the super user's password.
+    CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:", 11, NULL,(void*)ZOK));
+    CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK));
+    CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat));
+    zookeeper_close(zk);
+}
+
 CPPUNIT_TEST_SUITE_REGISTRATION(TestReconfigServer);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/c/tests/ZooKeeperQuorumServer.cc
----------------------------------------------------------------------
diff --git a/src/c/tests/ZooKeeperQuorumServer.cc b/src/c/tests/ZooKeeperQuorumServer.cc
index 23392cd..c38e385 100644
--- a/src/c/tests/ZooKeeperQuorumServer.cc
+++ b/src/c/tests/ZooKeeperQuorumServer.cc
@@ -21,18 +21,21 @@
 #include <cstdlib>
 #include <fstream>
 #include <sstream>
+#include <vector>
+#include <utility>
 #include <unistd.h>
 
 ZooKeeperQuorumServer::
-ZooKeeperQuorumServer(uint32_t id, uint32_t numServers) :
+ZooKeeperQuorumServer(uint32_t id, uint32_t numServers, std::string config, std::string env) :
     id_(id),
+    env_(env),
     numServers_(numServers) {
     const char* root = getenv("ZKROOT");
     if (root == NULL) {
         assert(!"Environment variable 'ZKROOT' is not set");
     }
     root_ = root;
-    createConfigFile();
+    createConfigFile(config);
     createDataDirectory();
     start();
 }
@@ -58,6 +61,9 @@ void ZooKeeperQuorumServer::
 start() {
     std::string command = root_ + "/bin/zkServer.sh start " +
                           getConfigFileName();
+    if (!env_.empty()) {
+        command = env_ + " " + command;
+    }
     assert(system(command.c_str()) == 0);
 }
 
@@ -102,7 +108,7 @@ isFollower() {
 }
 
 void ZooKeeperQuorumServer::
-createConfigFile() {
+createConfigFile(std::string config) {
     std::string command = "mkdir -p " + root_ + "/build/test/test-cppunit/conf";
     assert(system(command.c_str()) == 0);
     std::ofstream confFile;
@@ -118,6 +124,10 @@ createConfigFile() {
     for (int i = 0; i < numServers_; i++) {
         confFile << getServerString(i) << "\n";
     }
+    // Append additional config, if any.
+    if (!config.empty()) {
+      confFile << config << std::endl;
+    }
     confFile.close();
 }
 
@@ -188,3 +198,33 @@ getCluster(uint32_t numServers) {
     }
     assert(!"The cluster didn't start for 10 seconds");
 }
+
+std::vector<ZooKeeperQuorumServer*> ZooKeeperQuorumServer::
+getCluster(uint32_t numServers, ZooKeeperQuorumServer::tConfigPairs configs, std::string env) {
+    std::vector<ZooKeeperQuorumServer*> cluster;
+    std::string config;
+    for (ZooKeeperQuorumServer::tConfigPairs::const_iterator iter = configs.begin(); iter != configs.end(); ++iter) {
+        std::pair<std::string, std::string> pair = *iter;
+        config += (pair.first + "=" + pair.second + "\n");
+    }
+    for (int i = 0; i < numServers; i++) {
+        cluster.push_back(new ZooKeeperQuorumServer(i, numServers, config, env));
+    }
+
+    // Wait until all the servers start, and fail if they don't start within 10
+    // seconds.
+    for (int i = 0; i < 10; i++) {
+        int j = 0;
+        for (; j < cluster.size(); j++) {
+            if (cluster[j]->getMode() == "") {
+                // The server hasn't started.
+                sleep(1);
+                break;
+            }
+        }
+        if (j == cluster.size()) {
+            return cluster;
+        }
+    }
+    assert(!"The cluster didn't start for 10 seconds");
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/c/tests/ZooKeeperQuorumServer.h
----------------------------------------------------------------------
diff --git a/src/c/tests/ZooKeeperQuorumServer.h b/src/c/tests/ZooKeeperQuorumServer.h
index aa8b7cc..577072e 100644
--- a/src/c/tests/ZooKeeperQuorumServer.h
+++ b/src/c/tests/ZooKeeperQuorumServer.h
@@ -20,11 +20,16 @@
 #include <stdint.h>
 #include <string>
 #include <vector>
+#include <utility>
 
 class ZooKeeperQuorumServer {
   public:
     ~ZooKeeperQuorumServer();
+    typedef std::vector<std::pair<std::string, std::string> > tConfigPairs;
     static std::vector<ZooKeeperQuorumServer*> getCluster(uint32_t numServers);
+    static std::vector<ZooKeeperQuorumServer*> getCluster(uint32_t numServers,
+        tConfigPairs configs, /* Additional config options as a list of key/value pairs. */
+        std::string env       /* Additional environment variables when starting zkServer.sh. */);
     std::string getHostPort();
     uint32_t getClientPort();
     void start();
@@ -35,10 +40,11 @@ class ZooKeeperQuorumServer {
 
   private:
     ZooKeeperQuorumServer();
-    ZooKeeperQuorumServer(uint32_t id, uint32_t numServers);
+    ZooKeeperQuorumServer(uint32_t id, uint32_t numServers, std::string config = "",
+                          std::string env = "");
     ZooKeeperQuorumServer(const ZooKeeperQuorumServer& that);
     const ZooKeeperQuorumServer& operator=(const ZooKeeperQuorumServer& that);
-    void createConfigFile();
+    void createConfigFile(std::string config = "");
     std::string getConfigFileName();
     void createDataDirectory();
     std::string getDataDirectory();
@@ -52,6 +58,7 @@ class ZooKeeperQuorumServer {
     uint32_t numServers_;
     uint32_t id_;
     std::string root_;
+    std::string env_;
 };
 
 #endif  // ZOOKEEPER_QUORUM_SERVER_H

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/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 5a30da8..6302814 100644
--- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
+++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
@@ -931,6 +931,7 @@ server.3=zoo3:2888:3888</programlisting>
               feature. Default is "true"</para>
             </listitem>
           </varlistentry>
+
         </variablelist>
       </section>
 
@@ -1108,6 +1109,42 @@ server.3=zoo3:2888:3888</programlisting>
               </para>
             </listitem>
           </varlistentry>
+
+          <varlistentry>
+            <term>reconfigEnabled</term>
+
+            <listitem>
+              <para>(No Java system property)</para>
+
+              <para><emphasis role="bold">New in 3.5.3:</emphasis>
+                This controls the enabling or disabling of
+                <ulink url="zookeeperReconfig.html">
+                  Dynamic Reconfiguration</ulink> feature. When the feature
+                is enabled, users can perform reconfigure operations through
+                the ZooKeeper client API or through ZooKeeper command line tools
+                assuming users are authorized to perform such operations.
+                When the feature is disabled, no user, including the super user,
+                can perform a reconfiguration. Any attempt to reconfigure will return an error.
+                <emphasis role="bold">"reconfigEnabled"</emphasis> option can be set as
+                <emphasis role="bold">"reconfigEnabled=false"</emphasis> or
+                <emphasis role="bold">"reconfigEnabled=true"</emphasis>
+                to a server's config file, or using QuorumPeerConfig's
+                setReconfigEnabled method. The default value is false.
+
+                If present, the value should be consistent across every server in
+                the entire ensemble. Setting the value as true on some servers and false
+                on other servers will cause inconsistent behavior depending on which server
+                is elected as leader. If the leader has a setting of
+                <emphasis role="bold">"reconfigEnabled=true"</emphasis>, then the ensemble
+                will have reconfig feature enabled. If the leader has a setting of
+                <emphasis role="bold">"reconfigEnabled=false"</emphasis>, then the ensemble
+                will have reconfig feature disabled. It is thus recommended to have a consistent
+                value for <emphasis role="bold">"reconfigEnabled"</emphasis> across servers
+                in the ensemble.
+              </para>
+            </listitem>
+          </varlistentry>
+
         </variablelist>
         <para></para>
       </section>

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml
----------------------------------------------------------------------
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml b/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml
index 7168a01..c1c9ad7 100644
--- a/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml
+++ b/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml
@@ -83,6 +83,11 @@
         </listitem>
       </varlistentry>
     </variablelist>
+    <para><emphasis role="bold">Note:</emphasis> Starting with 3.5.3, the dynamic reconfiguration
+      feature is disabled by default, and has to be explicitly turned on via
+      <ulink url="zookeeperAdmin.html#sc_advancedConfiguration">
+        reconfigEnabled </ulink> configuration option.
+    </para>
   </section>
   <section id="ch_reconfig_format">
     <title>Changes to Configuration Format</title>
@@ -142,6 +147,25 @@
         recommend setting the flag to <emphasis>false</emphasis>. We expect that
         the legacy Standalone mode will be deprecated in the future.</para>
     </section>
+    <section id="sc_reconfig_reconfigEnabled">
+      <title>The <emphasis>reconfigEnabled</emphasis> flag</title>
+      <para>Starting with 3.5.0 and prior to 3.5.3, there is no way to disable
+        dynamic reconfiguration feature. We would like to offer the option of
+        disabling reconfiguration feature because with reconfiguration enabled,
+        we have a security concern that a malicious actor can make arbitrary changes
+        to the configuration of a ZooKeeper ensemble, including adding a compromised
+        server to the ensemble. We prefer to leave to the discretion of the user to
+        decide whether to enable it or not and make sure that the appropriate security
+        measure are in place. So in 3.5.3 the <ulink url="zookeeperAdmin.html#sc_advancedConfiguration">
+          reconfigEnabled </ulink> configuration option is introduced
+        such that the reconfiguration feature can be completely disabled and any attempts
+        to reconfigure a cluster through reconfig API with or without authentication
+        will fail by default, unless <emphasis role="bold">reconfigEnabled</emphasis> is set to
+        <emphasis role="bold">true</emphasis>.
+      </para>
+      <para>To set the option to true, the configuration file (zoo.cfg) should contain:</para>
+      <para><computeroutput>reconfigEnabled=true</computeroutput></para>
+    </section>
     <section id="sc_reconfig_file">
       <title>Dynamic configuration file</title>
       <para>Starting with 3.5.0 we're distinguishing between dynamic
@@ -252,6 +276,7 @@ server.3=125.23.63.25:2782:2785:participant</programlisting>
       clientPort/clientPortAddress statements (although if you specify client
       ports in the new format, these statements are now redundant).</para>
   </section>
+
   <section id="ch_reconfig_dyn">
     <title>Dynamic Reconfiguration of the ZooKeeper Ensemble</title>
     <para>The ZooKeeper Java and C API were extended with getConfig and reconfig
@@ -260,6 +285,110 @@ server.3=125.23.63.25:2782:2785:participant</programlisting>
       here using the Java CLI, but note that you can similarly use the C CLI or
       invoke the commands directly from a program just like any other ZooKeeper
       command.</para>
+
+    <section id="ch_reconfig_api">
+      <title>API</title>
+      <para>There are two sets of APIs for both Java and C client.
+      </para>
+      <variablelist>
+        <varlistentry>
+          <term><emphasis role="bold">Reconfiguration API</emphasis></term>
+
+          <listitem>
+            <para>Reconfiguration API is used to reconfigure the ZooKeeper cluster.
+              Starting with 3.5.3, reconfiguration Java APIs are moved into ZooKeeperAdmin class
+              from ZooKeeper class, and use of this API requires ACL setup and user
+              authentication (see <xref linkend="sc_reconfig_access_control"/> for more information.).
+            </para>
+          </listitem>
+        </varlistentry>
+
+        <varlistentry>
+          <term><emphasis role="bold">Get Configuration API</emphasis></term>
+          <listitem>
+            <para>Get configuration APIs are used to retrieve ZooKeeper cluster configuration information
+              stored in /zookeeper/config znode. Use of this API does not require specific setup or authentication,
+            because /zookeeper/config is readable to any users.</para>
+          </listitem>
+        </varlistentry>
+      </variablelist>
+    </section>
+
+    <section id="sc_reconfig_access_control">
+      <title>Security</title>
+      <para>Prior to <emphasis role="bold">3.5.3</emphasis>, there is no enforced security mechanism
+        over reconfig so any ZooKeeper clients that can connect to ZooKeeper server ensemble
+        will have the ability to change the state of a ZooKeeper cluster via reconfig.
+        It is thus possible for a malicious client to add compromised server to an ensemble,
+        e.g., add a compromised server, or remove legitimate servers.
+        Cases like these could be security vulnerabilities on a case by case basis.
+      </para>
+      <para>To address this security concern, we introduced access control over reconfig
+        starting from <emphasis role="bold">3.5.3</emphasis> such that only a specific set of users
+        can use reconfig commands or APIs, and these users need be configured explicitly. In addition,
+        the setup of ZooKeeper cluster must enable authentication so ZooKeeper clients can be authenticated.
+      </para>
+      <para>
+        We also provides an escape hatch for users who operate and interact with a ZooKeeper ensemble in a secured
+        environment (i.e. behind company firewall). For those users who want to use reconfiguration feature but
+        don't want the overhead of configuring an explicit list of authorized user for reconfig access checks,
+        they can set <ulink url="zookeeperAdmin.html#sc_authOptions">"skipACL"</ulink> to "yes" which will
+        skip ACL check and allow any user to reconfigure cluster.
+      </para>
+      <para>
+        Overall, ZooKeeper provides flexible configuration options for the reconfigure feature
+        that allow a user to choose based on user's security requirement.
+        We leave to the discretion of the user to decide appropriate security measure are in place.
+      </para>
+      <variablelist>
+        <varlistentry>
+          <term><emphasis role="bold">Access Control</emphasis></term>
+
+          <listitem>
+            <para>The dynamic configuration is stored in a special znode
+              ZooDefs.CONFIG_NODE = /zookeeper/config. This node by default is read only
+              for all users, except super user and users that's explicitly configured for write
+              access.
+            </para>
+
+            <para>Clients that need to use reconfig commands or reconfig API should be configured as users
+              that have write access to CONFIG_NODE. By default, only the super user has full control including
+              write access to CONFIG_NODE. Additional users can be granted write access through superuser
+              by setting an ACL that has write permission associated with specified user.
+            </para>
+
+            <para>A few examples of how to setup ACLs and use reconfiguration API with authentication can be found in
+              ReconfigExceptionTest.java and TestReconfigServer.cc.</para>
+          </listitem>
+        </varlistentry>
+
+        <varlistentry>
+          <term><emphasis role="bold">Authentication</emphasis></term>
+
+          <listitem>
+            <para>Authentication of users is orthogonal to the access control and is delegated to
+              existing authentication mechanism supported by ZooKeeper's pluggable authentication schemes.
+              See <ulink
+                      url="https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL"
+              >ZooKeeper and SASL</ulink> for more details on this topic.
+            </para>
+          </listitem>
+        </varlistentry>
+
+        <varlistentry>
+          <term><emphasis role="bold">Disable ACL check</emphasis></term>
+          <listitem>
+            <para>
+              ZooKeeper supports <ulink
+                    url="zookeeperAdmin.html#sc_authOptions">"skipACL"</ulink> option such that ACL
+              check will be completely skipped, if skipACL is set to "yes". In such cases any unauthenticated
+              users can use reconfig API.
+            </para>
+          </listitem>
+        </varlistentry>
+      </variablelist>
+    </section>
+
     <section id="sc_reconfig_retrieving">
       <title>Retrieving the current dynamic configuration</title>
       <para>The dynamic configuration is stored in a special znode

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/ClientCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java
index 12dd51c..9e6c154 100644
--- a/src/java/main/org/apache/zookeeper/ClientCnxn.java
+++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java
@@ -1523,14 +1523,14 @@ public class ClientCnxn {
         sendThread.sendPacket(p);
     }
 
-    Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
+    public Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
             Record response, AsyncCallback cb, String clientPath,
             String serverPath, Object ctx, WatchRegistration watchRegistration) {
         return queuePacket(h, r, request, response, cb, clientPath, serverPath,
                 ctx, watchRegistration, null);
     }
 
-    Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
+    public Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
             Record response, AsyncCallback cb, String clientPath,
             String serverPath, Object ctx, WatchRegistration watchRegistration,
             WatchDeregistration watchDeregistration) {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/KeeperException.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/KeeperException.java b/src/java/main/org/apache/zookeeper/KeeperException.java
index a05f1ab..ea2b395 100644
--- a/src/java/main/org/apache/zookeeper/KeeperException.java
+++ b/src/java/main/org/apache/zookeeper/KeeperException.java
@@ -139,6 +139,8 @@ public abstract class KeeperException extends Exception {
                 return new EphemeralOnLocalSessionException();
             case NOWATCHER:
                 return new NoWatcherException();
+            case RECONFIGDISABLED:
+                return new ReconfigDisabledException();
             case OK:
             default:
                 throw new IllegalArgumentException("Invalid exception code");
@@ -384,7 +386,9 @@ public abstract class KeeperException extends Exception {
         /** Attempt to create ephemeral node on a local session */
         EPHEMERALONLOCALSESSION (EphemeralOnLocalSession),
         /** Attempts to remove a non-existing watcher */
-        NOWATCHER (-121);
+        NOWATCHER (-121),
+        /** Attempts to perform a reconfiguration operation when reconfiguration feature is disabled. */
+        RECONFIGDISABLED(-123);
 
         private static final Map<Integer,Code> lookup
             = new HashMap<Integer,Code>();
@@ -469,6 +473,8 @@ public abstract class KeeperException extends Exception {
                 return "Ephemeral node on local session";
             case NOWATCHER:
                 return "No such watcher";
+            case RECONFIGDISABLED:
+                return "Reconfig is disabled";
             default:
                 return "Unknown error " + code;
         }
@@ -515,7 +521,7 @@ public abstract class KeeperException extends Exception {
 
     @Override
     public String getMessage() {
-        if (path == null) {
+        if (path == null || path.isEmpty()) {
             return "KeeperErrorCode = " + getCodeMessage(code);
         }
         return "KeeperErrorCode = " + getCodeMessage(code) + " for " + path;
@@ -795,4 +801,14 @@ public abstract class KeeperException extends Exception {
             super(Code.NOWATCHER, path);
         }
     }
+
+    /**
+     * @see Code#RECONFIGDISABLED
+     */
+    public static class ReconfigDisabledException extends KeeperException {
+        public ReconfigDisabledException() { super(Code.RECONFIGDISABLED); }
+        public ReconfigDisabledException(String path) {
+            super(Code.RECONFIGDISABLED, path);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/ZooKeeper.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java
index 1c9ed4e..cf274a4 100644
--- a/src/java/main/org/apache/zookeeper/ZooKeeper.java
+++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java
@@ -39,7 +39,6 @@ import org.apache.zookeeper.client.StaticHostProvider;
 import org.apache.zookeeper.client.ZKClientConfig;
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.common.StringUtils;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.CheckWatchesRequest;
@@ -57,7 +56,6 @@ import org.apache.zookeeper.proto.GetChildrenRequest;
 import org.apache.zookeeper.proto.GetChildrenResponse;
 import org.apache.zookeeper.proto.GetDataRequest;
 import org.apache.zookeeper.proto.GetDataResponse;
-import org.apache.zookeeper.proto.ReconfigRequest;
 import org.apache.zookeeper.proto.RemoveWatchesRequest;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.RequestHeader;
@@ -156,7 +154,7 @@ public class ZooKeeper {
         Environment.logEnv("Client environment:", LOG);
     }
 
-    private final HostProvider hostProvider;
+    protected final HostProvider hostProvider;
 
     /**
      * This function allows a client to update the connection string by providing 
@@ -215,7 +213,7 @@ public class ZooKeeper {
         return cnxn.zooKeeperSaslClient;
     }
 
-    private final ZKWatchManager watchManager;
+    protected final ZKWatchManager watchManager;
 
     private final ZKClientConfig clientConfig;
 
@@ -223,19 +221,19 @@ public class ZooKeeper {
         return clientConfig;
     }
 
-    List<String> getDataWatches() {
+    protected List<String> getDataWatches() {
         synchronized(watchManager.dataWatches) {
             List<String> rc = new ArrayList<String>(watchManager.dataWatches.keySet());
             return rc;
         }
     }
-    List<String> getExistWatches() {
+    protected List<String> getExistWatches() {
         synchronized(watchManager.existWatches) {
             List<String> rc =  new ArrayList<String>(watchManager.existWatches.keySet());
             return rc;
         }
     }
-    List<String> getChildWatches() {
+    protected List<String> getChildWatches() {
         synchronized(watchManager.childWatches) {
             List<String> rc = new ArrayList<String>(watchManager.childWatches.keySet());
             return rc;
@@ -262,7 +260,7 @@ public class ZooKeeper {
             this.disableAutoWatchReset = disableAutoWatchReset;
         }
 
-        private volatile Watcher defaultWatcher;
+        protected volatile Watcher defaultWatcher;
 
         final private void addTo(Set<Watcher> from, Set<Watcher> to) {
             if (from != null) {
@@ -529,7 +527,7 @@ public class ZooKeeper {
     /**
      * Register a watcher for a particular path.
      */
-    abstract class WatchRegistration {
+    public abstract class WatchRegistration {
         private Watcher watcher;
         private String clientPath;
         public WatchRegistration(Watcher watcher, String clientPath)
@@ -2177,85 +2175,6 @@ public class ZooKeeper {
     public void getConfig(boolean watch, DataCallback cb, Object ctx) {
         getConfig(watch ? watchManager.defaultWatcher : null, cb, ctx);
     }
-    
-    /**
-     * Reconfigure - add/remove servers. Return the new configuration.
-     * @param joiningServers
-     *                a comma separated list of servers being added (incremental reconfiguration)
-     * @param leavingServers
-     *                a comma separated list of servers being removed (incremental reconfiguration)
-     * @param newMembers
-     *                a comma separated list of new membership (non-incremental reconfiguration)
-     * @param fromConfig
-     *                version of the current configuration (optional - causes reconfiguration to throw an exception if configuration is no longer current)
-     * @param stat the stat of /zookeeper/config znode will be copied to this
-     *             parameter if not null.
-     * @return new configuration
-     * @throws InterruptedException If the server transaction is interrupted.
-     * @throws KeeperException If the server signals an error with a non-zero error code.     
-     */
-    public byte[] reconfig(String joiningServers, String leavingServers, String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException
-    {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.reconfig);       
-        ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);        
-        GetDataResponse response = new GetDataResponse();       
-        ReplyHeader r = cnxn.submitRequest(h, request, response, null);
-        if (r.getErr() != 0) {
-            throw KeeperException.create(KeeperException.Code.get(r.getErr()), "");
-        }
-        if (stat != null) {
-            DataTree.copyStat(response.getStat(), stat);
-        }
-        return response.getData();
-    }
-
-    /**
-     * Convenience wrapper around reconfig that takes Lists of strings instead of comma-separated servers.
-     *
-     * @see #reconfig
-     *
-     */
-    public byte[] reconfig(List<String> joiningServers, List<String> leavingServers, List<String> newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException
-    {
-        return reconfig(StringUtils.joinStrings(joiningServers, ","), 
-        		StringUtils.joinStrings(leavingServers, ","), 
-        		StringUtils.joinStrings(newMembers, ","), 
-        		fromConfig, stat);
-    }
-
-    /**
-     * The Asynchronous version of reconfig. 
-     *
-     * @see #reconfig
-     *      
-     **/
-    public void reconfig(String joiningServers, String leavingServers,
-        String newMembers, long fromConfig, DataCallback cb, Object ctx)
-    {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.reconfig);       
-        ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
-        GetDataResponse response = new GetDataResponse();
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
-               ZooDefs.CONFIG_NODE, ZooDefs.CONFIG_NODE, ctx, null);
-    }
- 
-    /**
-     * Convenience wrapper around asynchronous reconfig that takes Lists of strings instead of comma-separated servers.
-     *
-     * @see #reconfig
-     *
-     */
-    public void reconfig(List<String> joiningServers,
-        List<String> leavingServers, List<String> newMembers, long fromConfig,
-        DataCallback cb, Object ctx)
-    {
-        reconfig(StringUtils.joinStrings(joiningServers, ","), 
-        		StringUtils.joinStrings(leavingServers, ","), 
-        		StringUtils.joinStrings(newMembers, ","), 
-        		fromConfig, cb, ctx);
-    }
    
     /**
      * Set the data for the node of the given path if such a node exists and the

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
index c39395a..a72c5bf 100644
--- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
+++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
@@ -66,6 +66,7 @@ import org.apache.zookeeper.cli.SetQuotaCommand;
 import org.apache.zookeeper.cli.StatCommand;
 import org.apache.zookeeper.cli.SyncCommand;
 import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 
 /**
  * The command line client to ZooKeeper.
@@ -275,15 +276,14 @@ public class ZooKeeperMain {
         if (zk != null && zk.getState().isAlive()) {
             zk.close();
         }
+
         host = newHost;
         boolean readOnly = cl.getOption("readonly") != null;
         if (cl.getOption("secure") != null) {
             System.setProperty(ZKClientConfig.SECURE_CLIENT, "true");
             System.out.println("Secure connection is enabled");
         }
-        zk = new ZooKeeper(host,
-                 Integer.parseInt(cl.getOption("timeout")),
-                 new MyWatcher(), readOnly);
+        zk = new ZooKeeperAdmin(host, Integer.parseInt(cl.getOption("timeout")), new MyWatcher(), readOnly);
     }
     
     public static void main(String args[]) throws CliException, IOException, InterruptedException
@@ -296,8 +296,6 @@ public class ZooKeeperMain {
         cl.parseOptions(args);
         System.out.println("Connecting to " + cl.getOption("server"));
         connectToZK(cl.getOption("server"));
-        //zk = new ZooKeeper(cl.getOption("server"),
-//                Integer.parseInt(cl.getOption("timeout")), new MyWatcher());
     }
 
     public ZooKeeperMain(ZooKeeper zk) {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/admin/ZooKeeperAdmin.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/admin/ZooKeeperAdmin.java b/src/java/main/org/apache/zookeeper/admin/ZooKeeperAdmin.java
new file mode 100644
index 0000000..f60e8d5
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/admin/ZooKeeperAdmin.java
@@ -0,0 +1,250 @@
+/**
+ * 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.admin;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.AsyncCallback.DataCallback;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.StringUtils;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.GetDataResponse;
+import org.apache.zookeeper.proto.ReconfigRequest;
+import org.apache.zookeeper.proto.ReplyHeader;
+import org.apache.zookeeper.proto.RequestHeader;
+import org.apache.zookeeper.server.DataTree;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the main class for ZooKeeperAdmin client library.
+ * This library is used to perform cluster administration tasks,
+ * such as reconfigure cluster membership. The ZooKeeperAdmin class
+ * inherits ZooKeeper and has similar usage pattern as ZooKeeper class.
+ * Please check {@link ZooKeeper} class document for more details.
+ *
+ * @since 3.5.3
+ */
+public class ZooKeeperAdmin extends ZooKeeper {
+    private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAdmin.class);
+
+    /**
+     * Create a ZooKeeperAdmin object which is used to perform dynamic reconfiguration
+     * operations.
+     *
+     * @param connectString
+     *            comma separated host:port pairs, each corresponding to a zk
+     *            server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If
+     *            the optional chroot suffix is used the example would look
+     *            like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a"
+     *            where the client would be rooted at "/app/a" and all paths
+     *            would be relative to this root - ie getting/setting/etc...
+     *            "/foo/bar" would result in operations being run on
+     *            "/app/a/foo/bar" (from the server perspective).
+     * @param sessionTimeout
+     *            session timeout in milliseconds
+     * @param watcher
+     *            a watcher object which will be notified of state changes, may
+     *            also be notified for node events
+     *
+     * @throws IOException
+     *             in cases of network failure
+     * @throws IllegalArgumentException
+     *             if an invalid chroot path is specified
+     *
+     * @see ZooKeeper#ZooKeeper(String, int, Watcher)
+     *
+     */
+    public ZooKeeperAdmin(String connectString, int sessionTimeout, Watcher watcher)
+        throws IOException {
+        super(connectString, sessionTimeout, watcher);
+    }
+
+    /**
+     * Create a ZooKeeperAdmin object which is used to perform dynamic reconfiguration
+     * operations.
+     *
+     * @param connectString
+     *            comma separated host:port pairs, each corresponding to a zk
+     *            server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If
+     *            the optional chroot suffix is used the example would look
+     *            like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a"
+     *            where the client would be rooted at "/app/a" and all paths
+     *            would be relative to this root - ie getting/setting/etc...
+     *            "/foo/bar" would result in operations being run on
+     *            "/app/a/foo/bar" (from the server perspective).
+     * @param sessionTimeout
+     *            session timeout in milliseconds
+     * @param watcher
+     *            a watcher object which will be notified of state changes, may
+     *            also be notified for node events
+     * @param conf
+     *            passing this conf object gives each client the flexibility of
+     *            configuring properties differently compared to other instances
+     *
+     * @throws IOException
+     *             in cases of network failure
+     * @throws IllegalArgumentException
+     *             if an invalid chroot path is specified
+     *
+     * @see ZooKeeper#ZooKeeper(String, int, Watcher, ZKClientConfig)
+     */
+    public ZooKeeperAdmin(String connectString, int sessionTimeout, Watcher watcher,
+            ZKClientConfig conf) throws IOException {
+        super(connectString, sessionTimeout, watcher, conf);
+    }
+
+    /**
+     * Create a ZooKeeperAdmin object which is used to perform dynamic reconfiguration
+     * operations.
+     *
+     * @param connectString
+     *            comma separated host:port pairs, each corresponding to a zk
+     *            server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If
+     *            the optional chroot suffix is used the example would look
+     *            like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a"
+     *            where the client would be rooted at "/app/a" and all paths
+     *            would be relative to this root - ie getting/setting/etc...
+     *            "/foo/bar" would result in operations being run on
+     *            "/app/a/foo/bar" (from the server perspective).
+     * @param sessionTimeout
+     *            session timeout in milliseconds
+     * @param watcher
+     *            a watcher object which will be notified of state changes, may
+     *            also be notified for node events
+     * @param canBeReadOnly
+     *            whether the created client is allowed to go to
+     *            read-only mode in case of partitioning. Read-only mode
+     *            basically means that if the client can't find any majority
+     *            servers but there's partitioned server it could reach, it
+     *            connects to one in read-only mode, i.e. read requests are
+     *            allowed while write requests are not. It continues seeking for
+     *            majority in the background.
+     *
+     * @throws IOException
+     *             in cases of network failure
+     * @throws IllegalArgumentException
+     *             if an invalid chroot path is specified
+     *
+     * @see ZooKeeper#ZooKeeper(String, int, Watcher, boolean)
+     */
+    public ZooKeeperAdmin(String connectString, int sessionTimeout, Watcher watcher,
+                     boolean canBeReadOnly) throws IOException {
+        super(connectString, sessionTimeout, watcher, canBeReadOnly);
+    }
+
+    /**
+     * Reconfigure - add/remove servers. Return the new configuration.
+     * @param joiningServers
+     *                a comma separated list of servers being added (incremental reconfiguration)
+     * @param leavingServers
+     *                a comma separated list of servers being removed (incremental reconfiguration)
+     * @param newMembers
+     *                a comma separated list of new membership (non-incremental reconfiguration)
+     * @param fromConfig
+     *                version of the current configuration
+     *                (optional - causes reconfiguration to throw an exception if configuration is no longer current)
+     * @param stat the stat of /zookeeper/config znode will be copied to this
+     *             parameter if not null.
+     * @return new configuration
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     */
+    public byte[] reconfig(String joiningServers, String leavingServers,
+                           String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.reconfig);
+        ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
+        GetDataResponse response = new GetDataResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response, null);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()), "");
+        }
+        if (stat != null) {
+            DataTree.copyStat(response.getStat(), stat);
+        }
+        return response.getData();
+    }
+
+    /**
+     * Convenience wrapper around reconfig that takes Lists of strings instead of comma-separated servers.
+     *
+     * @see #reconfig
+     *
+     */
+    public byte[] reconfig(List<String> joiningServers, List<String> leavingServers,
+                           List<String> newMembers, long fromConfig,
+                           Stat stat) throws KeeperException, InterruptedException {
+        return reconfig(StringUtils.joinStrings(joiningServers, ","),
+                        StringUtils.joinStrings(leavingServers, ","),
+                        StringUtils.joinStrings(newMembers, ","),
+                        fromConfig, stat);
+    }
+
+    /**
+     * The Asynchronous version of reconfig.
+     *
+     * @see #reconfig
+     *
+     **/
+    public void reconfig(String joiningServers, String leavingServers,
+        String newMembers, long fromConfig, DataCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.reconfig);
+        ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
+        GetDataResponse response = new GetDataResponse();
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+               ZooDefs.CONFIG_NODE, ZooDefs.CONFIG_NODE, ctx, null);
+    }
+
+    /**
+     * Convenience wrapper around asynchronous reconfig that takes Lists of strings instead of comma-separated servers.
+     *
+     * @see #reconfig
+     *
+     */
+    public void reconfig(List<String> joiningServers,
+        List<String> leavingServers, List<String> newMembers, long fromConfig,
+        DataCallback cb, Object ctx) {
+        reconfig(StringUtils.joinStrings(joiningServers, ","),
+                 StringUtils.joinStrings(leavingServers, ","),
+                 StringUtils.joinStrings(newMembers, ","),
+                 fromConfig, cb, ctx);
+    }
+
+    /**
+     * String representation of this ZooKeeperAdmin client. Suitable for things
+     * like logging.
+     *
+     * Do NOT count on the format of this string, it may change without
+     * warning.
+     *
+     * @since 3.5.3
+     */
+    @Override
+    public String toString() {
+        return super.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/cli/CliCommand.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/cli/CliCommand.java b/src/java/main/org/apache/zookeeper/cli/CliCommand.java
index 3d0a90b..41e3d85 100644
--- a/src/java/main/org/apache/zookeeper/cli/CliCommand.java
+++ b/src/java/main/org/apache/zookeeper/cli/CliCommand.java
@@ -25,7 +25,6 @@ import org.apache.zookeeper.ZooKeeper;
  * base class for all CLI commands
  */
 abstract public class CliCommand {
-
     protected ZooKeeper zk;
     protected PrintStream out;
     protected PrintStream err;
@@ -63,7 +62,7 @@ abstract public class CliCommand {
 
     /**
      * set the zookeper instance
-     * @param zk the zookeper instance
+     * @param zk the ZooKeeper instance.
      */
     public void setZk(ZooKeeper zk) {
         this.zk = zk;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java b/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
index deb7914..a0709f3 100644
--- a/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
+++ b/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
@@ -18,12 +18,11 @@
 package org.apache.zookeeper.cli;
 
 import java.io.FileInputStream;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.cli.*;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 
@@ -146,7 +145,16 @@ public class ReconfigCommand extends CliCommand {
     public boolean exec() throws CliException {
         try {
             Stat stat = new Stat();
-            byte[] curConfig = zk.reconfig(joining,
+            if (!(zk instanceof ZooKeeperAdmin)) {
+                // This should never happen when executing reconfig command line,
+                // because it is guaranteed that we have a ZooKeeperAdmin instance ready
+                // to use in CliCommand stack.
+                // The only exception would be in test code where clients can directly set
+                // ZooKeeper object to ZooKeeperMain.
+                return false;
+            }
+
+            byte[] curConfig = ((ZooKeeperAdmin)zk).reconfig(joining,
                     leaving, members, version, stat);
             out.println("Committed new configuration:\n" + new String(curConfig));
             

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/DataTree.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java
index 4666578..9be80f9 100644
--- a/src/java/main/org/apache/zookeeper/server/DataTree.java
+++ b/src/java/main/org/apache/zookeeper/server/DataTree.java
@@ -245,15 +245,23 @@ public class DataTree {
         addConfigNode();
     }
 
-     public void addConfigNode() {
-    	 DataNode zookeeperZnode = nodes.get(procZookeeper);
-         if (zookeeperZnode!=null) { // should always be the case
-        	 zookeeperZnode.addChild(configChildZookeeper);
-         } else {
-        	 LOG.error("There's no /zookeeper znode - this should never happen");
-         }
-         nodes.put(configZookeeper, configDataNode);   
-     }
+    public void addConfigNode() {
+        DataNode zookeeperZnode = nodes.get(procZookeeper);
+        if (zookeeperZnode != null) { // should always be the case
+            zookeeperZnode.addChild(configChildZookeeper);
+        } else {
+            assert false : "There's no /zookeeper znode - this should never happen.";
+        }
+
+        nodes.put(configZookeeper, configDataNode);
+        try {
+            // Reconfig node is access controlled by default (ZOOKEEPER-2014).
+            setACL(configZookeeper, ZooDefs.Ids.READ_ACL_UNSAFE, -1);
+        } catch (KeeperException.NoNodeException e) {
+            assert false : "There's no " + configZookeeper +
+                    " znode - this should never happen.";
+        }
+    }
 
     /**
      * is the path one of the special paths owned by zookeeper.

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
index e275f9c..4d3b1ea 100644
--- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
+++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
@@ -425,6 +425,15 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                 addChangeRecord(nodeRecord);
                 break;
             case OpCode.reconfig:
+                if (!QuorumPeerConfig.isReconfigEnabled()) {
+                    LOG.error("Reconfig operation requested but reconfig feature is disabled.");
+                    throw new KeeperException.ReconfigDisabledException();
+                }
+
+                if (skipACL) {
+                    LOG.warn("skipACL is set, reconfig operation will skip ACL checks!");
+                }
+
                 zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
                 ReconfigRequest reconfigRequest = (ReconfigRequest)record; 
                 LeaderZooKeeperServer lzks;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
index d4f150b..d23c04b 100644
--- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
+++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
@@ -169,7 +169,8 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
      * @param tickTime the ticktime for the server
      * @throws IOException
      */
-    public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime) throws IOException {
+    public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime)
+            throws IOException {
         this(txnLogFactory, tickTime, -1, -1, new ZKDatabase(txnLogFactory));
     }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
index 885a5e1..5bfeed3 100644
--- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
+++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
@@ -110,7 +110,8 @@ public class ZooKeeperServerMain {
      * @throws IOException
      * @throws AdminServerException
      */
-    public void runFromConfig(ServerConfig config) throws IOException, AdminServerException {
+    public void runFromConfig(ServerConfig config)
+            throws IOException, AdminServerException {
         LOG.info("Starting server");
         FileTxnSnapLog txnLog = null;
         try {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
index 24a4ec1..cb8f1c2 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
@@ -60,6 +60,7 @@ public class QuorumPeerConfig {
     public static final String nextDynamicConfigFileSuffix = ".dynamic.next";
 
     private static boolean standaloneEnabled = true;
+    private static boolean reconfigEnabled = false;
 
     protected InetSocketAddress clientPortAddress;
     protected InetSocketAddress secureClientPortAddress;
@@ -279,7 +280,15 @@ public class QuorumPeerConfig {
                 } else if (value.toLowerCase().equals("false")) {
                     setStandaloneEnabled(false);
                 } else {
-                    throw new ConfigException("Invalid option for standalone mode. Choose 'true' or 'false.'");
+                    throw new ConfigException("Invalid option " + value + " for standalone mode. Choose 'true' or 'false.'");
+                }
+            } else if (key.equals("reconfigEnabled")) {
+                if (value.toLowerCase().equals("true")) {
+                    setReconfigEnabled(true);
+                } else if (value.toLowerCase().equals("false")) {
+                    setReconfigEnabled(false);
+                } else {
+                    throw new ConfigException("Invalid option " + value + " for reconfigEnabled flag. Choose 'true' or 'false.'");
                 }
             } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                 throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
@@ -732,7 +741,13 @@ public class QuorumPeerConfig {
     }
     
     public static void setStandaloneEnabled(boolean enabled) {
-	standaloneEnabled = enabled;
+        standaloneEnabled = enabled;
+    }
+
+    public static boolean isReconfigEnabled() { return reconfigEnabled; }
+
+    public static void setReconfigEnabled(boolean enabled) {
+        reconfigEnabled = enabled;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
index bfe8588..bd49dbf 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
@@ -126,7 +126,9 @@ public class QuorumPeerMain {
         }
     }
 
-    public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServerException {
+    public void runFromConfig(QuorumPeerConfig config)
+            throws IOException, AdminServerException
+    {
       try {
           ManagedUtil.registerLog4jMBeans();
       } catch (JMException e) {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java
----------------------------------------------------------------------
diff --git a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java
index 109c1b5..2ed516c 100644
--- a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java
+++ b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java
@@ -25,7 +25,6 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.HashMap;
 
-
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/TestableZooKeeper.java b/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
index 4d46fdf..c69033c 100644
--- a/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
+++ b/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
@@ -25,10 +25,11 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.jute.Record;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.RequestHeader;
 
-public class TestableZooKeeper extends ZooKeeper {
+public class TestableZooKeeper extends ZooKeeperAdmin {
 
     public TestableZooKeeper(String host, int sessionTimeout,
             Watcher watcher) throws IOException {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/server/DataTreeTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/DataTreeTest.java b/src/java/test/org/apache/zookeeper/server/DataTreeTest.java
index d726643..8b2bd80 100644
--- a/src/java/test/org/apache/zookeeper/server/DataTreeTest.java
+++ b/src/java/test/org/apache/zookeeper/server/DataTreeTest.java
@@ -26,12 +26,10 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.DataTree;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.zookeeper.server.DataNode;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
@@ -200,29 +198,34 @@ public class DataTreeTest extends ZKTestCase {
         BinaryOutputArchive oa = new BinaryOutputArchive(out) {
             @Override
             public void writeRecord(Record r, String tag) throws IOException {
-                DataNode node = (DataNode) r;
-                if (node.data.length == 1 && node.data[0] == 42) {
-                    final Semaphore semaphore = new Semaphore(0);
-                    new Thread(new Runnable() {
-                        @Override
-                        public void run() {
-                            synchronized (markerNode) {
-                                //When we lock markerNode, allow writeRecord to continue
-                                semaphore.release();
+                // Need check if the record is a DataNode instance because of changes in ZOOKEEPER-2014
+                // which adds default ACL to config node.
+                if (r instanceof DataNode) {
+                    DataNode node = (DataNode) r;
+                    if (node.data.length == 1 && node.data[0] == 42) {
+                        final Semaphore semaphore = new Semaphore(0);
+                        new Thread(new Runnable() {
+                            @Override
+                            public void run() {
+                                synchronized (markerNode) {
+                                    //When we lock markerNode, allow writeRecord to continue
+                                    semaphore.release();
+                                }
                             }
+                        }).start();
+
+                        try {
+                            boolean acquired = semaphore.tryAcquire(30, TimeUnit.SECONDS);
+                            //This is the real assertion - could another thread lock
+                            //the DataNode we're currently writing
+                            Assert.assertTrue("Couldn't acquire a lock on the DataNode while we were calling tree.serialize", acquired);
+                        } catch (InterruptedException e1) {
+                            throw new RuntimeException(e1);
                         }
-                    }).start();
-
-                    try {
-                        boolean acquired = semaphore.tryAcquire(30, TimeUnit.SECONDS);
-                        //This is the real assertion - could another thread lock
-                        //the DataNode we're currently writing
-                        Assert.assertTrue("Couldn't acquire a lock on the DataNode while we were calling tree.serialize", acquired);
-                    } catch (InterruptedException e1) {
-                        throw new RuntimeException(e1);
+                        ranTestCase.set(true);
                     }
-                    ranTestCase.set(true);
                 }
+
                 super.writeRecord(r, tag);
             }
         };

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java
index 4debe74..85284f6 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java
@@ -49,7 +49,8 @@ public class LearnerTest extends ZKTestCase {
 
         Learner learner;
 
-        public SimpleLearnerZooKeeperServer(FileTxnSnapLog ftsl, QuorumPeer self) throws IOException {
+        public SimpleLearnerZooKeeperServer(FileTxnSnapLog ftsl, QuorumPeer self)
+                throws IOException {
             super(ftsl, 2000, 2000, 2000, new ZKDatabase(ftsl), self);
         }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/server/quorum/RaceConditionTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/RaceConditionTest.java b/src/java/test/org/apache/zookeeper/server/quorum/RaceConditionTest.java
index ef7f3df..46ebcf2 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/RaceConditionTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/RaceConditionTest.java
@@ -151,7 +151,8 @@ public class RaceConditionTest extends QuorumPeerTestBase {
         }
 
         public CustomQuorumPeer(Map<Long, QuorumServer> quorumPeers, File snapDir, File logDir, int clientPort,
-                int electionAlg, long myid, int tickTime, int initLimit, int syncLimit) throws IOException {
+                int electionAlg, long myid, int tickTime, int initLimit, int syncLimit)
+                throws IOException {
             super(quorumPeers, snapDir, logDir, electionAlg, myid, tickTime, initLimit, syncLimit, false,
                     ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), new QuorumMaj(quorumPeers));
         }
@@ -234,7 +235,8 @@ public class RaceConditionTest extends QuorumPeerTestBase {
 
     private static class MockTestQPMain extends TestQPMain {
         @Override
-        public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServerException {
+        public void runFromConfig(QuorumPeerConfig config)
+                throws IOException, AdminServerException {
             quorumPeer = new CustomQuorumPeer(config.getQuorumVerifier().getAllMembers(), config.getDataDir(),
                     config.getDataLogDir(), config.getClientPortAddress().getPort(), config.getElectionAlg(),
                     config.getServerId(), config.getTickTime(), config.getInitLimit(), config.getSyncLimit());

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java
index 241af52..8bc04bd 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java
@@ -20,6 +20,7 @@ package org.apache.zookeeper.server.quorum;
 
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 import org.apache.zookeeper.common.StringUtils;
 import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.ReconfigTest;
@@ -61,6 +62,8 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
     @Before
     public void setup() {
         ClientBase.setupTestEnv();
+        System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest",
+                "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/);
     }
 
     /**
@@ -80,6 +83,7 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
                     + clientPorts[i];
             sb.append(server + "\n");
         }
+
         String currentQuorumCfgSection = sb.toString();
 
         MainThread mt[] = new MainThread[SERVER_COUNT];
@@ -145,14 +149,16 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
             oldServers.add(servers[i]);
             sb.append(servers[i] + "\n");
         }
+
         String quorumCfgSection = sb.toString();
 
         MainThread mt[] = new MainThread[NEW_SERVER_COUNT];
         ZooKeeper zk[] = new ZooKeeper[NEW_SERVER_COUNT];
+        ZooKeeperAdmin zkAdmin[] = new ZooKeeperAdmin[NEW_SERVER_COUNT];
 
         // start old cluster
         for (int i = 0; i < SERVER_COUNT; i++) {
-            mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection);
+            mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection, "reconfigEnabled=true\n");
             mt[i].start();
         }
 
@@ -164,6 +170,9 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
                     ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i],
                             CONNECTION_TIMEOUT));
             zk[i] = ClientBase.createZKClient("127.0.0.1:" + clientPorts[i]);
+            zkAdmin[i] = new ZooKeeperAdmin("127.0.0.1:" + clientPorts[i],
+                    ClientBase.CONNECTION_TIMEOUT, this);
+            zkAdmin[i].addAuthInfo("digest", "super:test".getBytes());
 
             Properties cfg = ReconfigLegacyTest.readPropertiesFromFile(mt[i].confFile);
             String filename = cfg.getProperty("dynamicConfigFile", "");
@@ -186,7 +195,7 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
             }
         }
 
-        ReconfigTest.reconfig(zk[1], null, null, newServers, -1);
+        ReconfigTest.reconfig(zkAdmin[1], null, null, newServers, -1);
 
         // start additional new servers
         for (int i = SERVER_COUNT; i < NEW_SERVER_COUNT; i++) {
@@ -230,6 +239,7 @@ public class ReconfigBackupTest extends QuorumPeerTestBase {
         for (int i = 0; i < SERVER_COUNT; i++) {
             mt[i].shutdown();
             zk[i].close();
+            zkAdmin[i].close();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73e102a5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java
index 301837d..6da5181 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java
@@ -30,6 +30,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.admin.AdminServer.AdminServerException;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -38,6 +39,7 @@ import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,6 +49,13 @@ public class ReconfigDuringLeaderSyncTest extends QuorumPeerTestBase {
     private static int SERVER_COUNT = 3;
     private MainThread[] mt;
 
+    @Before
+    public void setup() {
+        System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest",
+                "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/);
+        QuorumPeerConfig.setReconfigEnabled(true);
+    }
+
     /**
      * <pre>
      * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2172.
@@ -86,8 +95,9 @@ public class ReconfigDuringLeaderSyncTest extends QuorumPeerTestBase {
                     ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], CONNECTION_TIMEOUT));
         }
         CountdownWatcher watch = new CountdownWatcher();
-        ZooKeeper preReconfigClient = new ZooKeeper("127.0.0.1:" + clientPorts[0], ClientBase.CONNECTION_TIMEOUT,
-                watch);
+        ZooKeeperAdmin preReconfigClient = new ZooKeeperAdmin("127.0.0.1:" + clientPorts[0],
+                ClientBase.CONNECTION_TIMEOUT, watch);
+        preReconfigClient.addAuthInfo("digest", "super:test".getBytes());
         watch.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
 
         // new server joining
@@ -198,7 +208,8 @@ public class ReconfigDuringLeaderSyncTest extends QuorumPeerTestBase {
         private boolean newLeaderMessage = false;
 
         public CustomQuorumPeer(Map<Long, QuorumServer> quorumPeers, File snapDir, File logDir, int clientPort,
-                int electionAlg, long myid, int tickTime, int initLimit, int syncLimit) throws IOException {
+                int electionAlg, long myid, int tickTime, int initLimit, int syncLimit)
+                throws IOException {
             super(quorumPeers, snapDir, logDir, electionAlg, myid, tickTime, initLimit, syncLimit, false,
                     ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), new QuorumMaj(quorumPeers));
         }
@@ -241,7 +252,8 @@ public class ReconfigDuringLeaderSyncTest extends QuorumPeerTestBase {
 
     private static class MockTestQPMain extends TestQPMain {
         @Override
-        public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServerException {
+        public void runFromConfig(QuorumPeerConfig config)
+                throws IOException, AdminServerException {
             quorumPeer = new CustomQuorumPeer(config.getQuorumVerifier().getAllMembers(), config.getDataDir(),
                     config.getDataLogDir(), config.getClientPortAddress().getPort(), config.getElectionAlg(),
                     config.getServerId(), config.getTickTime(), config.getInitLimit(), config.getSyncLimit());