You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by hanm <gi...@git.apache.org> on 2016/10/31 04:01:58 UTC

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

GitHub user hanm opened a pull request:

    https://github.com/apache/zookeeper/pull/96

    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/

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hanm/zookeeper ZOOKEEPER-2014

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/96.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #96
    
----
commit 6d18cffde99d4cf5298e045d6c0f23b36fd62925
Author: Michael Han <ha...@cloudera.com>
Date:   2016-10-31T03:58:11Z

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

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86679034
  
    --- Diff: src/java/main/org/apache/zookeeper/server/DataTree.java ---
    @@ -245,15 +245,25 @@ public 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 {
    +            LOG.error("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) {
    +            LOG.error("Fail to set ACL on {} - this should never happen: {}", configZookeeper, e);
    --- End diff --
    
    actually if we are asserting above, perhaps we should also assert here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86673398
  
    --- Diff: src/c/tests/TestReconfigServer.cc ---
    @@ -324,4 +336,109 @@ 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.
    +    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_[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);
    +
    +    // 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.
    +    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_[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);
    --- End diff --
    
    connect to follower seems repeated, can we move it to a private helper method?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86679094
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReconfigExceptionTest.java ---
    @@ -0,0 +1,220 @@
    +/**
    + * 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 java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeoutException;
    +
    +import org.apache.zookeeper.ZKTestCase;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.PortAssignment;
    +import org.apache.zookeeper.admin.ZooKeeperAdmin;
    +import org.apache.zookeeper.data.ACL;
    +import org.apache.zookeeper.data.Id;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.junit.After;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReconfigExceptionTest extends ZKTestCase {
    +    private static final Logger LOG = LoggerFactory
    +            .getLogger(ReconfigExceptionTest.class);
    +    private static String authProvider = "zookeeper.DigestAuthenticationProvider.superDigest";
    +    // Use DigestAuthenticationProvider.base64Encode or
    +    // run ZooKeeper jar with org.apache.zookeeper.server.auth.DigestAuthenticationProvider to generate password.
    +    // An example:
    +    // java -cp zookeeper-3.6.0-SNAPSHOT.jar:lib/log4j-1.2.17.jar:lib/slf4j-log4j12-1.7.5.jar:
    +    // lib/slf4j-api-1.7.5.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider super:test
    +    // The password here is 'test'.
    +    private static String superDigest = "super:D/InIHSb7yEEbrWz8b9l71RjZJU=";
    +    private QuorumUtil qu;
    +    private ZooKeeperAdmin zkAdmin;
    +
    +    @Before
    +    public void setup() throws InterruptedException, KeeperException.NoNodeException {
    --- End diff --
    
    we don't throw NoNodeException anymore


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86679058
  
    --- Diff: src/java/test/org/apache/zookeeper/server/DataTreeTest.java ---
    @@ -200,29 +198,34 @@ public void testSerializeDoesntLockDataNodeWhileWriting() throws Exception {
             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) {
    --- End diff --
    
    is there a reason we added the instanceof here? if we didn't need it before, why do we need it now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86679075
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ClientBase.java ---
    @@ -356,7 +356,7 @@ private static int getPort(String hostPort) {
          */
         public static void startServerInstance(File dataDir,
                 ServerCnxnFactory factory, String hostPort) throws IOException,
    -            InterruptedException {
    +            InterruptedException, KeeperException.NoNodeException {
    --- End diff --
    
    this isn't needed anymore right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86673475
  
    --- Diff: src/java/main/org/apache/zookeeper/server/DataTree.java ---
    @@ -245,15 +245,25 @@ public 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
    --- End diff --
    
    nit: spaces around `!=`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86673489
  
    --- Diff: src/java/main/org/apache/zookeeper/server/DataTree.java ---
    @@ -245,15 +245,25 @@ public 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
    --- End diff --
    
    i think this should be an assertion (and we can drop the LOG.error call)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86678939
  
    --- Diff: src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml ---
    @@ -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 depends on which server
    --- End diff --
    
    depends -> depending


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r86679106
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReconfigExceptionTest.java ---
    @@ -0,0 +1,220 @@
    +/**
    + * 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 java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeoutException;
    +
    +import org.apache.zookeeper.ZKTestCase;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.PortAssignment;
    +import org.apache.zookeeper.admin.ZooKeeperAdmin;
    +import org.apache.zookeeper.data.ACL;
    +import org.apache.zookeeper.data.Id;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.junit.After;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReconfigExceptionTest extends ZKTestCase {
    +    private static final Logger LOG = LoggerFactory
    +            .getLogger(ReconfigExceptionTest.class);
    +    private static String authProvider = "zookeeper.DigestAuthenticationProvider.superDigest";
    +    // Use DigestAuthenticationProvider.base64Encode or
    +    // run ZooKeeper jar with org.apache.zookeeper.server.auth.DigestAuthenticationProvider to generate password.
    +    // An example:
    +    // java -cp zookeeper-3.6.0-SNAPSHOT.jar:lib/log4j-1.2.17.jar:lib/slf4j-log4j12-1.7.5.jar:
    +    // lib/slf4j-api-1.7.5.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider super:test
    +    // The password here is 'test'.
    +    private static String superDigest = "super:D/InIHSb7yEEbrWz8b9l71RjZJU=";
    +    private QuorumUtil qu;
    +    private ZooKeeperAdmin zkAdmin;
    +
    +    @Before
    +    public void setup() throws InterruptedException, KeeperException.NoNodeException {
    --- End diff --
    
    +1 awesome work! i only found little nits! thanx for sticking with this!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #96: ZOOKEEPER-2014: Only admin should be allowed to reconfi...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/96
  
    Thanks @breed and @rgs1 for your time and review feedback. Pull request, patch, and review board is now updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by fpj <gi...@git.apache.org>.
Github user fpj commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r87536973
  
    --- Diff: src/java/test/org/apache/zookeeper/server/DataTreeTest.java ---
    @@ -200,29 +198,34 @@ public void testSerializeDoesntLockDataNodeWhileWriting() throws Exception {
             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) {
    --- End diff --
    
    @hanm hmm, I'm not sure about this. In the changes for `DataTree`, we only set the ACL of the `/zookeeper/config` znode, but setting ACLs was something we were doing before, so I'm confused about why we can have a mix of znode records and ACL records with the changes proposed here. Could you clarify, please?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/zookeeper/pull/96


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #96: ZOOKEEPER-2014: Only admin should be allowed to ...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/96#discussion_r87543684
  
    --- Diff: src/java/test/org/apache/zookeeper/server/DataTreeTest.java ---
    @@ -200,29 +198,34 @@ public void testSerializeDoesntLockDataNodeWhileWriting() throws Exception {
             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) {
    --- End diff --
    
    @fpj - we were not setting ACLs on intrinsic znodes (i.e. /zookeeper/config) ZooKeeper implicitly created while initializing a DataTree before. And for this test case, it only creates znodes, not ACLs. As a result, it's reasonable for the previous test case to assume every record that's serializing is a DataNode record. Now with this patch, there is an ACL implicitly created when /zookeeper/config node is created, so the previous assumption (that all records to be serialized are DataNode record) does not hold. Thus, a change is required.
    
    For reference, you could put a break point on https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ReferenceCountedACLCache.java#L133 while running this test case, and you will see there is one ACL that's serialized. Now you can remove the ACL associated with /zookeeper/config at https://github.com/apache/zookeeper/pull/96/files#diff-a676d93082759105dd8c79c0a76a8007R259, and you will see the break point on ReferenceCountedACLCache.java previous set not get hit. That is the difference.
    
    Another way to experiment this is to create an ACL in this test (without applying this pull request first), something like:
    `final DataNode markerNode = tree.getNode("/marker");
      tree.setACL("/marker", ZooDefs.Ids.READ_ACL_UNSAFE, -1);` will do. Then we will see the same type casting failure - this simulates what this PR will do in terms of changing the type of records. Basically I think the root cause is the test itself could be made more robust, by eliminate the assumptions (that every record is a DataNode) that might not always hold.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---