Uploaded image for project: 'ActiveMQ Artemis'
  1. ActiveMQ Artemis
  2. ARTEMIS-1222

Deadlock when using MQTT in combination with AMQP

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 2.1.0
    • 2.3.0
    • AMQP, Broker, MQTT
    • None
    • Linux

    Description

      Publishing messages via MQTT and consuming then via AMQP produces the following deadlock after a short while of running:

      Jun 09 09:21:17 localhost artemis[21194]: Found one Java-level deadlock:
      Jun 09 09:21:17 localhost artemis[21194]: =============================
      Jun 09 09:21:17 localhost artemis[21194]: "Thread-2 (activemq-netty-threads)":
      Jun 09 09:21:17 localhost artemis[21194]:   waiting for ownable synchronizer 0xaaf45410, (a java.util.concurrent.locks.ReentrantLock$NonfairSync),
      Jun 09 09:21:17 localhost artemis[21194]:   which is held by "Thread-1 (activemq-netty-threads)"
      Jun 09 09:21:17 localhost artemis[21194]: "Thread-1 (activemq-netty-threads)":
      Jun 09 09:21:17 localhost artemis[21194]:   waiting to lock monitor 0x0001ac98 (object 0xa749a4f8, a org.apache.activemq.artemis.core.server.impl.QueueImpl),
      Jun 09 09:21:17 localhost artemis[21194]:   which is held by "Thread-2 (activemq-netty-threads)"
      Jun 09 09:21:17 localhost artemis[21194]: Java stack information for the threads listed above:
      Jun 09 09:21:17 localhost artemis[21194]: ===================================================
      Jun 09 09:21:17 localhost artemis[21194]: "Thread-2 (activemq-netty-threads)":
      Jun 09 09:21:17 localhost artemis[21194]:         at sun.misc.Unsafe.park(Native Method)
      Jun 09 09:21:17 localhost artemis[21194]:         - parking to wait for  <0xaaf45410> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
      Jun 09 09:21:17 localhost artemis[21194]:         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      Jun 09 09:21:17 localhost artemis[21194]:         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
      Jun 09 09:21:17 localhost artemis[21194]:         at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
      Jun 09 09:21:17 localhost artemis[21194]:         at java.util.concurrent.locks.ReentrantLock.tryLock(ReentrantLock.java:442)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.handler.ProtonHandler.tryLock(ProtonHandler.java:139)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext.tryLock(AMQPConnectionContext.java:133)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext.deliverMessage(ProtonServerSenderContext.java:680)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback.sendMessage(AMQPSessionCallback.java:570)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl.deliverStandardMessage(ServerConsumerImpl.java:1057)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl.proceedDeliver(ServerConsumerImpl.java:432)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.QueueImpl.proceedDeliver(QueueImpl.java:2705)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.QueueImpl.deliverDirect(QueueImpl.java:2690)
      Jun 09 09:21:17 localhost artemis[21194]:         - locked <0xa749a4f8> (a org.apache.activemq.artemis.core.server.impl.QueueImpl)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.QueueImpl.addTail(QueueImpl.java:650)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.addReferences(PostOfficeImpl.java:1297)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.access$000(PostOfficeImpl.java:93)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl$1.done(PostOfficeImpl.java:1157)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl.executeOnCompletion(OperationContextImpl.java:181)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl.executeOnCompletion(OperationContextImpl.java:130)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager.afterCompleteOperations(AbstractJournalStorageManager.java:303)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.processRoute(PostOfficeImpl.java:1149)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.route(PostOfficeImpl.java:787)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.route(PostOfficeImpl.java:689)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.ServerSessionImpl.doSend(ServerSessionImpl.java:1674)
      Jun 09 09:21:17 localhost artemis[21194]:         - locked <0xa74a6740> (a org.apache.activemq.artemis.core.server.impl.ServerSessionImpl)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.ServerSessionImpl.send(ServerSessionImpl.java:1351)
      Jun 09 09:21:17 localhost artemis[21194]:         - locked <0xa74a6740> (a org.apache.activemq.artemis.core.server.impl.ServerSessionImpl)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.protocol.mqtt.MQTTPublishManager.sendInternal(MQTTPublishManager.java:164)
      Jun 09 09:21:17 localhost artemis[21194]:         - locked <0xa74cff90> (a java.lang.Object)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.protocol.mqtt.MQTTPublishManager.handleMessage(MQTTPublishManager.java:134)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.protocol.mqtt.MQTTProtocolHandler.handlePublish(MQTTProtocolHandler.java:201)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.protocol.mqtt.MQTTProtocolHandler.channelRead(MQTTProtocolHandler.java:117)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:293)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:267)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1334)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:926)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:134)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:624)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:559)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:476)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:438)
      Jun 09 09:21:17 localhost artemis[21194]:         at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
      Jun 09 09:21:17 localhost artemis[21194]:         at java.lang.Thread.run(Thread.java:745)
      Jun 09 09:21:17 localhost artemis[21194]: "Thread-1 (activemq-netty-threads)":
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.RefsOperation.afterCommit(RefsOperation.java:156)
      Jun 09 09:21:17 localhost artemis[21194]:         - waiting to lock <0xa749a4f8> (a org.apache.activemq.artemis.core.server.impl.QueueImpl)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.transaction.impl.TransactionImpl.afterCommit(TransactionImpl.java:546)
      Jun 09 09:21:17 localhost artemis[21194]:         - locked <0xa3693f08> (a org.apache.activemq.artemis.core.transaction.impl.TransactionImpl)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.transaction.impl.TransactionImpl.access$100(TransactionImpl.java:38)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.transaction.impl.TransactionImpl$2.done(TransactionImpl.java:304)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl.executeOnCompletion(OperationContextImpl.java:181)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl.executeOnCompletion(OperationContextImpl.java:130)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager.afterCompleteOperations(AbstractJournalStorageManager.java:303)
      Jun 09 09:21:17 localhost artemis[21194]:         at org.apache.activemq.artemis.core.transaction.impl.TransactionImpl.commit(TransactionImpl.java:295)
      Jun 09 09:21:18 localhost artemis[21194]:         - locked <0xa3693f88> (a java.lang.Object)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.core.transaction.impl.TransactionImpl.commit(TransactionImpl.java:246)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl.individualAcknowledge(ServerConsumerImpl.java:893)
      Jun 09 09:21:18 localhost artemis[21194]:         - locked <0xa74c6aa0> (a org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback.ack(AMQPSessionCallback.java:358)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext.onMessage(ProtonServerSenderContext.java:582)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext.onDelivery(AMQPConnectionContext.java:494)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.handler.Events.dispatch(Events.java:92)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.handler.ProtonHandler.dispatch(ProtonHandler.java:347)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.handler.ProtonHandler.flush(ProtonHandler.java:276)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.handler.ProtonHandler.inputBuffer(ProtonHandler.java:240)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext.inputBuffer(AMQPConnectionContext.java:121)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.protocol.amqp.broker.ActiveMQProtonRemotingConnection.bufferReceived(ActiveMQProtonRemotingConnection.java:138)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.core.remoting.server.impl.RemotingServiceImpl$DelegatingBufferHandler.bufferReceived(RemotingServiceImpl.java:631)
      Jun 09 09:21:18 localhost artemis[21194]:         at org.apache.activemq.artemis.core.remoting.impl.netty.ActiveMQChannelHandler.channelRead(ActiveMQChannelHandler.java:69)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1334)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:926)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:134)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:624)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:559)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:476)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:438)
      Jun 09 09:21:18 localhost artemis[21194]:         at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
      Jun 09 09:21:18 localhost artemis[21194]:         at java.lang.Thread.run(Thread.java:745)
      Jun 09 09:21:18 localhost artemis[21194]: Found 1 deadlock.
      Jun 09 09:21:18 localhost artemis[21194]: Heap
      Jun 09 09:21:18 localhost artemis[21194]:  def new generation   total 78656K, used 47089K [0xa3000000, 0xa8550000, 0xa8550000)
      Jun 09 09:21:18 localhost artemis[21194]:   eden space 69952K,  61% used [0xa3000000, 0xa5a54850, 0xa7450000)
      Jun 09 09:21:18 localhost artemis[21194]:   from space 8704K,  43% used [0xa7450000, 0xa77f7c48, 0xa7cd0000)
      Jun 09 09:21:18 localhost artemis[21194]:   to   space 8704K,   0% used [0xa7cd0000, 0xa7cd0000, 0xa8550000)
      Jun 09 09:21:18 localhost artemis[21194]:  tenured generation   total 174784K, used 44500K [0xa8550000, 0xb3000000, 0xb3000000)
      Jun 09 09:21:18 localhost artemis[21194]:    the space 174784K,  25% used [0xa8550000, 0xab0c52d0, 0xab0c5400, 0xb3000000)
      Jun 09 09:21:18 localhost artemis[21194]:  Metaspace       used 19140K, capacity 19410K, committed 19480K, reserved 19760K
      Jun 09 09:21:24 localhost artemis[21194]: 2017-06-09 09:21:24
      
      

      Attachments

        1. deadlock.txt
          141 kB
          Jens Reimann

        Activity

          People

            Unassigned Unassigned
            ctron Jens Reimann
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: