zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-2355) Ephemeral node is never deleted if follower fails while reading the proposal packet
Date Tue, 13 Jun 2017 21:01:00 GMT

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16048398#comment-16048398
] 

ASF GitHub Bot commented on ZOOKEEPER-2355:
-------------------------------------------

Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/112#discussion_r121795494
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java
---
    @@ -0,0 +1,222 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.server.quorum;
    +
    +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertNull;
    +
    +import java.io.IOException;
    +import java.net.SocketTimeoutException;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.PortAssignment;
    +import org.apache.zookeeper.ZooDefs.Ids;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
    +import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
    +import org.junit.After;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +public class EphemeralNodeDeletionTest extends QuorumPeerTestBase {
    +    private static int SERVER_COUNT = 3;
    +    private MainThread[] mt = new MainThread[SERVER_COUNT];
    +
    +    /**
    +     * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2355.
    +     * ZooKeeper ephemeral node is never deleted if follower fail while reading
    +     * the proposal packet.
    +     */
    +
    +    @Test(timeout = 120000)
    +    public void testEphemeralNodeDeletion() throws Exception {
    +        final int clientPorts[] = new int[SERVER_COUNT];
    +        StringBuilder sb = new StringBuilder();
    +        String server;
    +
    +        for (int i = 0; i < SERVER_COUNT; i++) {
    +            clientPorts[i] = PortAssignment.unique();
    +            server = "server." + i + "=127.0.0.1:" + PortAssignment.unique()
    +                    + ":" + PortAssignment.unique() + ":participant;127.0.0.1:"
    +                    + clientPorts[i];
    +            sb.append(server + "\n");
    +        }
    +        String currentQuorumCfgSection = sb.toString();
    +        // start all the servers
    +        for (int i = 0; i < SERVER_COUNT; i++) {
    +            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection,
    +                    false) {
    +                @Override
    +                public TestQPMain getTestQPMain() {
    +                    return new MockTestQPMain();
    +                }
    +            };
    +            mt[i].start();
    +        }
    +
    +        // ensure all servers started
    +        for (int i = 0; i < SERVER_COUNT; i++) {
    +            Assert.assertTrue("waiting for server " + i + " being up",
    +                    ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i],
    +                            CONNECTION_TIMEOUT));
    +        }
    +
    +        CountdownWatcher watch = new CountdownWatcher();
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + clientPorts[1],
    +                ClientBase.CONNECTION_TIMEOUT, watch);
    +        watch.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
    +
    +        /**
    +         * now the problem scenario starts
    +         */
    +
    +        Stat firstEphemeralNode = new Stat();
    +
    +        // 1: create ephemeral node
    +        String nodePath = "/e1";
    +        zk.create(nodePath, "1".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.EPHEMERAL, firstEphemeralNode);
    +        assertEquals("Current session and ephemeral owner should be same",
    +                zk.getSessionId(), firstEphemeralNode.getEphemeralOwner());
    +
    +        // 2: inject network problem in one of the follower
    +        CustomQuorumPeer follower = (CustomQuorumPeer) getByServerState(mt,
    +                ServerState.FOLLOWING);
    +        follower.setInjectError(true);
    +
    +        // 3: close the session so that ephemeral node is deleted
    +        zk.close();
    +
    --- End diff --
    
    Yes the test case depends on these happened between starting of the partition (injected
fault) and ending of the partition. The key invariant I see here is to make sure that during
this period, the follower that currently disconnected with the quorum has gone through at
least one leader election cycle and also have gone through at least one SYNC cycle. Otherwise
the code path that this patch fixes will not be executed - i.e. there will be no errors during
syncing the DIFF and everything will just work as expected. We need trigger at least one fault
during the first DIFF sync cycle, and currently the test case is not doing this in a fine
grained way and potentially due to timing issue we can get flaky behavior.
    
    Though, I am not sure how to precisely control this - I think it will be a nice improvement
in test case to explicitly control the behavior as i just said, but this should not block
this patch landed.


> Ephemeral node is never deleted if follower fails while reading the proposal packet
> -----------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2355
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2355
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: quorum, server
>    Affects Versions: 3.4.8, 3.4.9, 3.4.10, 3.5.1, 3.5.2, 3.5.3
>            Reporter: Mohammad Arshad
>            Assignee: Mohammad Arshad
>            Priority: Critical
>         Attachments: ZOOKEEPER-2355-01.patch, ZOOKEEPER-2355-02.patch, ZOOKEEPER-2355-03.patch,
ZOOKEEPER-2355-04.patch, ZOOKEEPER-2355-05.patch
>
>
> ZooKeeper ephemeral node is never deleted if follower fail while reading the proposal
packet
> The scenario is as follows:
> # Configure three node ZooKeeper cluster, lets say nodes are A, B and C, start all, assume
A is leader, B and C are follower
> # Connect to any of the server and create ephemeral node /e1
> # Close the session, ephemeral node /e1 will go for deletion
> # While receiving delete proposal make Follower B to fail with {{SocketTimeoutException}}.
This we need to do to reproduce the scenario otherwise in production environment it happens
because of network fault.
> # Remove the fault, just check that faulted Follower is now connected with quorum
> # Connect to any of the server, create the same ephemeral node /e1, created is success.
> # Close the session,  ephemeral node /e1 will go for deletion
> # {color:red}/e1 is not deleted from the faulted Follower B, It should have been deleted
as it was again created with another session{color}
> # {color:green}/e1 is deleted from Leader A and other Follower C{color}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message