You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2016/10/05 02:11:21 UTC

[jira] [Commented] (APEXCORE-505) setup and activate calls in operator block heartbeat loop in container

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

ASF GitHub Bot commented on APEXCORE-505:
-----------------------------------------

GitHub user sandeshh opened a pull request:

    https://github.com/apache/apex-core/pull/405

    APEXCORE-505 Heartbeat loop was blocked waiting for operator activati…

    …on, the reason for this is that Stream activation(Only BufferServerSubscriber and WindowGenerator) waits for operator activation in a heartbeat thread. There is no need to have this synchronization, as Tuples are pulled from the queues by the operators.
    
    @vrozov @tweise please review

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

    $ git pull https://github.com/sandeshh/apex-core APEXCORE-505

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

    https://github.com/apache/apex-core/pull/405.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 #405
    
----
commit 596b996b1aa694bb9a0924f9094ef688df23678e
Author: Sandesh Hegde <sa...@gmail.com>
Date:   2016-10-05T01:29:57Z

    APEXCORE-505 Heartbeat loop was blocked waiting for operator activation, reason for this is that Stream activation(Only BufferServerSubscriber and WindowGenerator) waits for operator activation in heartbeat thread. After analysis and sanity testing, we don't see the need to have the synchronization between operator and stream activation.

----


