Aaron Gresch created STORM-3168:
-----------------------------------

             Summary: AsyncLocalizer cleanup appears to crash
                 Key: STORM-3168
                 URL: https://issues.apache.org/jira/browse/STORM-3168
             Project: Apache Storm
          Issue Type: Bug
    Affects Versions: 2.0.0
            Reporter: Aaron Gresch
            Assignee: Aaron Gresch


I was investigating these blobstore download messages which keep repeating for 
hours in the supervisor (and nimbus logs).  I turned on debug logging, and was 
expecting a cleanup debug message every 30 seconds 
([https://github.com/apache/storm/blob/master/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java#L606).]
  It did not log.  I restarted the supervisor, and it started logging again.  
It appears to have crashed with some error.  

We should make sure the cleanup runs continuously and logs any failures to 
investigate.

 
{code:java}
2018-07-30 23:25:35.691 o.a.s.l.AsyncLocalizer AsyncLocalizer Executor - 2 
[ERROR] Could not update blob, will retry again later

java.util.concurrent.ExecutionException: java.lang.RuntimeException: Could not 
download...

        at 
java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) 
~[?:1.8.0_131]

        at 
java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895) 
~[?:1.8.0_131]

        at 
org.apache.storm.localizer.AsyncLocalizer.updateBlobs(AsyncLocalizer.java:303) 
~[storm-server-2.0.0.y.jar:2.0.0.y]

        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
[?:1.8.0_131]

        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
[?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
 [?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
 [?:1.8.0_131]

        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) 
[?:1.8.0_131]

        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
[?:1.8.0_131]

        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_131]

Caused by: java.lang.RuntimeException: Could not download...

        at 
org.apache.storm.localizer.AsyncLocalizer.lambda$downloadOrUpdate$69(AsyncLocalizer.java:268)
 ~[storm-server-2.0.0.y.jar:2.0.0.y]

        at 
java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1626)
 ~[?:1.8.0_131]

        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[?:1.8.0_131]

        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 ~[?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 ~[?:1.8.0_131]

        ... 3 more

Caused by: org.apache.storm.generated.KeyNotFoundException

        at 
org.apache.storm.generated.Nimbus$getBlobMeta_result$getBlobMeta_resultStandardScheme.read(Nimbus.java:25853)
 ~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.generated.Nimbus$getBlobMeta_result$getBlobMeta_resultStandardScheme.read(Nimbus.java:25821)
 ~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.generated.Nimbus$getBlobMeta_result.read(Nimbus.java:25752) 
~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.thrift.TServiceClient.receiveBase(TServiceClient.java:88) 
~[shaded-deps-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.generated.Nimbus$Client.recv_getBlobMeta(Nimbus.java:798) 
~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.generated.Nimbus$Client.getBlobMeta(Nimbus.java:785) 
~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.blobstore.NimbusBlobStore.getBlobMeta(NimbusBlobStore.java:85) 
~[storm-client-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.localizer.LocallyCachedTopologyBlob.getRemoteVersion(LocallyCachedTopologyBlob.java:122)
 ~[storm-server-2.0.0.y.jar:2.0.0.y]

        at 
org.apache.storm.localizer.AsyncLocalizer.lambda$downloadOrUpdate$69(AsyncLocalizer.java:252)
 ~[storm-server-2.0.0.y.jar:2.0.0.y]

        at 
java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1626)
 ~[?:1.8.0_131]

        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[?:1.8.0_131]

        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 ~[?:1.8.0_131]

        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 ~[?:1.8.0_131]

        ... 3 more
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to