Uploaded image for project: 'Apache Storm'
  1. Apache Storm
  2. STORM-839

Deadlock hazard in backtype.storm.messaging.netty.Client

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 0.9.4
    • 0.9.6
    • storm-core
    • None

    Description

      See the thread dump below that shows the deadlock. client-worker-1 is holding 7b5a7fa5 and waiting on 1446a1e9. Thread-10-disruptor-worker-transfer-queue is holding 1446a1e9 and is waiting on 7b5a7fa5.

      (Thread dump is truncated to show only the relevant parts)

      2015-05-28 15:37:15
      Full thread dump Java HotSpot(TM) 64-Bit Server VM (24.72-b04 mixed mode):

      "Thread-10-disruptor-worker-transfer-queue" - Thread t@52
      java.lang.Thread.State: BLOCKED
      at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.cleanUpWriteBuffer(AbstractNioWorker.java:398)

      • waiting to lock <7b5a7fa5> (a java.lang.Object) owned by "client-worker-1" t@25
        at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.writeFromUserCode(AbstractNioWorker.java:128)
        at org.apache.storm.netty.channel.socket.nio.NioClientSocketPipelineSink.eventSunk(NioClientSocketPipelineSink.java:84)
        at org.apache.storm.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendDownstream(DefaultChannelPipeline.java:779)
        at org.apache.storm.netty.channel.Channels.write(Channels.java:725)
        at org.apache.storm.netty.handler.codec.oneone.OneToOneEncoder.doEncode(OneToOneEncoder.java:71)
        at org.apache.storm.netty.handler.codec.oneone.OneToOneEncoder.handleDownstream(OneToOneEncoder.java:59)
        at org.apache.storm.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:591)
        at org.apache.storm.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:582)
        at org.apache.storm.netty.channel.Channels.write(Channels.java:704)
        at org.apache.storm.netty.channel.Channels.write(Channels.java:671)
        at org.apache.storm.netty.channel.AbstractChannel.write(AbstractChannel.java:248)
        at backtype.storm.messaging.netty.Client.flushMessages(Client.java:480)
      • locked <1446a1e9> (a backtype.storm.messaging.netty.Client)
        at backtype.storm.messaging.netty.Client.send(Client.java:412)
      • locked <1446a1e9> (a backtype.storm.messaging.netty.Client)
        at backtype.storm.utils.TransferDrainer.send(TransferDrainer.java:54)
        at backtype.storm.daemon.worker$mk_transfer_tuples_handler$fn_5014$fn_5015.invoke(worker.clj:334)
        at backtype.storm.daemon.worker$mk_transfer_tuples_handler$fn__5014.invoke(worker.clj:332)
        at backtype.storm.disruptor$clojure_handler$reify__1446.onEvent(disruptor.clj:58)
        at backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:125)
        at backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:99)
        at backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:80)
        at backtype.storm.disruptor$consume_loop_STAR_$fn__1459.invoke(disruptor.clj:94)
        at backtype.storm.util$async_loop$fn__458.invoke(util.clj:463)
        at clojure.lang.AFn.run(AFn.java:24)
        at java.lang.Thread.run(Unknown Source)

      Locked ownable synchronizers:

      • None

      "client-worker-1" - Thread t@25
      java.lang.Thread.State: BLOCKED
      at backtype.storm.messaging.netty.Client.closeChannelAndReconnect(Client.java:501)

      • waiting to lock <1446a1e9> (a backtype.storm.messaging.netty.Client) owned by "Thread-10-disruptor-worker-transfer-queue" t@52
        at backtype.storm.messaging.netty.Client.access$1400(Client.java:78)
        at backtype.storm.messaging.netty.Client$3.operationComplete(Client.java:492)
        at org.apache.storm.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:427)
        at org.apache.storm.netty.channel.DefaultChannelFuture.notifyListeners(DefaultChannelFuture.java:413)
        at org.apache.storm.netty.channel.DefaultChannelFuture.setFailure(DefaultChannelFuture.java:380)
        at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.cleanUpWriteBuffer(AbstractNioWorker.java:437)
      • locked <7b5a7fa5> (a java.lang.Object)
        at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.close(AbstractNioWorker.java:373)
        at org.apache.storm.netty.channel.socket.nio.NioWorker.read(NioWorker.java:93)
        at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108)
        at org.apache.storm.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:318)
        at org.apache.storm.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89)
        at org.apache.storm.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
        at org.apache.storm.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
        at org.apache.storm.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
        at java.lang.Thread.run(Unknown Source)

      Locked ownable synchronizers:

      • locked <75e528fd> (a java.util.concurrent.ThreadPoolExecutor$Worker)

      Attachments

        Activity

          People

            eshioji Enno Shioji
            eshioji Enno Shioji
            Votes:
            1 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: