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

NPE in WorkerState.transferLocalBatch when receiving messages for a task that isn't the first task assigned to the executor

    XMLWordPrintableJSON

Details

    Description

      2018-07-02 20:32:28.944 [Worker-Transfer] ERROR org.apache.storm.utils.Utils - Async loop died!
      java.lang.NullPointerException: null
      	at org.apache.storm.daemon.worker.WorkerState.transferLocalBatch(WorkerState.java:538) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.messaging.DeserializingConnectionCallback.recv(DeserializingConnectionCallback.java:71) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.messaging.local.Context$LocalClient.send(Context.java:194) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.TransferDrainer.send(TransferDrainer.java:53) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.daemon.worker.WorkerTransfer.flush(WorkerTransfer.java:100) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consumeImpl(JCQueue.java:146) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consume(JCQueue.java:110) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consume(JCQueue.java:101) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.daemon.worker.WorkerTransfer.lambda$makeTransferThread$0(WorkerTransfer.java:82) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.Utils$2.run(Utils.java:353) [storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_144]
      2018-07-02 20:32:28.945 [Worker-Transfer] ERROR org.apache.storm.utils.Utils - Async loop died!
      java.lang.RuntimeException: java.lang.NullPointerException
      	at org.apache.storm.utils.Utils$2.run(Utils.java:368) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_144]
      Caused by: java.lang.NullPointerException
      	at org.apache.storm.daemon.worker.WorkerState.transferLocalBatch(WorkerState.java:538) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.messaging.DeserializingConnectionCallback.recv(DeserializingConnectionCallback.java:71) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.messaging.local.Context$LocalClient.send(Context.java:194) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.TransferDrainer.send(TransferDrainer.java:53) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.daemon.worker.WorkerTransfer.flush(WorkerTransfer.java:100) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consumeImpl(JCQueue.java:146) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consume(JCQueue.java:110) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.JCQueue.consume(JCQueue.java:101) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.daemon.worker.WorkerTransfer.lambda$makeTransferThread$0(WorkerTransfer.java:82) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	at org.apache.storm.utils.Utils$2.run(Utils.java:353) ~[storm-client-2.0.0-SNAPSHOT.jar:2.0.0-SNAPSHOT]
      	... 1 more
      

      When tuples are received, the method looks up which JCQueue to send to. It does this with the tuple destination task id. The map it looks in only has the queues by the starting task id of the executor's range, so if the destination is e.g. task 4 for an executor with assignment [3, 4], we hit an NPE.

      Attachments

        Issue Links

          Activity

            People

              srdo Stig Rohde Døssing
              srdo Stig Rohde Døssing
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 40m
                  40m