You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by lvfangmin <gi...@git.apache.org> on 2018/09/28 06:26:34 UTC

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

GitHub user lvfangmin opened a pull request:

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

    ZOOKEEPER-3125: Fixing pzxid consistent issue when replaying a txn for a deleted node

    Port this to 3.5, this also fixed the issue where the pzxid might be overwritten by a smaller one when replaying deleteNode txn, which will be fixed in a separate PR for 3.6.

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

    $ git pull https://github.com/lvfangmin/zookeeper ZOOKEEPER-3125-3.5

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

    https://github.com/apache/zookeeper/pull/647.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 #647
    
----
commit 90ef67cf79c0eaec4b8b3389a8711cce8ba25255
Author: Fangmin Lyu <al...@...>
Date:   2018-09-12T11:29:13Z

    ZOOKEEPER-3125: Fixing pzxid consistent issue when replaying a txn for a deleted node
    
    Author: Fangmin Lyu <al...@fb.com>
    
    Reviewers: breed@apache.org, andor@apache.org
    
    Closes #605 from lvfangmin/ZOOKEEPER-3125

----


---

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

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

    https://github.com/apache/zookeeper/pull/647#discussion_r221200641
  
    --- Diff: src/java/test/org/apache/zookeeper/server/DataTreeTest.java ---
    @@ -153,6 +153,31 @@ public void testIncrementCversion() throws Exception {
                     (newCversion == prevCversion + 1 && newPzxid == prevPzxid + 1));
         }
     
    +    @Test
    +    public void testPzxidUpdatedWhenDeletingNonExistNode() throws Exception {
    --- End diff --
    
    This new test is also part of the fix.


---

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

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

    https://github.com/apache/zookeeper/pull/647#discussion_r221201278
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java ---
    @@ -162,6 +167,98 @@ public void process(String path) {
                     new String(zk[followerA].getData(node2, null, null)));
         }
     
    +    /**
    +     * It's possibel during SNAP sync, the parent is serialized before the
    +     * child get deleted during sending the snapshot over.
    +     *
    +     * In which case, we need to make sure the pzxid get correctly updated
    +     * when applying the txns received.
    +     */
    +    @Test
    +    public void testPZxidUpdatedDuringSnapSyncing() throws Exception {
    +        LOG.info("Enable force snapshot sync");
    +        System.setProperty(LearnerHandler.FORCE_SNAP_SYNC, "true");
    +
    +        final String parent = "/testPZxidUpdatedWhenDeletingNonExistNode";
    +        final String child = parent + "/child";
    +        createEmptyNode(zk[leaderId], parent);
    +        createEmptyNode(zk[leaderId], child);
    +
    +        LOG.info("shutdown follower {}", followerA);
    +        mt[followerA].shutdown();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTING);
    +
    +        LOG.info("Set up ZKDatabase to catch the node serializing in DataTree");
    +        addSerializeListener(leaderId, parent, child);
    +
    +        LOG.info("Restart follower A to trigger a SNAP sync with leader");
    +        mt[followerA].start();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTED);
    +
    +        LOG.info("Check and make sure the pzxid of the parent is the same " +
    +                "on leader and follower A");
    +        compareStat(parent, leaderId, followerA);
    +    }
    +
    +    /**
    +     * It's possible during taking fuzzy snapshot, the parent is serialized
    +     * before the child get deleted in the fuzzy range.
    +     *
    +     * In which case, we need to make sure the pzxid get correctly updated
    +     * when replaying the txns.
    +     */
    +    @Test
    +    public void testPZxidUpdatedWhenLoadingSnapshot() throws Exception {
    +
    +        final String parent = "/testPZxidUpdatedDuringTakingSnapshot";
    +        final String child = parent + "/child";
    +        createEmptyNode(zk[followerA], parent);
    +        createEmptyNode(zk[followerA], child);
    +
    +        LOG.info("Set up ZKDatabase to catch the node serializing in DataTree");
    +        addSerializeListener(followerA, parent, child);
    +
    +        LOG.info("Take snapshot on follower A");
    +        ZooKeeperServer zkServer = mt[followerA].main.quorumPeer.getActiveServer();
    +        zkServer.takeSnapshot();
    +
    +        LOG.info("Restarting follower A to load snapshot");
    +        mt[followerA].shutdown();
    +        mt[followerA].start();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTED);
    +
    +        LOG.info("Check and make sure the pzxid of the parent is the same " +
    +                "on leader and follower A");
    +        compareStat(parent, leaderId, followerA);
    +    }
    +
    +    private void addSerializeListener(int sid, String parent, String child) {
    +        final ZooKeeper zkClient = zk[followerA];
    +        CustomDataTree dt =
    +                (CustomDataTree) mt[sid].main.quorumPeer.getZkDb().getDataTree();
    +        dt.addListener(parent, new NodeSerializeListener() {
    +            @Override
    +            public void nodeSerialized(String path) {
    +                try {
    +                    zkClient.delete(child, -1);
    +                    LOG.info("Deleted the child node after the parent is serialized");
    +                } catch (Exception e) {
    +                    LOG.error("Error when deleting node {}", e);
    +                }
    +            }
    +        });
    +    }
    +
    +    private void compareStat(String path, int sid, int compareWithSid) throws Exception {
    +        Stat stat1 = new Stat();
    +        zk[sid].getData(path, null, stat1);
    +
    +        Stat stat2 = new Stat();
    +        zk[compareWithSid].getData(path, null, stat2);
    --- End diff --
    
    @hanm mentioned a flaky test issue related to this part:
    https://github.com/apache/zookeeper/pull/605/files#r219628119
    I don't see answer in the original PR, how are you dealing with the problem?


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @lvfangmin Got it. Fine. Go ahead please. Just make sure that all patches go under the same Jira, so that they couldn't get lost.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2389/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    Merged, close this PR.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar should we get this in?


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @lvfangmin Sounds acceptable.
    If the flaky cannot be fixed with my suggestion (waiting for client to disconnect), let's put the retry back in.
    I'll commit afterwards. Thanks.


---

[GitHub] zookeeper issue #647: ZOOKEEPER-3125: Fixing pzxid consistent issue when rep...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar it's not fixing multiple issues but the same in ZOOKEEPER-3125, the fix on master has bug we found recently, I think we should include this when port back to 3.5


---

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

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

    https://github.com/apache/zookeeper/pull/647#discussion_r221200308
  
    --- Diff: src/java/main/org/apache/zookeeper/server/DataTree.java ---
    @@ -527,6 +527,24 @@ public void deleteNode(String path, long zxid)
             int lastSlash = path.lastIndexOf('/');
             String parentName = path.substring(0, lastSlash);
             String childName = path.substring(lastSlash + 1);
    +
    +        // The child might already be deleted during taking fuzzy snapshot,
    +        // but we still need to update the pzxid here before throw exception
    +        // for no such child
    +        DataNode parent = nodes.get(parentName);
    +        if (parent == null) {
    +            throw new KeeperException.NoNodeException();
    +        }
    +        synchronized (parent) {
    +            parent.removeChild(childName);
    +            // Only update pzxid when the zxid is larger than the current pzxid,
    +            // otherwise we might override higher pzxid set by a following create 
    +            // Txn, which could cause the cversion and pzxid inconsistent
    +            if (zxid > parent.stat.getPzxid()) {
    --- End diff --
    
    Looks like this is the fix you're talking about and which will be fixed separately in 3.6. Is that correct?


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    >>  I added a junit retry rule class to retry with specific exception
    
    LGTM, thanks @lvfangmin 


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar @hanm I added a junit retry rule class to retry with specific exception, currently I only use it to catch the connection loss exception in FuzzySnapshotRelatedTest, we can use this in other tests if there is similar issue.
    
    If this looks good to you I'll add it to 3.6 as well.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2376/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    retest this please


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    I think we can add Retry rules as long as the cause of the flaky is clear (e.g. this case, since ConnectionLoss is a well known flaky cause); what I worried previously was to apply it unanimously without analyzing the actual cause. 


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2474/



---

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

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

    https://github.com/apache/zookeeper/pull/647#discussion_r221390760
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java ---
    @@ -162,6 +167,98 @@ public void process(String path) {
                     new String(zk[followerA].getData(node2, null, null)));
         }
     
    +    /**
    +     * It's possibel during SNAP sync, the parent is serialized before the
    +     * child get deleted during sending the snapshot over.
    +     *
    +     * In which case, we need to make sure the pzxid get correctly updated
    +     * when applying the txns received.
    +     */
    +    @Test
    +    public void testPZxidUpdatedDuringSnapSyncing() throws Exception {
    +        LOG.info("Enable force snapshot sync");
    +        System.setProperty(LearnerHandler.FORCE_SNAP_SYNC, "true");
    +
    +        final String parent = "/testPZxidUpdatedWhenDeletingNonExistNode";
    +        final String child = parent + "/child";
    +        createEmptyNode(zk[leaderId], parent);
    +        createEmptyNode(zk[leaderId], child);
    +
    +        LOG.info("shutdown follower {}", followerA);
    +        mt[followerA].shutdown();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTING);
    +
    +        LOG.info("Set up ZKDatabase to catch the node serializing in DataTree");
    +        addSerializeListener(leaderId, parent, child);
    +
    +        LOG.info("Restart follower A to trigger a SNAP sync with leader");
    +        mt[followerA].start();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTED);
    +
    +        LOG.info("Check and make sure the pzxid of the parent is the same " +
    +                "on leader and follower A");
    +        compareStat(parent, leaderId, followerA);
    +    }
    +
    +    /**
    +     * It's possible during taking fuzzy snapshot, the parent is serialized
    +     * before the child get deleted in the fuzzy range.
    +     *
    +     * In which case, we need to make sure the pzxid get correctly updated
    +     * when replaying the txns.
    +     */
    +    @Test
    +    public void testPZxidUpdatedWhenLoadingSnapshot() throws Exception {
    +
    +        final String parent = "/testPZxidUpdatedDuringTakingSnapshot";
    +        final String child = parent + "/child";
    +        createEmptyNode(zk[followerA], parent);
    +        createEmptyNode(zk[followerA], child);
    +
    +        LOG.info("Set up ZKDatabase to catch the node serializing in DataTree");
    +        addSerializeListener(followerA, parent, child);
    +
    +        LOG.info("Take snapshot on follower A");
    +        ZooKeeperServer zkServer = mt[followerA].main.quorumPeer.getActiveServer();
    +        zkServer.takeSnapshot();
    +
    +        LOG.info("Restarting follower A to load snapshot");
    +        mt[followerA].shutdown();
    +        mt[followerA].start();
    +        QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTED);
    +
    +        LOG.info("Check and make sure the pzxid of the parent is the same " +
    +                "on leader and follower A");
    +        compareStat(parent, leaderId, followerA);
    +    }
    +
    +    private void addSerializeListener(int sid, String parent, String child) {
    +        final ZooKeeper zkClient = zk[followerA];
    +        CustomDataTree dt =
    +                (CustomDataTree) mt[sid].main.quorumPeer.getZkDb().getDataTree();
    +        dt.addListener(parent, new NodeSerializeListener() {
    +            @Override
    +            public void nodeSerialized(String path) {
    +                try {
    +                    zkClient.delete(child, -1);
    +                    LOG.info("Deleted the child node after the parent is serialized");
    +                } catch (Exception e) {
    +                    LOG.error("Error when deleting node {}", e);
    +                }
    +            }
    +        });
    +    }
    +
    +    private void compareStat(String path, int sid, int compareWithSid) throws Exception {
    +        Stat stat1 = new Stat();
    +        zk[sid].getData(path, null, stat1);
    +
    +        Stat stat2 = new Stat();
    +        zk[compareWithSid].getData(path, null, stat2);
    --- End diff --
    
    I made a comment on that before sending this diff out, I was trying to hear the opinion before making the test change here. Seems we agreed on not using RetryRule, I'll improve this test class.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    I remember I commented in Jira ZOOKEEPER-3157, not sure why it didn't show up. 
    
    I mentioned that we still need RetryRule, because there might be temporary quorum unstable issues like what we found on our test environment. The quorum set up in the test might be down due to leader election in case there is heavy load/limited resources on that test environment. We have seen this happened internally, so it's better to have retry for ConnectionLoss in this case.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2304/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar I can understand your concern, let's remove the RetryRule for now, we can add it when necessary.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2518/



---

[GitHub] zookeeper issue #647: ZOOKEEPER-3125: Fixing pzxid consistent issue when rep...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2279/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar here is the scenario shows why the previous fix on master has problem:
    
    1. parent A is in it's parent's serializing list
    2. before it's being serialized, child 1 was deleted in txn T1, and child 2 was created in txn T2
    3. when parent A is serialized, it's cversion and pzxid is already updated correctly by T2
    4. when reloading from disk, T1 will update the pzxid and left cversion there
    5. T2 checked the node, and it's already there, then it goes to the patching process, and it found the parent's cversion is already up to date and skipped patching it, which leaves the pzxid in stale state


---

[GitHub] zookeeper pull request #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue ...

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

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


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @lvfangmin Given that I've already provided a fix for the flakyness in #657 , do we still need this retry rule?


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    @anmolnar what's your opinion with @hanm 's reply?


---

[GitHub] zookeeper pull request #647: ZOOKEEPER-3125: Fixing pzxid consistent issue w...

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

    https://github.com/apache/zookeeper/pull/647#discussion_r221391402
  
    --- Diff: src/java/main/org/apache/zookeeper/server/DataTree.java ---
    @@ -527,6 +527,24 @@ public void deleteNode(String path, long zxid)
             int lastSlash = path.lastIndexOf('/');
             String parentName = path.substring(0, lastSlash);
             String childName = path.substring(lastSlash + 1);
    +
    +        // The child might already be deleted during taking fuzzy snapshot,
    +        // but we still need to update the pzxid here before throw exception
    +        // for no such child
    +        DataNode parent = nodes.get(parentName);
    +        if (parent == null) {
    +            throw new KeeperException.NoNodeException();
    +        }
    +        synchronized (parent) {
    +            parent.removeChild(childName);
    +            // Only update pzxid when the zxid is larger than the current pzxid,
    +            // otherwise we might override higher pzxid set by a following create 
    +            // Txn, which could cause the cversion and pzxid inconsistent
    +            if (zxid > parent.stat.getPzxid()) {
    --- End diff --
    
    This is a new bug I found recently, the previous change in ZOOKEEPER-3125 has a bug which could revert pzxid. Given it's a known issue in that Jira, I think it's reasonable to fix it in the same patch here before porting back to 3.5.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    I'll commit this once the testing part is finalized.


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2454/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2458/



---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    I still have bad feelings about introducing `RetryRule` in this patch. I haven't seen connectionLoss errors recently on the builds, neither on this patch after the fix and not sure it's a good thing to introduce it on an ad-hoc basis. Given that this 3.5.
    @hanm What are your thoughts?


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    Merged. Thanks @lvfangmin !


---

[GitHub] zookeeper issue #647: [ZOOKEEPER-3125] Fixing pzxid consistent issue when re...

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

    https://github.com/apache/zookeeper/pull/647
  
    retest this please


---