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

AMQ224079: The process for the virtual machine will be killed

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Duplicate
    • 2.17.0
    • None
    • AMQP, Broker
    • centos 7.9 prod, AWS m5.Xlarge

    Description

      We have below architecture in master/slave

      producer/consumer --> Apache QPID (1.18) --> Artemis 2.17 (master/slave)
      

      We see our master broker going down abruptly with below threaddump. Could you please help and suggest a solution? The below issue is only reported in production.

      One more problem - we saw message count in ExpiryQueue spiked from 1K to 13K after manually restarting broker. 

      2021-06-15 16:22:10,250 ERROR [org.apache.activemq.artemis.core.server] AMQ224079: The process for the virtual machine will be killed, as component org.apache.activemq.artemis.core.io.buffer.TimedBuffer@40639fab is not responsive
      2021-06-15 16:22:11,033 WARN  [org.apache.activemq.artemis.core.server] AMQ222199: Thread dump: *******************************************************************************
      Complete Thread dump 
      "Thread-38676 (ActiveMQ-IO-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$7@5f5effb0)" Id=243147 BLOCKED on org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage@3a185aa5 owned by "Thread-16 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=125
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage.ensureMessageDataScanned(AMQPMessage.java:572)
      	-  blocked on org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage@3a185aa5
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage.getExpiration(AMQPMessage.java:962)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessagePersister.encode(AMQPLargeMessagePersister.java:97)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessagePersister.encode(AMQPLargeMessagePersister.java:32)
      	at org.apache.activemq.artemis.core.journal.impl.dataformat.JournalAddRecord.encode(JournalAddRecord.java:72)
      	at org.apache.activemq.artemis.core.io.buffer.TimedBuffer.addBytes(TimedBuffer.java:321)
      	-  locked org.apache.activemq.artemis.core.io.buffer.TimedBuffer@40639fab
      	at org.apache.activemq.artemis.core.io.AbstractSequentialFile.write(AbstractSequentialFile.java:231)
      	at org.apache.activemq.artemis.core.journal.impl.JournalImpl.appendRecord(JournalImpl.java:2937)
      	at org.apache.activemq.artemis.core.journal.impl.JournalImpl.access$100(JournalImpl.java:92)
      	at org.apache.activemq.artemis.core.journal.impl.JournalImpl$1.run(JournalImpl.java:850)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:42)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:31)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase.executePendingTasks(ProcessorBase.java:65)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase$$Lambda$39/2124562732.run(Unknown Source)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      	Number of locked synchronizers = 1
      	- java.util.concurrent.ThreadPoolExecutor$Worker@556802c0
      "Thread-38675 (ActiveMQ-IO-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$7@5f5effb0)" Id=243105 BLOCKED on org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage@3a185aa5 owned by "Thread-16 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=125
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage.ensureMessageDataScanned(AMQPMessage.java:572)
      	-  blocked on org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage@3a185aa5
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage.getExpiration(AMQPMessage.java:962)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessagePersister.encode(AMQPLargeMessagePersister.java:97)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessagePersister.encode(AMQPLargeMessagePersister.java:32)
      	at org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReplicationAddMessage.encodeRest(ReplicationAddMessage.java:85)
      	at org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.encodeRest(PacketImpl.java:403)
      	at org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.encode(PacketImpl.java:326)
      	at org.apache.activemq.artemis.core.protocol.core.impl.ChannelImpl.send(ChannelImpl.java:370)
      	-  locked java.lang.Object@6474af0b
      	at org.apache.activemq.artemis.core.protocol.core.impl.ChannelImpl.send(ChannelImpl.java:314)
      	at org.apache.activemq.artemis.core.replication.ReplicationManager.lambda$sendReplicatePacket$0(ReplicationManager.java:391)
      	at org.apache.activemq.artemis.core.replication.ReplicationManager$$Lambda$245/59388390.run(Unknown Source)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:42)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:31)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase.executePendingTasks(ProcessorBase.java:65)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase$$Lambda$39/2124562732.run(Unknown Source)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      	Number of locked synchronizers = 1
      	- java.util.concurrent.ThreadPoolExecutor$Worker@5765287e
      "qtp1129874021-239133" Id=239133 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:842)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:892)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-239132" Id=239132 RUNNABLE (in native)
      	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:93)
      	at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
      	-  locked sun.nio.ch.Util$3@21763de1
      	-  locked java.util.Collections$UnmodifiableSet@29d19aa8
      	-  locked sun.nio.ch.EPollSelectorImpl@109d9daf
      	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
      	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)
      	at org.eclipse.jetty.io.ManagedSelector.nioSelect(ManagedSelector.java:149)
      	at org.eclipse.jetty.io.ManagedSelector.select(ManagedSelector.java:156)
      	at org.eclipse.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:572)
      	at org.eclipse.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:509)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.produceTask(EatWhatYouKill.java:360)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:184)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:171)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:129)
      	at org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:375)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:773)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:905)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-239131" Id=239131 RUNNABLE (in native)
      	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:93)
      	at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
      	-  locked sun.nio.ch.Util$3@51d06440
      	-  locked java.util.Collections$UnmodifiableSet@4159c2ae
      	-  locked sun.nio.ch.EPollSelectorImpl@647814f7
      	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
      	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)
      	at org.eclipse.jetty.io.ManagedSelector.nioSelect(ManagedSelector.java:149)
      	at org.eclipse.jetty.io.ManagedSelector.select(ManagedSelector.java:156)
      	at org.eclipse.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:572)
      	at org.eclipse.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:509)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.produceTask(EatWhatYouKill.java:360)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:184)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:171)
      	at org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:129)
      	at org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:375)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:773)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:905)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-239123" Id=239123 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:842)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:892)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-238445" Id=238445 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:842)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:892)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-238442" Id=238442 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:842)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:892)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-232612" Id=232612 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55ebcfbf
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:842)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:892)
      	at java.lang.Thread.run(Thread.java:748)
      "Session-Scheduler-5d93ff21-1" Id=19726 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@6cded303
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@6cded303
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Token Expiration Handler" Id=7544 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@43101c07
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@43101c07
      	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 java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "Thread-11 (activemq-netty-threads)" Id=7538 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-10 (activemq-netty-threads)" Id=7537 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-9 (activemq-netty-threads)" Id=7534 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-8 (activemq-netty-threads)" Id=7533 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-7 (activemq-netty-threads)" Id=7532 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-6 (activemq-netty-threads)" Id=7530 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-5 (activemq-netty-threads)" Id=7529 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-3 (activemq-netty-threads)" Id=580 RUNNABLE
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-4 (activemq-netty-threads)" Id=153 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Connector-Scheduler-29147c12-1" Id=147 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5f13a4cf
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5f13a4cf
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "Thread-3 (activemq-netty-threads)" Id=146 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-2 (activemq-netty-threads)" Id=145 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (activemq-netty-threads)" Id=144 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-2 (activemq-netty-threads)" Id=143 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (activemq-netty-threads)" Id=142 RUNNABLE
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (activemq-netty-threads)" Id=141 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (activemq-netty-threads)" Id=140 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:148)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:141)
      	at io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "activemq-failure-check-thread" Id=139 TIMED_WAITING on java.util.concurrent.CountDownLatch$Sync@1a113149
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.CountDownLatch$Sync@1a113149
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
      	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
      	at org.apache.activemq.artemis.core.remoting.server.impl.RemotingServiceImpl$FailureCheckAndFlushThread.run(RemotingServiceImpl.java:782)
      "Thread-29 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=138 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-28 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=137 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-27 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=136 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-26 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=135 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-25 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=134 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-24 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=133 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-23 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=132 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-22 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=131 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-21 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=130 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-20 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=129 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-19 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=128 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-18 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=127 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-17 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=126 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-16 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=125 WAITING on java.util.concurrent.CountDownLatch$Sync@2f662f7b
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.CountDownLatch$Sync@2f662f7b
      	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 org.apache.activemq.artemis.utils.SimpleFutureImpl.get(SimpleFutureImpl.java:62)
      	at org.apache.activemq.artemis.core.journal.impl.JournalImpl.checkKnownRecordID(JournalImpl.java:1155)
      	at org.apache.activemq.artemis.core.journal.impl.JournalImpl.appendDeleteRecord(JournalImpl.java:989)
      	at org.apache.activemq.artemis.core.replication.ReplicatedJournal.appendDeleteRecord(ReplicatedJournal.java:233)
      	at org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager.confirmPendingLargeMessage(AbstractJournalStorageManager.java:359)
      	at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.confirmLargeMessageSend(PostOfficeImpl.java:1620)
      	-  locked org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage@3a185aa5
      	at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.processRoute(PostOfficeImpl.java:1562)
      	at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.route(PostOfficeImpl.java:1191)
      	at org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl.route(PostOfficeImpl.java:1063)
      	at org.apache.activemq.artemis.core.server.impl.ServerSessionImpl.doSend(ServerSessionImpl.java:2172)
      	-  locked org.apache.activemq.artemis.core.server.impl.ServerSessionImpl@18f48d32
      	at org.apache.activemq.artemis.core.server.impl.ServerSessionImpl.send(ServerSessionImpl.java:1812)
      	-  locked org.apache.activemq.artemis.core.server.impl.ServerSessionImpl@18f48d32
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback.inSessionSend(AMQPSessionCallback.java:563)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback.lambda$serverSend$2(AMQPSessionCallback.java:522)
      	at org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback$$Lambda$275/60269086.run(Unknown Source)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:42)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:31)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase.executePendingTasks(ProcessorBase.java:65)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase$$Lambda$39/2124562732.run(Unknown Source)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      	Number of locked synchronizers = 1
      	- java.util.concurrent.ThreadPoolExecutor$Worker@4fd856a6
      "Thread-15 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=124 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-14 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=123 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-13 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=122 TIMED_WAITING on java.util.concurrent.CountDownLatch$Sync@2bc1cd6b
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.CountDownLatch$Sync@2bc1cd6b
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
      	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
      	at org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl$StaticConnector.connect(ServerLocatorImpl.java:1733)
      	at org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl.connect(ServerLocatorImpl.java:540)
      	-  locked org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl@4ad40808
      	at org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl.connect(ServerLocatorImpl.java:528)
      	at org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl$4.run(ServerLocatorImpl.java:486)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:42)
      	at org.apache.activemq.artemis.utils.actors.OrderedExecutor.doTask(OrderedExecutor.java:31)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase.executePendingTasks(ProcessorBase.java:65)
      	at org.apache.activemq.artemis.utils.actors.ProcessorBase$$Lambda$39/2124562732.run(Unknown Source)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      	Number of locked synchronizers = 1
      	- java.util.concurrent.ThreadPoolExecutor$Worker@43dac048
      "Thread-12 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=121 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-11 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=120 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-10 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=119 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-9 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=118 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-8 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=117 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-7 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=116 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-6 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=115 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-5 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=114 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-4 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=113 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-3 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=112 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-11 (ActiveMQ-client-netty-threads)" Id=111 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-10 (ActiveMQ-client-netty-threads)" Id=110 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-2 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=109 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "activemq-buffer-timeout" Id=108 BLOCKED on org.apache.activemq.artemis.core.io.buffer.TimedBuffer@40639fab owned by "Thread-38676 (ActiveMQ-IO-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$7@5f5effb0)" Id=243147
      	at org.apache.activemq.artemis.core.io.buffer.TimedBuffer.flushBatch(TimedBuffer.java:349)
      	-  blocked on org.apache.activemq.artemis.core.io.buffer.TimedBuffer@40639fab
      	at org.apache.activemq.artemis.core.io.buffer.TimedBuffer$CheckTimer.run(TimedBuffer.java:457)
      	at java.lang.Thread.run(Thread.java:748)
      "Thread-9 (ActiveMQ-client-netty-threads)" Id=82 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-8 (ActiveMQ-client-netty-threads)" Id=81 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-7 (ActiveMQ-client-netty-threads)" Id=80 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-6 (ActiveMQ-client-netty-threads)" Id=79 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-4 (ActiveMQ-client-global-scheduled-threads)" Id=78 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-5 (ActiveMQ-client-netty-threads)" Id=77 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "DestroyJavaVM" Id=73 RUNNABLE
      "New Relic Faster Harvest Service" Id=72 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@f5bca59
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@f5bca59
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Harvest Service" Id=71 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@2ad4e44c
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@2ad4e44c
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "Thread-4 (ActiveMQ-scheduled-threads)" Id=70 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-4 (ActiveMQ-client-netty-threads)" Id=69 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-3 (ActiveMQ-scheduled-threads)" Id=68 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-2 (ActiveMQ-scheduled-threads)" Id=67 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (ActiveMQ-scheduled-threads)" Id=65 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (ActiveMQ-scheduled-threads)" Id=64 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72b26e37
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-3 (ActiveMQ-client-netty-threads)" Id=63 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=62 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (ActiveMQ-server-org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$6@41b13f3d)" Id=61 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@118c8383
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:112)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor$ThreadPoolQueue.poll(ActiveMQThreadPoolExecutor.java:45)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Session-HouseKeeper-4583b617-1" Id=60 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7e70d099
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7e70d099
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "qtp1129874021-54-acceptor-0@3538898e-ServerConnector@29147c12\{SSL, (ssl, http/1.1)}{activemq-prod-master2a.odprivatecloud.com:9161}" Id=54 RUNNABLE (in native)
      	at sun.nio.ch.ServerSocketChannelImpl.accept0(Native Method)
      	at sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:421)
      	at sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:249)
      	-  locked java.lang.Object@4517ff4e
      	at org.eclipse.jetty.server.ServerConnector.accept(ServerConnector.java:388)
      	at org.eclipse.jetty.server.AbstractConnector$Acceptor.run(AbstractConnector.java:702)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:773)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:905)
      	at java.lang.Thread.run(Thread.java:748)
      "Thread-2 (ActiveMQ-client-netty-threads)" Id=50 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-3 (ActiveMQ-client-global-scheduled-threads)" Id=49 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-2 (ActiveMQ-client-global-scheduled-threads)" Id=48 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (ActiveMQ-client-netty-threads)" Id=47 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-1 (ActiveMQ-client-global-scheduled-threads)" Id=44 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (ActiveMQ-client-global-scheduled-threads)" Id=43 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7a3ece06
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (ActiveMQ-client-netty-threads)" Id=42 RUNNABLE (in native)
      	at io.netty.channel.epoll.Native.epollWait0(Native Method)
      	at io.netty.channel.epoll.Native.epollWait(Native.java:132)
      	at io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:281)
      	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
      	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      "Thread-0 (-scheduled-threads)" Id=39 RUNNABLE
      	at sun.management.ThreadImpl.dumpThreads0(Native Method)
      	at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:496)
      	at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:484)
      	at org.apache.activemq.artemis.utils.ThreadDumpUtil.threadDump(ThreadDumpUtil.java:47)
      	at org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl.threadDump(ActiveMQServerImpl.java:1128)
      	at org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl.lambda$initializeCriticalAnalyzer$0(ActiveMQServerImpl.java:714)
      	at org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl$$Lambda$37/619621481.run(Unknown Source)
      	at org.apache.activemq.artemis.utils.critical.CriticalAnalyzerImpl.fireActions(CriticalAnalyzerImpl.java:155)
      	at org.apache.activemq.artemis.utils.critical.CriticalAnalyzerImpl.check(CriticalAnalyzerImpl.java:140)
      	at org.apache.activemq.artemis.utils.critical.CriticalAnalyzerImpl$1.run(CriticalAnalyzerImpl.java:53)
      	at org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent.runForExecutor(ActiveMQScheduledComponent.java:313)
      	at org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent.bookedRunForScheduler(ActiveMQScheduledComponent.java:328)
      	at org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent.runForScheduler(ActiveMQScheduledComponent.java:339)
      	at org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent.lambda$start$0(ActiveMQScheduledComponent.java:166)
      	at org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent$$Lambda$36/2096842550.run(Unknown Source)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.activemq.artemis.utils.ActiveMQThreadFactory$1.run(ActiveMQThreadFactory.java:118)
      	Number of locked synchronizers = 1
      	- java.util.concurrent.ThreadPoolExecutor$Worker@7650ded6
      "ActiveMQ Artemis Server Shutdown Timer" Id=37 TIMED_WAITING on java.util.TaskQueue@23421c3e
      	at java.lang.Object.wait(Native Method)
      	-  waiting on java.util.TaskQueue@23421c3e
      	at java.util.TimerThread.mainLoop(Timer.java:552)
      	at java.util.TimerThread.run(Timer.java:505)
      "New Relic RPM Connection Service" Id=31 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72dd385d
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@72dd385d
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Deadlock Detector" Id=30 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@47192a59
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@47192a59
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Sampler Service" Id=29 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@616c94b7
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@616c94b7
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Transaction Service Processor" Id=28 TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@6662a68f
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@6662a68f
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
      	at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Utilization Service 2" Id=27 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55f2e591
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55f2e591
      	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 java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Utilization Service" Id=26 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55f2e591
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@55f2e591
      	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 java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Retransformer" Id=25 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@4ef620a2
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@4ef620a2
      	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:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "New Relic Jar Analysis Thread" Id=24 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b380c7
      	at sun.misc.Unsafe.park(Native Method)
      	-  waiting on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b380c7
      	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 java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      "Signal Dispatcher" Id=5 RUNNABLE
      "Finalizer" Id=3 WAITING on java.lang.ref.ReferenceQueue$Lock@33960080
      	at java.lang.Object.wait(Native Method)
      	-  waiting on java.lang.ref.ReferenceQueue$Lock@33960080
      	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:144)
      	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:165)
      	at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:216)
      "Reference Handler" Id=2 WAITING on java.lang.ref.Reference$Lock@718ab50e
      	at java.lang.Object.wait(Native Method)
      	-  waiting on java.lang.ref.Reference$Lock@718ab50e
      	at java.lang.Object.wait(Object.java:502)
      	at java.lang.ref.Reference.tryHandlePending(Reference.java:191)
      	at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)
      ===============================================================================
      End Thread dump 
      *******************************************************************************
      

      Attachments

        1. broker.xml
          12 kB
          mahendra anil sonawale
        2. image-2021-06-17-22-15-11-880.png
          70 kB
          mahendra anil sonawale

        Issue Links

          Activity

            People

              Unassigned Unassigned
              mahendra s mahendra anil sonawale
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: