kafka-jira 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] (KAFKA-3575) Use console consumer access topic that does not exist, can not use "Control + C" to exit process
Date Wed, 21 Jun 2017 11:19:00 GMT

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

ASF GitHub Bot commented on KAFKA-3575:
---------------------------------------

GitHub user tombentley opened a pull request:

    https://github.com/apache/kafka/pull/3395

    KAFKA-3575: Use console consumer access topic that does not exist, ca…

    …n not use "Control + C" to exit process
    
    A finally block is not guaranteed to execute in the event of Ctrl+C happening
    while in the try or catch blocks. Decrementing the latch in the finally block
    therefore made the shutdown hook hang waiting for something that would
    never happen and the JVM couldn't exit while it was running the shutdown hook.
    
    Replacing the latch with an atomic flag to say whether we've run the cleanup
    code allows us to either run it from the shutdown hook, or the finally block.
    It should thus definitely run once. When run from the shutdown hook the main
    thread would no longer be running, so it should be threadsafe.
    
    The contribution is my original work and I license the work to the project under the project's
open source license.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tombentley/kafka KAFKA-3575

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/3395.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3395
    
----
commit 66b1ff6d37d0eab36e00f1251d0463a7b13cd11d
Author: Tom Bentley <tbentley@redhat.com>
Date:   2017-06-21T11:11:27Z

    KAFKA-3575: Use console consumer access topic that does not exist, can not use "Control
+ C" to exit process
    
    A finally block is not guaranteed to execute in the event of Ctrl+C happening
    while in the try or catch blocks. Decrementing the latch in the finally block
    therefore made the shutdown hook hang waiting for something that would
    never happen and the JVM couldn't exit while it was running the shutdown hook.
    
    Replacing the latch with an atomic flag to say whether we've run the cleanup
    code allows us to either run it from the shutdown hook, or the finally block.
    It should thus definitely run once. When run from the shutdown hook the main
    thread would no longer be running, so it should be threadsafe.

----


> Use console consumer access topic that does not exist, can not use "Control + C" to exit
process
> ------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-3575
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3575
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.9.0.0
>         Environment: SUSE Linux Enterprise Server 11 SP3
>            Reporter: NieWang
>            Priority: Minor
>
> 1.  use "sh kafka-console-consumer.sh --zookeeper 10.252.23.133:2181 --topic topic_02"
 start console consumer. topic_02 does not exist.
