Repository: kafka Updated Branches: refs/heads/trunk 4d0759713 -> 0607953f3
kafka-1574; unit tests can hang on socketserver shutdown; patched by Jun Rao; reviewed by Jay Kreps and Guozhang Wang Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0607953f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0607953f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0607953f Branch: refs/heads/trunk Commit: 0607953f3ad5a8a77bdd83870897541af66c7ced Parents: 4d07597 Author: Jun Rao <[email protected]> Authored: Mon Aug 11 07:45:50 2014 -0700 Committer: Jun Rao <[email protected]> Committed: Mon Aug 11 07:45:50 2014 -0700 ---------------------------------------------------------------------- core/src/main/scala/kafka/network/SocketServer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kafka/blob/0607953f/core/src/main/scala/kafka/network/SocketServer.scala ---------------------------------------------------------------------- diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 8e99de0..9693bc0 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -102,7 +102,7 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ protected val selector = Selector.open(); private val startupLatch = new CountDownLatch(1) private val shutdownLatch = new CountDownLatch(1) - private val alive = new AtomicBoolean(false) + private val alive = new AtomicBoolean(true) /** * Initiates a graceful shutdown by signaling to stop and waiting for the shutdown to complete @@ -122,7 +122,6 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ * Record that the thread startup is complete */ protected def startupComplete() = { - alive.set(true) startupLatch.countDown }
