From commits-return-6460-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Mon Jun 25 12:30:50 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id B1B46180627 for ; Mon, 25 Jun 2018 12:30:49 +0200 (CEST) Received: (qmail 39146 invoked by uid 500); 25 Jun 2018 10:30:48 -0000 Mailing-List: contact commits-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 commits@zookeeper.apache.org Received: (qmail 39134 invoked by uid 99); 25 Jun 2018 10:30:48 -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, 25 Jun 2018 10:30:48 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id AB536E09EE; Mon, 25 Jun 2018 10:30:48 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: andor@apache.org To: commits@zookeeper.apache.org Message-Id: <21368490ce9f46fa9e55265beeeb3e9c@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: zookeeper git commit: ZOOKEEPER-3059: EventThread leak in case of Sasl AuthFailed Date: Mon, 25 Jun 2018 10:30:48 +0000 (UTC) Repository: zookeeper Updated Branches: refs/heads/branch-3.5 6748a0e3f -> f81fb443f ZOOKEEPER-3059: EventThread leak in case of Sasl AuthFailed Since authFailed is similar to session expired and is considered a fatal event, we should clean up after ourselves once we get a AuthFailed, other wise this results in an unavoidable and un-cleanable thread leak of EventThread since the close operation is also a no-op (we return after checking for isAlive). Author: Abhishek Singh Chouhan Reviewers: Andor Molnar Closes #541 from abhishek-chouhan/master and squashes the following commits: c54a83a4 [Abhishek Singh Chouhan] ZOOKEEPER-3059 EventThread leak in case of Sasl AuthFailed. Adding testcase for the scenario c1d9d7af [Abhishek Singh Chouhan] ZOOKEEPER-3059 EventThread leak in case of Sasl AuthFailed (cherry picked from commit 1fb644662b8e0530dec2c5668a3e49b3f614e9de) Signed-off-by: Andor Molnar Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/f81fb443 Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/f81fb443 Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/f81fb443 Branch: refs/heads/branch-3.5 Commit: f81fb443f772d363f411b356048e949cd90bcc3a Parents: 6748a0e Author: Abhishek Singh Chouhan Authored: Mon Jun 25 12:30:19 2018 +0200 Committer: Andor Molnar Committed: Mon Jun 25 12:30:42 2018 +0200 ---------------------------------------------------------------------- .../main/org/apache/zookeeper/ClientCnxn.java | 6 ++- .../test/org/apache/zookeeper/SaslAuthTest.java | 45 ++++++++++++++++++-- 2 files changed, 47 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/zookeeper/blob/f81fb443/src/java/main/org/apache/zookeeper/ClientCnxn.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 2eef575..ba601bc 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -820,7 +820,8 @@ public class ClientCnxn { if(replyHdr.getErr() == KeeperException.Code.AUTHFAILED.intValue()) { state = States.AUTH_FAILED; eventThread.queueEvent( new WatchedEvent(Watcher.Event.EventType.None, - Watcher.Event.KeeperState.AuthFailed, null) ); + Watcher.Event.KeeperState.AuthFailed, null) ); + eventThread.queueEventOfDeath(); } if (LOG.isDebugEnabled()) { LOG.debug("Got auth sessionid:0x" @@ -1164,6 +1165,9 @@ public class ClientCnxn { eventThread.queueEvent(new WatchedEvent( Watcher.Event.EventType.None, authState,null)); + if (state == States.AUTH_FAILED) { + eventThread.queueEventOfDeath(); + } } } to = readTimeout - clientCnxnSocket.getIdleRecv(); http://git-wip-us.apache.org/repos/asf/zookeeper/blob/f81fb443/src/java/test/org/apache/zookeeper/SaslAuthTest.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/SaslAuthTest.java b/src/java/test/org/apache/zookeeper/SaslAuthTest.java index eac0703..088fe1f 100644 --- a/src/java/test/org/apache/zookeeper/SaslAuthTest.java +++ b/src/java/test/org/apache/zookeeper/SaslAuthTest.java @@ -26,8 +26,10 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.zookeeper.ClientCnxn.EventThread; import org.apache.zookeeper.ClientCnxn.SendThread; import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooDefs.Ids; @@ -88,7 +90,7 @@ public class SaslAuthTest extends ClientBase { System.clearProperty("java.security.auth.login.config"); } - private AtomicInteger authFailed = new AtomicInteger(0); + private final CountDownLatch authFailed = new CountDownLatch(1); @Override protected TestableZooKeeper createClient(String hp) @@ -102,7 +104,7 @@ public class SaslAuthTest extends ClientBase { @Override public synchronized void process(WatchedEvent event) { if (event.getState() == KeeperState.AuthFailed) { - authFailed.incrementAndGet(); + authFailed.countDown(); } else { super.process(event); @@ -210,4 +212,41 @@ public class SaslAuthTest extends ClientBase { saslLoginFailedField.setBoolean(sendThread, true); } + @Test + public void testThreadsShutdownOnAuthFailed() throws Exception { + MyWatcher watcher = new MyWatcher(); + ZooKeeper zk = null; + try { + zk = new ZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher); + watcher.waitForConnected(CONNECTION_TIMEOUT); + try { + zk.addAuthInfo("FOO", "BAR".getBytes()); + zk.getData("/path1", false, null); + Assert.fail("Should get auth state error"); + } catch (KeeperException.AuthFailedException e) { + if (!authFailed.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)) { + Assert.fail("Should have called my watcher"); + } + } + Field cnxnField = zk.getClass().getDeclaredField("cnxn"); + cnxnField.setAccessible(true); + ClientCnxn clientCnxn = (ClientCnxn) cnxnField.get(zk); + Field sendThreadField = clientCnxn.getClass().getDeclaredField("sendThread"); + sendThreadField.setAccessible(true); + SendThread sendThread = (SendThread) sendThreadField.get(clientCnxn); + Field eventThreadField = clientCnxn.getClass().getDeclaredField("eventThread"); + eventThreadField.setAccessible(true); + EventThread eventThread = (EventThread) eventThreadField.get(clientCnxn); + sendThread.join(CONNECTION_TIMEOUT); + eventThread.join(CONNECTION_TIMEOUT); + Assert.assertFalse("SendThread did not shutdown after authFail", sendThread.isAlive()); + Assert.assertFalse("EventThread did not shutdown after authFail", + eventThread.isAlive()); + } finally { + if (zk != null) { + zk.close(); + } + } + } + }