> 2. you can not use "Control + C" to exit console consumer process. The process is blocked.
> 3. use jstack check process stack, as follows:
> linux:~ # jstack 122967
> 2016-04-18 15:46:06
> Full thread dump Java HotSpot(TM) 64-Bit Server VM (25.66-b17 mixed mode):
> "Attach Listener" #29 daemon prio=9 os_prio=0 tid=0x0000000001781800 nid=0x1e0c8 waiting
on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Thread-4" #27 prio=5 os_prio=0 tid=0x00000000018a4000 nid=0x1e08a waiting on condition
[0x00007ffbe5ac0000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e00ed3b8> (a java.util.concurrent.CountDownLatch$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
>         at kafka.tools.ConsoleConsumer$$anon$1.run(ConsoleConsumer.scala:101)
> "SIGINT handler" #28 daemon prio=9 os_prio=0 tid=0x00000000019d5800 nid=0x1e089 in Object.wait()
[0x00007ffbe5bc1000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.$$YJP$$wait(Native Method)
>         at java.lang.Object.wait(Object.java)
>         at java.lang.Thread.join(Thread.java:1245)
>         - locked <0x00000000e71fd4e8> (a kafka.tools.ConsoleConsumer$$anon$1)
>         at java.lang.Thread.join(Thread.java:1319)
>         at java.lang.ApplicationShutdownHooks.runHooks(ApplicationShutdownHooks.java:106)
>         at java.lang.ApplicationShutdownHooks$1.run(ApplicationShutdownHooks.java:46)
>         at java.lang.Shutdown.runHooks(Shutdown.java:123)
>         at java.lang.Shutdown.sequence(Shutdown.java:167)
>         at java.lang.Shutdown.exit(Shutdown.java:212)
>         - locked <0x00000000e00abfd8> (a java.lang.Class for java.lang.Shutdown)
>         at java.lang.Terminator$1.handle(Terminator.java:52)
>         at sun.misc.Signal$1.run(Signal.java:212)
>         at java.lang.Thread.run(Thread.java:745)
> "metrics-meter-tick-thread-2" #20 daemon prio=5 os_prio=0 tid=0x00007ffbec77a800 nid=0x1e079
waiting on condition [0x00007ffbe66c8000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e6fa6438> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1088)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1067)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1127)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> "metrics-meter-tick-thread-1" #19 daemon prio=5 os_prio=0 tid=0x00007ffbec783000 nid=0x1e078
waiting on condition [0x00007ffbe67c9000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e6fa6438> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1081)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1067)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1127)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> "main-EventThread" #17 daemon prio=5 os_prio=0 tid=0x00007ffbe08c8800 nid=0x1e077 waiting
on condition [0x00007ffbe6aca000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e6fa66a8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:494)
> "main-SendThread(10.252.23.133:2181)" #16 daemon prio=5 os_prio=0 tid=0x00007ffbe08c1000
nid=0x1e076 runnable [0x00007ffbe6bcb000]
>    java.lang.Thread.State: RUNNABLE
>         at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method)
>         at sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java)
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked <0x00000000e6fa6958> (a sun.nio.ch.Util$2)
>         - locked <0x00000000e6fa6968> (a java.util.Collections$UnmodifiableSet)
>         - locked <0x00000000e6fa6910> (a sun.nio.ch.EPollSelectorImpl)
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:349)
>         at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
> "ZkClient-EventThread-15-10.252.23.133:2181" #15 daemon prio=5 os_prio=0 tid=0x00007ffbe0812800
nid=0x1e075 waiting on condition [0x00007ffbe6f0b000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e6e0eae0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:67)
> "Service Thread" #13 daemon prio=9 os_prio=0 tid=0x00007ffbe008c800 nid=0x1e072 runnable
[0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C1 CompilerThread2" #12 daemon prio=9 os_prio=0 tid=0x00007ffbe0054000 nid=0x1e071 waiting
on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread1" #11 daemon prio=9 os_prio=0 tid=0x0000000000f59000 nid=0x1e070 waiting
on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread0" #10 daemon prio=9 os_prio=0 tid=0x00007ffbe0047000 nid=0x1e06f waiting
on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "YJPAgent-OOMESnapshotDetector" #9 daemon prio=10 os_prio=0 tid=0x00007ffbec1dc000 nid=0x1e06e
runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "YJPAgent-CPUSampler" #8 daemon prio=10 os_prio=0 tid=0x00007ffbec1db800 nid=0x1e06d
runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "YJPAgent-RequestListener" #7 daemon prio=1 os_prio=0 tid=0x00007ffbec1da800 nid=0x1e06c
runnable [0x00007ffbea0fd000]
>    java.lang.Thread.State: RUNNABLE
>         at java.net.PlainSocketImpl.$$YJP$$socketAccept(Native Method)
>         at java.net.PlainSocketImpl.socketAccept(PlainSocketImpl.java)
>         at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
>         at java.net.ServerSocket.implAccept(ServerSocket.java:545)
>         at java.net.ServerSocket.accept(ServerSocket.java:513)
>         at com.yourkit.runtime.Core$4.run(Core.java:726)
>         at java.lang.Thread.run(Thread.java:745)
> "YJPAgent-Telemetry" #6 daemon prio=5 os_prio=0 tid=0x00007ffbec1ce800 nid=0x1e06b waiting
on condition [0x00007ffbea1fe000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.$$YJP$$sleep(Native Method)
>         at java.lang.Thread.sleep(Thread.java)
>         at com.yourkit.util.Util.sleep(Util.java:64)
>         at com.yourkit.runtime.TelemetryThread.run(TelemetryThread.java:579)
> "Signal Dispatcher" #5 daemon prio=9 os_prio=0 tid=0x0000000000d66000 nid=0x1e06a runnable
[0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (Concurrent GC)" #4 daemon prio=9 os_prio=0 tid=0x0000000000dfc800
nid=0x1e069 waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" #3 daemon prio=8 os_prio=0 tid=0x0000000000c0a000 nid=0x1e068 in Object.wait()
[0x00007ffbea81f000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.$$YJP$$wait(Native Method)
>         at java.lang.Object.wait(Object.java)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)
>         - locked <0x00000000e00c6d50> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)
> "Reference Handler" #2 daemon prio=10 os_prio=0 tid=0x0000000000bf9000 nid=0x1e067 in
Object.wait() [0x00007ffbea920000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.$$YJP$$wait(Native Method)
>         at java.lang.Object.wait(Object.java)
>         at java.lang.Object.wait(Object.java:502)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:157)
>         - locked <0x00000000e00c6f08> (a java.lang.ref.Reference$Lock)
> "main" #1 prio=5 os_prio=0 tid=0x000000000079a800 nid=0x1e05a waiting on condition [0x00007ffc034b4000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000000e6f510e0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:63)
>         at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33)
>         at kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:66)
>         at kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:58)
>         at kafka.consumer.OldConsumer.receive(BaseConsumer.scala:79)
>         at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:110)
>         at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69)
>         at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:47)
>         at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
> "VM Thread" os_prio=0 tid=0x0000000000be9000 nid=0x1e066 runnable 
> "Gang worker#0 (Parallel GC Threads)" os_prio=0 tid=0x00000000007e6800 nid=0x1e05b runnable

> "Gang worker#1 (Parallel GC Threads)" os_prio=0 tid=0x00000000007e8000 nid=0x1e05c runnable

> "Gang worker#2 (Parallel GC Threads)" os_prio=0 tid=0x00000000007e9800 nid=0x1e05d runnable

> "Gang worker#3 (Parallel GC Threads)" os_prio=0 tid=0x00000000007eb000 nid=0x1e05e runnable

> "G1 Main Concurrent Mark GC Thread" os_prio=0 tid=0x0000000000820000 nid=0x1e064 runnable

> "Gang worker#0 (G1 Parallel Marking Threads)" os_prio=0 tid=0x0000000000822000 nid=0x1e065
runnable 
> "G1 Concurrent Refinement Thread#0" os_prio=0 tid=0x00000000007f5000 nid=0x1e063 runnable

> "G1 Concurrent Refinement Thread#1" os_prio=0 tid=0x00000000007f3000 nid=0x1e062 runnable

> "G1 Concurrent Refinement Thread#2" os_prio=0 tid=0x00000000007f1800 nid=0x1e061 runnable

> "G1 Concurrent Refinement Thread#3" os_prio=0 tid=0x00000000007ef800 nid=0x1e060 runnable

> "G1 Concurrent Refinement Thread#4" os_prio=0 tid=0x00000000007ed800 nid=0x1e05f runnable

> "VM Periodic Task Thread" os_prio=0 tid=0x00007ffbec1fc800 nid=0x1e073 waiting on condition

> JNI global references: 7020



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

Mime
View raw message