Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 28892200C92 for ; Mon, 12 Jun 2017 08:52:37 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 272BB160BD9; Mon, 12 Jun 2017 06:52:37 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 46B14160BD6 for ; Mon, 12 Jun 2017 08:52:36 +0200 (CEST) Received: (qmail 85426 invoked by uid 500); 12 Jun 2017 06:52:35 -0000 Mailing-List: contact dev-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list dev@zookeeper.apache.org Received: (qmail 85415 invoked by uid 99); 12 Jun 2017 06:52:34 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 12 Jun 2017 06:52:34 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C64D8E381C; Mon, 12 Jun 2017 06:52:34 +0000 (UTC) From: JiangJiafu To: dev@zookeeper.apache.org Reply-To: dev@zookeeper.apache.org References: In-Reply-To: Subject: [GitHub] zookeeper pull request #112: ZOOKEEPER-2355:Ephemeral node is never deleted ... Content-Type: text/plain Message-Id: <20170612065234.C64D8E381C@git1-us-west.apache.org> Date: Mon, 12 Jun 2017 06:52:34 +0000 (UTC) archived-at: Mon, 12 Jun 2017 06:52:37 -0000 Github user JiangJiafu commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/112#discussion_r121320033 --- 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 -- Can this code make sure that the problem scenario happen again? When zk.close() is called, the leader should send closeSession request to all followers, since the error is injected, readPacket of the follower will throw an exception, which cause the follower to exist and begin to look for new leader. Is there any chance that the before the follower.setInjectError(false) is called bebofe makeFollower() is called? If the answer is yes, then the problem scenario does not happen. According to the patch, the problem will happen when follower fails to sync with leader while the peerLastZxid is updated to a new one. I am not sure about this, I hope I'm wrong. --- 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. ---