> setup and activate calls in operator block heartbeat loop in container
> ----------------------------------------------------------------------
>
>                 Key: APEXCORE-505
>                 URL: https://issues.apache.org/jira/browse/APEXCORE-505
>             Project: Apache Apex Core
>          Issue Type: Bug
>            Reporter: Ashwin Chandra Putta
>            Assignee: Sandesh
>
> The setup and activate calls in the operators block the StreamingContainer heartbeat loop to send heartbeats to application master. As a result, if activation/setup takes more than the heartbeat timeout for any given operator within a container, the app master ends up killing the container for heartbeat timeout even though container is active.
> To test this: I created simple test application and added sleep for 40 seconds in setup or activate calls. The application master shows the heart beat timeout message and kills the container with operator. Please find the stack trace on the container while it was active as follows:
> sleep in activate callback
> ===================
> {code}
> Full thread dump Java HotSpot(TM) 64-Bit Server VM (24.65-b04 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x00007f8060ac0000 nid=0x6d2f waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "1/randomGenerator:RandomNumberGenerator" prio=10 tid=0x00007f8060b9b800 nid=0x65a2 waiting on condition [0x00007f8050bc8000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at com.datatorrent.sample.MyOperator.activate(MyOperator.java:63)
> 	at com.datatorrent.stram.engine.Node.activate(Node.java:619)
> 	at com.datatorrent.stram.engine.GenericNode.activate(GenericNode.java:205)
> 	at com.datatorrent.stram.engine.StreamingContainer.setupNode(StreamingContainer.java:1336)
> 	at com.datatorrent.stram.engine.StreamingContainer.access$100(StreamingContainer.java:130)
> 	at com.datatorrent.stram.engine.StreamingContainer$2.run(StreamingContainer.java:1396)
> "ProcessWideEventLoop" prio=10 tid=0x00007f8060afb800 nid=0x658b runnable [0x00007f8050cc9000]
>    java.lang.Thread.State: RUNNABLE
> 	at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
> 	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:87)
> 	- locked <0x00000007d186a380> (a com.datatorrent.netlet.OptimizedEventLoop$SelectedSelectionKeySet)
> 	- locked <0x00000007d185a4a0> (a java.util.Collections$UnmodifiableSet)
> 	- locked <0x00000007d185a070> (a sun.nio.ch.EPollSelectorImpl)
> 	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
> 	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:102)
> 	at com.datatorrent.netlet.OptimizedEventLoop.runEventLoop(OptimizedEventLoop.java:185)
> 	at com.datatorrent.netlet.OptimizedEventLoop.runEventLoop(OptimizedEventLoop.java:157)
> 	at com.datatorrent.netlet.DefaultEventLoop.run(DefaultEventLoop.java:156)
> 	at java.lang.Thread.run(Thread.java:745)
> "Dispatcher-0" daemon prio=10 tid=0x00007f8060ae8000 nid=0x6586 waiting on condition [0x00007f8050dca000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007dc5fa118> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
> 	at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
> 	at net.engio.mbassy.bus.AbstractSyncAsyncMessageBus$1.run(AbstractSyncAsyncMessageBus.java:51)
> 	at java.lang.Thread.run(Thread.java:745)
> "org.apache.hadoop.hdfs.PeerCache@389ed39" daemon prio=10 tid=0x00007f8060abf800 nid=0x6577 waiting on condition [0x00007f8050fcc000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at org.apache.hadoop.hdfs.PeerCache.run(PeerCache.java:255)
> 	at org.apache.hadoop.hdfs.PeerCache.access$000(PeerCache.java:46)
> 	at org.apache.hadoop.hdfs.PeerCache$1.run(PeerCache.java:124)
> 	at java.lang.Thread.run(Thread.java:745)
> "IPC Parameter Sending Thread #0" daemon prio=10 tid=0x00007f8060a2f800 nid=0x6572 waiting on condition [0x00007f80510cd000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007d511c0c0> (a java.util.concurrent.SynchronousQueue$TransferStack)
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:460)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:359)
> 	at java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:942)
> 	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> "Thread-2" daemon prio=10 tid=0x00007f80609dd000 nid=0x656e runnable [0x00007f80516f2000]
>    java.lang.Thread.State: RUNNABLE
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher.doPoll0(Native Method)
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher.access$900(DomainSocketWatcher.java:52)
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher$1.run(DomainSocketWatcher.java:474)
> 	at java.lang.Thread.run(Thread.java:745)
> "process reaper" daemon prio=10 tid=0x00007f80607c3000 nid=0x655d waiting on condition [0x00007f80580df000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007d2697c90> (a java.util.concurrent.SynchronousQueue$TransferStack)
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:460)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:359)
> 	at java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:942)
> 	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> "Service Thread" daemon prio=10 tid=0x00007f80600bd800 nid=0x6554 runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread1" daemon prio=10 tid=0x00007f80600bb000 nid=0x6553 waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread0" daemon prio=10 tid=0x00007f80600b8000 nid=0x6552 waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x00007f80600b6800 nid=0x6551 runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x00007f8060096800 nid=0x6550 in Object.wait() [0x00007f8052dec000]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	- waiting on <0x00000007d0005608> (a java.lang.ref.ReferenceQueue$Lock)
> 	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:135)
> 	- locked <0x00000007d0005608> (a java.lang.ref.ReferenceQueue$Lock)
> 	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:151)
> 	at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)
> "Reference Handler" daemon prio=10 tid=0x00007f8060094800 nid=0x654f in Object.wait() [0x00007f8052eed000]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	- waiting on <0x00000007d0005190> (a java.lang.ref.Reference$Lock)
> 	at java.lang.Object.wait(Object.java:503)
> 	at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:133)
> 	- locked <0x00000007d0005190> (a java.lang.ref.Reference$Lock)
> "main" prio=10 tid=0x00007f8060012000 nid=0x6538 waiting on condition [0x00007f806487d000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007ded02ed0> (a java.util.concurrent.CountDownLatch$Sync)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:994)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1303)
> 	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:236)
> 	at com.datatorrent.stram.engine.StreamingContainer.activate(StreamingContainer.java:1474)
> 	- locked <0x00000007dc538160> (a com.datatorrent.stram.engine.StreamingContainer)
> 	at com.datatorrent.stram.engine.StreamingContainer.deploy(StreamingContainer.java:874)
> 	- locked <0x00000007dc538160> (a com.datatorrent.stram.engine.StreamingContainer)
> 	at com.datatorrent.stram.engine.StreamingContainer.processHeartbeatResponse(StreamingContainer.java:814)
> 	at com.datatorrent.stram.engine.StreamingContainer.heartbeatLoop(StreamingContainer.java:707)
> 	at com.datatorrent.stram.engine.StreamingContainer.main(StreamingContainer.java:313)
> "VM Thread" prio=10 tid=0x00007f8060090800 nid=0x654e runnable 
> "GC task thread#0 (ParallelGC)" prio=10 tid=0x00007f8060028000 nid=0x653a runnable 
> "GC task thread#1 (ParallelGC)" prio=10 tid=0x00007f806002a000 nid=0x653b runnable 
> "GC task thread#2 (ParallelGC)" prio=10 tid=0x00007f806002c000 nid=0x653c runnable 
> "GC task thread#3 (ParallelGC)" prio=10 tid=0x00007f806002d800 nid=0x653d runnable 
> "GC task thread#4 (ParallelGC)" prio=10 tid=0x00007f806002f800 nid=0x653e runnable 
> "GC task thread#5 (ParallelGC)" prio=10 tid=0x00007f8060031800 nid=0x653f runnable 
> "GC task thread#6 (ParallelGC)" prio=10 tid=0x00007f8060033000 nid=0x6540 runnable 
> "GC task thread#7 (ParallelGC)" prio=10 tid=0x00007f8060035000 nid=0x6541 runnable 
> "GC task thread#8 (ParallelGC)" prio=10 tid=0x00007f8060037000 nid=0x6542 runnable 
> "GC task thread#9 (ParallelGC)" prio=10 tid=0x00007f8060039000 nid=0x6543 runnable 
> "GC task thread#10 (ParallelGC)" prio=10 tid=0x00007f806003a800 nid=0x6544 runnable 
> "GC task thread#11 (ParallelGC)" prio=10 tid=0x00007f806003c800 nid=0x6545 runnable 
> "GC task thread#12 (ParallelGC)" prio=10 tid=0x00007f806003e800 nid=0x6546 runnable 
> "GC task thread#13 (ParallelGC)" prio=10 tid=0x00007f8060040000 nid=0x6547 runnable 
> "GC task thread#14 (ParallelGC)" prio=10 tid=0x00007f8060042000 nid=0x6548 runnable 
> "GC task thread#15 (ParallelGC)" prio=10 tid=0x00007f8060044000 nid=0x6549 runnable 
> "GC task thread#16 (ParallelGC)" prio=10 tid=0x00007f8060045800 nid=0x654a runnable 
> "GC task thread#17 (ParallelGC)" prio=10 tid=0x00007f8060047800 nid=0x654b runnable 
> "VM Periodic Task Thread" prio=10 tid=0x00007f80600c8800 nid=0x6555 waiting on condition 
> JNI global references: 277
> {code}
> sleep in setup callback
> ===================
> {code}
> 2016-08-11 17:59:20
> Full thread dump Java HotSpot(TM) 64-Bit Server VM (24.65-b04 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x00007fda00ab8000 nid=0x22ea waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "1/randomGenerator:RandomNumberGenerator" prio=10 tid=0x00007fda00c08800 nid=0x1ffc waiting on condition [0x00007fd9f06a2000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at com.datatorrent.sample.MyOperator.setup(MyOperator.java:64)
> 	at com.datatorrent.sample.MyOperator.setup(MyOperator.java:25)
> 	at com.datatorrent.stram.engine.Node.setup(Node.java:187)
> 	at com.datatorrent.stram.engine.StreamingContainer.setupNode(StreamingContainer.java:1309)
> 	at com.datatorrent.stram.engine.StreamingContainer.access$100(StreamingContainer.java:130)
> 	at com.datatorrent.stram.engine.StreamingContainer$2.run(StreamingContainer.java:1396)
> "ProcessWideEventLoop" prio=10 tid=0x00007fda00af5800 nid=0x1fdb runnable [0x00007fd9f07a3000]
>    java.lang.Thread.State: RUNNABLE
> 	at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
> 	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:87)
> 	- locked <0x00000007d186a6e8> (a com.datatorrent.netlet.OptimizedEventLoop$SelectedSelectionKeySet)
> 	- locked <0x00000007d185a808> (a java.util.Collections$UnmodifiableSet)
> 	- locked <0x00000007d185a3d8> (a sun.nio.ch.EPollSelectorImpl)
> 	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
> 	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:102)
> 	at com.datatorrent.netlet.OptimizedEventLoop.runEventLoop(OptimizedEventLoop.java:185)
> 	at com.datatorrent.netlet.OptimizedEventLoop.runEventLoop(OptimizedEventLoop.java:157)
> 	at com.datatorrent.netlet.DefaultEventLoop.run(DefaultEventLoop.java:156)
> 	at java.lang.Thread.run(Thread.java:745)
> "Dispatcher-0" daemon prio=10 tid=0x00007fda00ae0000 nid=0x1fda waiting on condition [0x00007fd9f08a4000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007dc5fc5b0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
> 	at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
> 	at net.engio.mbassy.bus.AbstractSyncAsyncMessageBus$1.run(AbstractSyncAsyncMessageBus.java:51)
> 	at java.lang.Thread.run(Thread.java:745)
> "org.apache.hadoop.hdfs.PeerCache@762eefed" daemon prio=10 tid=0x00007fda00ab7800 nid=0x1fd7 waiting on condition [0x00007fd9f0aa6000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at org.apache.hadoop.hdfs.PeerCache.run(PeerCache.java:255)
> 	at org.apache.hadoop.hdfs.PeerCache.access$000(PeerCache.java:46)
> 	at org.apache.hadoop.hdfs.PeerCache$1.run(PeerCache.java:124)
> 	at java.lang.Thread.run(Thread.java:745)
> "IPC Parameter Sending Thread #0" daemon prio=10 tid=0x00007fda00a27800 nid=0x1fd5 waiting on condition [0x00007fd9f0ba7000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007d511c610> (a java.util.concurrent.SynchronousQueue$TransferStack)
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:460)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:359)
> 	at java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:942)
> 	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> "Thread-2" daemon prio=10 tid=0x00007fda009d4800 nid=0x1fd3 runnable [0x00007fd9f11cc000]
>    java.lang.Thread.State: RUNNABLE
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher.doPoll0(Native Method)
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher.access$900(DomainSocketWatcher.java:52)
> 	at org.apache.hadoop.net.unix.DomainSocketWatcher$1.run(DomainSocketWatcher.java:474)
> 	at java.lang.Thread.run(Thread.java:745)
> "process reaper" daemon prio=10 tid=0x00007fda007b3000 nid=0x1fc5 waiting on condition [0x00007fd9f8090000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007d2698208> (a java.util.concurrent.SynchronousQueue$TransferStack)
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:460)
> 	at java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:359)
> 	at java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:942)
> 	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> "Service Thread" daemon prio=10 tid=0x00007fda000bd800 nid=0x1fb2 runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread1" daemon prio=10 tid=0x00007fda000bb000 nid=0x1fb1 waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "C2 CompilerThread0" daemon prio=10 tid=0x00007fda000b8000 nid=0x1fb0 waiting on condition [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x00007fda000b6800 nid=0x1faf runnable [0x0000000000000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x00007fda00096800 nid=0x1fad in Object.wait() [0x00007fd9f28c6000]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	- waiting on <0x00000007d0005608> (a java.lang.ref.ReferenceQueue$Lock)
> 	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:135)
> 	- locked <0x00000007d0005608> (a java.lang.ref.ReferenceQueue$Lock)
> 	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:151)
> 	at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)
> "Reference Handler" daemon prio=10 tid=0x00007fda00094800 nid=0x1fac in Object.wait() [0x00007fd9f29c7000]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	- waiting on <0x00000007d0005190> (a java.lang.ref.Reference$Lock)
> 	at java.lang.Object.wait(Object.java:503)
> 	at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:133)
> 	- locked <0x00000007d0005190> (a java.lang.ref.Reference$Lock)
> "main" prio=10 tid=0x00007fda00012000 nid=0x1f98 waiting on condition [0x00007fda042f0000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007ded06778> (a java.util.concurrent.CountDownLatch$Sync)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:994)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1303)
> 	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:236)
> 	at com.datatorrent.stram.engine.StreamingContainer.activate(StreamingContainer.java:1474)
> 	- locked <0x00000007dc53a5f8> (a com.datatorrent.stram.engine.StreamingContainer)
> 	at com.datatorrent.stram.engine.StreamingContainer.deploy(StreamingContainer.java:874)
> 	- locked <0x00000007dc53a5f8> (a com.datatorrent.stram.engine.StreamingContainer)
> 	at com.datatorrent.stram.engine.StreamingContainer.processHeartbeatResponse(StreamingContainer.java:814)
> 	at com.datatorrent.stram.engine.StreamingContainer.heartbeatLoop(StreamingContainer.java:707)
> 	at com.datatorrent.stram.engine.StreamingContainer.main(StreamingContainer.java:313)
> "VM Thread" prio=10 tid=0x00007fda00090800 nid=0x1fab runnable 
> "GC task thread#0 (ParallelGC)" prio=10 tid=0x00007fda00028000 nid=0x1f99 runnable 
> "GC task thread#1 (ParallelGC)" prio=10 tid=0x00007fda0002a000 nid=0x1f9a runnable 
> "GC task thread#2 (ParallelGC)" prio=10 tid=0x00007fda0002c000 nid=0x1f9b runnable 
> "GC task thread#3 (ParallelGC)" prio=10 tid=0x00007fda0002d800 nid=0x1f9c runnable 
> "GC task thread#4 (ParallelGC)" prio=10 tid=0x00007fda0002f800 nid=0x1f9d runnable 
> "GC task thread#5 (ParallelGC)" prio=10 tid=0x00007fda00031800 nid=0x1f9e runnable 
> "GC task thread#6 (ParallelGC)" prio=10 tid=0x00007fda00033000 nid=0x1f9f runnable 
> "GC task thread#7 (ParallelGC)" prio=10 tid=0x00007fda00035000 nid=0x1fa0 runnable 
> "GC task thread#8 (ParallelGC)" prio=10 tid=0x00007fda00037000 nid=0x1fa1 runnable 
> "GC task thread#9 (ParallelGC)" prio=10 tid=0x00007fda00039000 nid=0x1fa2 runnable 
> "GC task thread#10 (ParallelGC)" prio=10 tid=0x00007fda0003a800 nid=0x1fa3 runnable 
> "GC task thread#11 (ParallelGC)" prio=10 tid=0x00007fda0003c800 nid=0x1fa4 runnable 
> "GC task thread#12 (ParallelGC)" prio=10 tid=0x00007fda0003e800 nid=0x1fa5 runnable 
> "GC task thread#13 (ParallelGC)" prio=10 tid=0x00007fda00040000 nid=0x1fa6 runnable 
> "GC task thread#14 (ParallelGC)" prio=10 tid=0x00007fda00042000 nid=0x1fa7 runnable 
> "GC task thread#15 (ParallelGC)" prio=10 tid=0x00007fda00044000 nid=0x1fa8 runnable 
> "GC task thread#16 (ParallelGC)" prio=10 tid=0x00007fda00045800 nid=0x1fa9 runnable 
> "GC task thread#17 (ParallelGC)" prio=10 tid=0x00007fda00047800 nid=0x1faa runnable 
> "VM Periodic Task Thread" prio=10 tid=0x00007fda000d0800 nid=0x1fb3 waiting on condition 
> JNI global references: 237
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)