Uploaded image for project: 'ActiveMQ Classic'
  1. ActiveMQ Classic
  2. AMQ-4632

Fix of AMQ-4328 causes deadlocks during set up of the bridge

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Incomplete
    • 5.8.0
    • None
    • Broker
    • 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.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              pdudits Patrik Dudits
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: