From 381889da3624758b8c76b7329a39c082e54b1524 Mon Sep 17 00:00:00 2001 From: Brian Nixon Date: Fri, 21 Jun 2019 16:01:51 -0700 Subject: [PATCH 1/3] ZOOKEEPER-3240: Close socket on Learner shutdown to avoid dangling socket --- .../org/apache/zookeeper/server/quorum/Follower.java | 6 +----- .../org/apache/zookeeper/server/quorum/Learner.java | 11 +++++++++++ .../org/apache/zookeeper/server/quorum/Observer.java | 6 +----- .../zookeeper/server/quorum/QuorumPeerMainTest.java | 3 +++ 4 files changed, 16 insertions(+), 10 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 1d809d20b3f..90626ac5b3d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -115,11 +115,7 @@ void followLeader() throws InterruptedException { } } catch (Exception e) { LOG.warn("Exception when following the leader", e); - try { - sock.close(); - } catch (IOException e1) { - e1.printStackTrace(); - } + closeSocket(); // clear pending revalidations pendingRevalidations.clear(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index 64ff0fb9ef9..f37dbeb9c3f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -673,6 +673,7 @@ public void shutdown() { self.setZooKeeperServer(null); self.closeAllConnections(); self.adminServer.setZooKeeperServer(null); + closeSocket(); // shutdown previous zookeeper if (zk != null) { zk.shutdown(); @@ -682,4 +683,14 @@ public void shutdown() { boolean isRunning() { return self.isRunning() && zk.isRunning(); } + + void closeSocket() { + try { + if (sock != null && !sock.isClosed()) { + sock.close(); + } + } catch (IOException e) { + LOG.warn("Ignoring error closing connection to leader", e); + } + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java index 6e84128c5ee..907aba87562 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java @@ -122,11 +122,7 @@ void observeLeader() throws Exception { } } catch (Exception e) { LOG.warn("Exception when observing the leader", e); - try { - sock.close(); - } catch (IOException e1) { - e1.printStackTrace(); - } + closeSocket(); // clear pending revalidations pendingRevalidations.clear(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index 146773bfbf1..9c4d276d3e5 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -827,7 +827,10 @@ public void testFailedTxnAsPartOfQuorumLoss() throws Exception { int leader = servers.findLeader(); Map outstanding = servers.mt[leader].main.quorumPeer.leader.outstandingProposals; // increase the tick time to delay the leader going to looking + int previousTick = servers.mt[leader].main.quorumPeer.tickTime; servers.mt[leader].main.quorumPeer.tickTime = LEADER_TIMEOUT_MS; + // let the previous tick on the leader exhaust itself so the new tick time takes effect + Thread.sleep(previousTick); LOG.warn("LEADER {}", leader); for (int i = 0; i < SERVER_COUNT; i++) { From 1598b3ad8ea3320a98eeb1e87f4ec21e5765df18 Mon Sep 17 00:00:00 2001 From: Brian Nixon Date: Fri, 28 Jun 2019 16:05:11 -0700 Subject: [PATCH 2/3] remove extraneous check on socket status --- .../main/java/org/apache/zookeeper/server/quorum/Learner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index f37dbeb9c3f..51979aabb5c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -686,7 +686,7 @@ boolean isRunning() { void closeSocket() { try { - if (sock != null && !sock.isClosed()) { + if (sock != null) { sock.close(); } } catch (IOException e) { From 57f5e891ce1b8375138dfa90b043be5178fe9bff Mon Sep 17 00:00:00 2001 From: Brian Nixon Date: Fri, 28 Jun 2019 15:58:52 -0700 Subject: [PATCH 3/3] delete ReconfigFailureCasesTest::testLeaderTimesoutOnNewQuorum as it relies on the fixed buggy behavior --- .../quorum/ReconfigFailureCasesTest.java | 43 ------------------- 1 file changed, 43 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java index a0b9760ebb6..bd9e5881c25 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java @@ -159,49 +159,6 @@ public void testReconfigVersionConditionFails() throws Exception { ReconfigTest.closeAllHandles(zkArr, zkAdminArr); } - /* - * Tests that if a quorum of a new config is synced with the leader and a reconfig - * is allowed to start but then the new quorum is lost, the leader will time out and - * we go to leader election. - */ - @Test - public void testLeaderTimesoutOnNewQuorum() throws Exception { - qu = new QuorumUtil(1); // create 3 servers - qu.disableJMXTest = true; - qu.startAll(); - ZooKeeper[] zkArr = ReconfigTest.createHandles(qu); - ZooKeeperAdmin[] zkAdminArr = ReconfigTest.createAdminHandles(qu); - - List leavingServers = new ArrayList(); - leavingServers.add("3"); - qu.shutdown(2); - try { - // Since we just shut down server 2, its still considered "synced" - // by the leader, which allows us to start the reconfig - // (PrepRequestProcessor checks that a quorum of the new - // config is synced before starting a reconfig). - // We try to remove server 3, which requires a quorum of {1,2,3} - // (we have that) and of {1,2}, but 2 is down so we won't get a - // quorum of new config ACKs. - zkAdminArr[1].reconfigure(null, leavingServers, null, -1, null); - Assert.fail("Reconfig should have failed since we don't have quorum of new config"); - } catch (KeeperException.ConnectionLossException e) { - // We expect leader to lose quorum of proposed config and time out - } catch (Exception e) { - Assert.fail("Should have been ConnectionLossException!"); - } - - // The leader should time out and remaining servers should go into - // LOOKING state. A new leader won't be established since that - // would require completing the reconfig, which is not possible while - // 2 is down. - Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, - qu.getPeer(1).peer.getServerState()); - Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, - qu.getPeer(3).peer.getServerState()); - ReconfigTest.closeAllHandles(zkArr, zkAdminArr); - } - /* * Converting an observer into a participant may sometimes fail with a * NewConfigNoQuorum exception. This test-case demonstrates the scenario.