Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Incomplete
-
5.8.0
-
None
-
None
-
Windows 2008 R2, JDK x64 1.7.0_21
Description
We are experiencing a situation where a broker deadlocks when starting a duplex bridge and the startup interferes with message delivery (I suppose). The broker is running embedded in Glassfish 3.1.2.2.
Thread Execution Information: ----------------------- Thread "ActiveMQ Transport: tcp://xx/a.b.c.d:61616@61883" thread-id: 108 thread-state: WAITING Waiting on lock: java.util.concurrent.locks.ReentrantLock$NonfairSync@462eb43f at: sun.misc.Unsafe.park(Native Method) 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.acquireQueued(AbstractQueuedSynchronizer.java:867) at: java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at: java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at: java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at: org.apache.activemq.transport.MutexTransport.oneway(MutexTransport.java:66) at: org.apache.activemq.transport.ResponseCorrelator.oneway(ResponseCorrelator.java:60) at: org.apache.activemq.broker.TransportConnection.dispatch(TransportConnection.java:1378) at: org.apache.activemq.broker.TransportConnection.processDispatch(TransportConnection.java:897) at: org.apache.activemq.broker.TransportConnection.dispatchSync(TransportConnection.java:857) at: org.apache.activemq.broker.TransportConnection$1.onCommand(TransportConnection.java:186) at: org.apache.activemq.transport.ResponseCorrelator.onCommand(ResponseCorrelator.java:116) at: org.apache.activemq.transport.MutexTransport.onCommand(MutexTransport.java:50) at: org.apache.activemq.transport.vm.VMTransport.doDispatch(VMTransport.java:138) at: org.apache.activemq.transport.vm.VMTransport.dispatch(VMTransport.java:127) at: org.apache.activemq.transport.vm.VMTransport.oneway(VMTransport.java:104) at: org.apache.activemq.transport.MutexTransport.oneway(MutexTransport.java:68) at: org.apache.activemq.transport.ResponseCorrelator.asyncRequest(ResponseCorrelator.java:81) at: org.apache.activemq.transport.ResponseCorrelator.request(ResponseCorrelator.java:86) at: org.apache.activemq.network.DemandForwardingBridgeSupport.addSubscription(DemandForwardingBridgeSupport.java:902) at: org.apache.activemq.network.DemandForwardingBridgeSupport.addConsumerInfo(DemandForwardingBridgeSupport.java:1183) at: org.apache.activemq.network.DemandForwardingBridgeSupport.serviceRemoteConsumerAdvisory(DemandForwardingBridgeSupport.java:763) at: org.apache.activemq.network.DemandForwardingBridgeSupport.serviceRemoteCommand(DemandForwardingBridgeSupport.java:614) at: org.apache.activemq.network.DemandForwardingBridgeSupport$3.onCommand(DemandForwardingBridgeSupport.java:219) at: org.apache.activemq.transport.ResponseCorrelator.onCommand(ResponseCorrelator.java:116) at: org.apache.activemq.transport.MutexTransport.onCommand(MutexTransport.java:50) at: org.apache.activemq.transport.WireFormatNegotiator.onCommand(WireFormatNegotiator.java:113) at: org.apache.activemq.transport.AbstractInactivityMonitor.onCommand(AbstractInactivityMonitor.java:288) at: org.apache.activemq.transport.TransportSupport.doConsume(TransportSupport.java:83) at: org.apache.activemq.transport.tcp.TcpTransport.doRun(TcpTransport.java:214) at: org.apache.activemq.transport.tcp.TcpTransport.run(TcpTransport.java:196) at: java.lang.Thread.run(Thread.java:722) Thread Synchronization Statistics: ----------------------- This thread is blocked waiting on lock owned currently by thread named: ActiveMQ BrokerService[iconQR1] Task-69, id: 1.118 Number of times this thread was blocked (to enter/reenter a Monitor): 3 Number of times this thread waited for a notification (i.e. it was in WAITING or TIMED_WAITING state): 2 Total CPU time for this thread: 0 seconds 265.625.000 nanoseconds. User-level CPU time for this thread: 0 seconds 265.625.000 nanoseconds. Object Monitors currently held or requested by this thread: [] Ownable Synchronizers (e.g. ReentrantLock and ReentrantReadWriteLock) held by this thread: [] Thread Execution Information: ----------------------- Thread "ActiveMQ BrokerService[iconQR1] Task-69" thread-id: 1.118 thread-state: WAITING Waiting on lock: java.util.concurrent.locks.ReentrantLock$NonfairSync@59a48eb6 at: sun.misc.Unsafe.park(Native Method) 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.acquireQueued(AbstractQueuedSynchronizer.java:867) at: java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at: java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at: java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at: org.apache.activemq.transport.MutexTransport.oneway(MutexTransport.java:66) at: org.apache.activemq.transport.ResponseCorrelator.oneway(ResponseCorrelator.java:60) at: org.apache.activemq.network.DemandForwardingBridgeSupport.serviceLocalCommand(DemandForwardingBridgeSupport.java:972) at: org.apache.activemq.network.DemandForwardingBridgeSupport$2.onCommand(DemandForwardingBridgeSupport.java:201) at: org.apache.activemq.transport.ResponseCorrelator.onCommand(ResponseCorrelator.java:116) at: org.apache.activemq.transport.MutexTransport.onCommand(MutexTransport.java:50) at: org.apache.activemq.transport.vm.VMTransport.doDispatch(VMTransport.java:138) at: org.apache.activemq.transport.vm.VMTransport.dispatch(VMTransport.java:127) at: org.apache.activemq.transport.vm.VMTransport.oneway(VMTransport.java:104) at: org.apache.activemq.transport.MutexTransport.oneway(MutexTransport.java:68) at: org.apache.activemq.transport.ResponseCorrelator.oneway(ResponseCorrelator.java:60) at: org.apache.activemq.broker.TransportConnection.dispatch(TransportConnection.java:1378) at: org.apache.activemq.broker.TransportConnection.processDispatch(TransportConnection.java:897) at: org.apache.activemq.broker.TransportConnection.iterate(TransportConnection.java:943) at: org.apache.activemq.thread.PooledTaskRunner.runTask(PooledTaskRunner.java:129) at: org.apache.activemq.thread.PooledTaskRunner$1.run(PooledTaskRunner.java:47) at: java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at: java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at: java.lang.Thread.run(Thread.java:722) Thread Synchronization Statistics: ----------------------- This thread is blocked waiting on lock owned currently by thread named: ActiveMQ Transport: tcp://xx/a.b.c.d:61616@61883, id: 108 Number of times this thread was blocked (to enter/reenter a Monitor): 0 Number of times this thread waited for a notification (i.e. it was in WAITING or TIMED_WAITING state): 393 Total CPU time for this thread: 0 seconds 46.875.000 nanoseconds. User-level CPU time for this thread: 0 seconds 46.875.000 nanoseconds. Object Monitors currently held or requested by this thread: [] Ownable Synchronizers (e.g. ReentrantLock and ReentrantReadWriteLock) held by this thread: []
The solution working for us was to revert commit AMQ3274Test, org.apache.activemq.usecases.NetworkBridgeProducerFlowControlTest - sync needs to be conditional.