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

Nimbus will crash if pacemaker is restarted

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.0.0
    • None

    Description

      Below is the nimbus.log when I restarted pacemaker. Nimbus crashed because of NPE.

       

       

      2018-03-26 21:39:18.404 main o.a.s.z.LeaderElectorImp [INFO] Queued up for leader lock.
      2018-03-26 21:39:18.458 main o.a.s.d.m.MetricsUtils [INFO] Using statistics reporter plugin:org.apache.storm.daemon.metrics.reporters.JmxPreparableRepor
      ter
      2018-03-26 21:39:18.461 main o.a.s.d.m.r.JmxPreparableReporter [INFO] Preparing...
      2018-03-26 21:39:18.527 main o.a.s.m.StormMetricsRegistry [INFO] Started statistics report plugin...
      2018-03-26 21:39:18.710 main o.a.s.m.n.Login [INFO] successfully logged in.
      2018-03-26 21:39:18.738 Refresh-TGT o.a.s.m.n.Login [INFO] TGT refresh thread started.
      2018-03-26 21:39:18.739 main o.a.s.z.ClientZookeeper [INFO] Staring ZK Curator
      2018-03-26 21:39:18.739 main o.a.c.f.i.CuratorFrameworkImpl [INFO] Starting
      2018-03-26 21:39:18.747 Refresh-TGT o.a.s.m.n.Login [INFO] TGT valid starting at:        Mon Mar 26 21:39:18 UTC 2018
      2018-03-26 21:39:18.747 Refresh-TGT o.a.s.m.n.Login [INFO] TGT expires:                  Tue Mar 27 21:39:18 UTC 2018
      2018-03-26 21:39:18.747 Refresh-TGT o.a.s.m.n.Login [INFO] TGT refresh sleeping until: Tue Mar 27 17:39:22 UTC 2018
      2018-03-26 21:39:18.756 main o.a.z.ZooKeeper [INFO] Initiating client connection, connectString=openqe74blue-gw.blue.ygrid.yahoo.com:2181 sessionTimeout
      =60000 watcher=org.apache.curator.ConnectionState@148c7c4b
      2018-03-26 21:39:18.807 main o.a.c.f.i.CuratorFrameworkImpl [INFO] Default schema
      2018-03-26 21:39:18.814 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.c.ZooKeeperSaslClient [INFO] Client will use GSSAPI as SASL mec
      hanism.
      2018-03-26 21:39:18.815 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Opening socket connection to server openqe74b
      lue-gw.blue.ygrid.yahoo.com/10.215.68.156:2181. Will attempt to SASL-authenticate using Login Context section 'Client'
      2018-03-26 21:39:18.816 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Socket connection established to openqe74blue
      -gw.blue.ygrid.yahoo.com/10.215.68.156:2181, initiating session
      2018-03-26 21:39:18.817 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Session establishment complete on server open
      qe74blue-gw.blue.ygrid.yahoo.com/10.215.68.156:2181, sessionid = 0x1624f6d49dd0cdd, negotiated timeout = 40000
      2018-03-26 21:39:18.818 main-EventThread o.a.c.f.s.ConnectionStateManager [INFO] State change: CONNECTED
      2018-03-26 21:39:18.839 Curator-Framework-0 o.a.c.f.i.CuratorFrameworkImpl [INFO] backgroundOperationsLoop exiting
      2018-03-26 21:39:18.841 main o.a.z.ZooKeeper [INFO] Session: 0x1624f6d49dd0cdd closed
      2018-03-26 21:39:18.842 main-EventThread o.a.z.ClientCnxn [INFO] EventThread shut down
      2018-03-26 21:39:18.844 main o.a.s.z.ClientZookeeper [INFO] Staring ZK Curator
      2018-03-26 21:39:18.844 main o.a.c.f.i.CuratorFrameworkImpl [INFO] Starting
      2018-03-26 21:39:18.875 main o.a.z.ZooKeeper [INFO] Initiating client connection, connectString=openqe74blue-gw.blue.ygrid.yahoo.com:2181/storm_ystormQE
      _CI sessionTimeout=60000 watcher=org.apache.curator.ConnectionState@211febf3
      2018-03-26 21:39:18.908 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.c.ZooKeeperSaslClient [INFO] Client will use GSSAPI as SASL mec
      hanism.
      2018-03-26 21:39:18.909 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Opening socket connection to server openqe74b
      lue-gw.blue.ygrid.yahoo.com/10.215.68.156:2181. Will attempt to SASL-authenticate using Login Context section 'Client'
      2018-03-26 21:39:18.910 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Socket connection established to openqe74blue
      -gw.blue.ygrid.yahoo.com/10.215.68.156:2181, initiating session
      2018-03-26 21:39:18.911 main-SendThread(openqe74blue-gw.blue.ygrid.yahoo.com:2181) o.a.z.ClientCnxn [INFO] Session establishment complete on server open
      qe74blue-gw.blue.ygrid.yahoo.com/10.215.68.156:2181, sessionid = 0x1624f6d49dd0cde, negotiated timeout = 40000
      2018-03-26 21:39:18.920 main o.a.c.f.i.CuratorFrameworkImpl [INFO] Default schema
      2018-03-26 21:39:18.923 main-EventThread o.a.c.f.s.ConnectionStateManager [INFO] State change: CONNECTED
      2018-03-26 21:39:18.986 main o.a.s.d.n.Nimbus [INFO] Starting nimbus server for storm version '2.0.0.y'
      2018-03-26 21:39:19.931 main-EventThread o.a.s.z.Zookeeper [INFO] active-topology-blobs [] local-topology-blobs [] diff-topology-blobs []
      2018-03-26 21:39:19.932 main-EventThread o.a.s.z.Zookeeper [INFO] active-topology-dependencies [] local-blobs [] diff-topology-dependencies []
      2018-03-26 21:39:19.932 main-EventThread o.a.s.z.Zookeeper [INFO] Accepting leadership, all active topologies and corresponding dependencies found local
      ly.
      2018-03-26 21:39:20.636 timer o.a.s.d.n.Nimbus [INFO] Scheduling took 1381 ms for 0 topologies
      2018-03-26 21:39:20.901 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: open
      qe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:20.901 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 101ms (NOT MAX)
      2018-03-26 21:39:21.003 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: open
      qe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.003 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 102ms (NOT MAX)
      2018-03-26 21:39:21.106 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.106 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 106ms (NOT MAX)
      2018-03-26 21:39:21.214 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.214 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 115ms (NOT MAX)
      2018-03-26 21:39:21.331 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.331 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 129ms (NOT MAX)
      2018-03-26 21:39:21.462 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.462 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 162ms (NOT MAX)
      2018-03-26 21:39:21.626 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.626 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 176ms (NOT MAX)
      2018-03-26 21:39:21.807 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:21.807 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 319ms (NOT MAX)
      2018-03-26 21:39:21.888 timer o.a.s.p.PacemakerClient [ERROR] error attempting to write to a channel {}
      org.apache.storm.pacemaker.PacemakerConnectionException: Timed out waiting for channel ready.
              at org.apache.storm.pacemaker.PacemakerClient.waitUntilReady(PacemakerClient.java:213) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:182) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:22.128 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:22.128 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 603ms (NOT MAX)
      2018-03-26 21:39:22.733 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:22.733 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 868ms (NOT MAX)
      2018-03-26 21:39:22.888 timer o.a.s.p.PacemakerClient [ERROR] error attempting to write to a channel {}
      org.apache.storm.pacemaker.PacemakerConnectionException: Timed out waiting for channel ready.
              at org.apache.storm.pacemaker.PacemakerClient.waitUntilReady(PacemakerClient.java:213) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:182) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:23.603 client-boss-1 o.a.s.p.PacemakerClientHandler [WARN] Connection to pacemaker failed. Trying to reconnect Connection refused: openqe74blue-n1.blue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:23.603 client-boss-1 o.a.s.u.StormBoundedExponentialBackoffRetry [WARN] WILL SLEEP FOR 1494ms (NOT MAX)
      2018-03-26 21:39:23.888 timer o.a.s.p.PacemakerClient [ERROR] error attempting to write to a channel {}
      org.apache.storm.pacemaker.PacemakerConnectionException: Timed out waiting for channel ready.
              at org.apache.storm.pacemaker.PacemakerClient.waitUntilReady(PacemakerClient.java:213) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:182) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:24.889 timer o.a.s.p.PacemakerClient [ERROR] error attempting to write to a channel {}
      org.apache.storm.pacemaker.PacemakerConnectionException: Timed out waiting for channel ready.
              at org.apache.storm.pacemaker.PacemakerClient.waitUntilReady(PacemakerClient.java:213) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:182) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:25.100 client-worker-4 o.a.s.m.n.KerberosSaslClientHandler [INFO] Connection established from /10.215.76.240:36922 to openqe74blue-n1.b        at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:25.100 client-worker-4 o.a.s.m.n.KerberosSaslClientHandler [INFO] Connection established from /10.215.76.240:36922 to openqe74blue-n1.b
      lue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:25.107 client-worker-4 o.a.s.m.n.KerberosSaslNettyClient [INFO] Creating Kerberos Client.
      2018-03-26 21:39:25.116 client-worker-4 o.a.s.m.n.Login [INFO] successfully logged in.
      2018-03-26 21:39:25.121 client-worker-4 o.a.s.m.n.KerberosSaslNettyClient [INFO] Got Client: com.sun.security.sasl.gsskerb.GssKrb5Client@116ce525
      2018-03-26 21:39:25.753 client-worker-1 o.a.s.m.n.KerberosSaslClientHandler [INFO] Connection established from /10.215.76.240:37614 to openqe74blue-n2.blue.ygrid.yahoo.com/10.215.76.243:6699
      2018-03-26 21:39:25.753 client-worker-1 o.a.s.m.n.KerberosSaslNettyClient [INFO] Creating Kerberos Client.
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:24.889 timer o.a.s.p.PacemakerClient [ERROR] error attempting to write to a channel {}
      org.apache.storm.pacemaker.PacemakerConnectionException: Timed out waiting for channel ready.
              at org.apache.storm.pacemaker.PacemakerClient.waitUntilReady(PacemakerClient.java:213) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:182) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClient.send(PacemakerClient.java:197) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.pacemaker.PacemakerClientPool.sendAll(PacemakerClientPool.java:65) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:193) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:25.100 client-worker-4 o.a.s.m.n.KerberosSaslClientHandler [INFO] Connection established from /10.215.76.240:36922 to openqe74blue-n1.b
      lue.ygrid.yahoo.com/10.215.76.240:6699
      2018-03-26 21:39:25.107 client-worker-4 o.a.s.m.n.KerberosSaslNettyClient [INFO] Creating Kerberos Client.
      2018-03-26 21:39:25.116 client-worker-4 o.a.s.m.n.Login [INFO] successfully logged in.
      2018-03-26 21:39:25.121 client-worker-4 o.a.s.m.n.KerberosSaslNettyClient [INFO] Got Client: com.sun.security.sasl.gsskerb.GssKrb5Client@116ce525
      2018-03-26 21:39:25.753 client-worker-1 o.a.s.m.n.KerberosSaslClientHandler [INFO] Connection established from /10.215.76.240:37614 to openqe74blue-n2.b
      lue.ygrid.yahoo.com/10.215.76.243:6699
      2018-03-26 21:39:25.753 client-worker-1 o.a.s.m.n.KerberosSaslNettyClient [INFO] Creating Kerberos Client.
      2018-03-26 21:39:25.763 client-worker-1 o.a.s.m.n.Login [INFO] successfully logged in.
      2018-03-26 21:39:25.765 client-worker-1 o.a.s.m.n.KerberosSaslNettyClient [INFO] Got Client: com.sun.security.sasl.gsskerb.GssKrb5Client@493cfe64
      2018-03-26 21:39:26.596 timer o.a.s.d.n.Nimbus [ERROR] Error while processing event
      java.lang.RuntimeException: java.lang.NullPointerException
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2508) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$1.run(StormTimer.java:207) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:81) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      Caused by: java.lang.NullPointerException
              at org.apache.storm.cluster.PaceMakerStateStorage.get_worker_hb_children(PaceMakerStateStorage.java:195) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.cluster.StormClusterStateImpl.heartbeatStorms(StormClusterStateImpl.java:408) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.topoIdsToClean(Nimbus.java:765) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.doCleanup(Nimbus.java:2148) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$launchServer$36(Nimbus.java:2506) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              ... 2 more
      2018-03-26 21:39:26.596 timer o.a.s.u.Utils [ERROR] Halting process: Error while processing event
      java.lang.RuntimeException: Halting process: Error while processing event
              at org.apache.storm.utils.Utils.exitProcess(Utils.java:469) ~[storm-client-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.daemon.nimbus.Nimbus.lambda$new$23(Nimbus.java:1154) ~[storm-server-2.0.0.y.jar:2.0.0.y]
              at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:106) ~[storm-client-2.0.0.y.jar:2.0.0.y]
      2018-03-26 21:39:26.600 Thread-16 o.a.s.u.Utils [INFO] Halting after 5 seconds
      2018-03-26 21:39:26.606 Thread-15 o.a.s.d.n.Nimbus [INFO] Shutting down master
      2018-03-26 21:39:31.600 Thread-16 o.a.s.u.Utils [WARN] Forcing Halt...
      

       

       

      This is because when https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java#L195-L198 happens,

       

      HBMessage ret = messages[next];
      if(ret == null) {
      // This can happen if we lost the connection and subsequently reconnected or timed out.
      send(m);
      }
      messages[next] = null;
      LOG.debug("Got Response: {}", ret);
      return ret;
      

      it returns null result. And the null result is inserted into https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClientPool.java#L65-L66

      for(String s : servers) {
      HBMessage response = getClientForServer(s).send(m);
      responses.add(response);
      }
      

      which leads to https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java#L195

       

      for(HBMessage response : responses) {
      if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
      LOG.error("get_worker_hb_children: Invalid Response Type");
      continue;
      }
      if(response.get_data().get_nodes().get_pulseIds() != null) {
      retSet.addAll(response.get_data().get_nodes().get_pulseIds());
      }
      }
      

       

      and this is where NPE happens 

      Attachments

        Issue Links

          Activity

            People

              ethanli Ethan Li
              ethanli Ethan Li
              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 - 50m
                  50m