[ 
https://issues.apache.org/jira/browse/STORM-4154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Diogo Pereira updated STORM-4154:
---------------------------------
    Description: 
When deploying or terminating a topology in a distributed cluster, we 
occasionally encounter downtime on the Nimbus machines. Below is an example 
stack trace:
{code:java}
2025-01-07T08:56:48.088Z Utils [ERROR] Received error in thread 
BLOB-STORE-TIMER.. terminating server...
java.lang.Error: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:666) 
~[storm-client-2.7.0.jar:2.7.0]
        at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:670) 
~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.utils.Utils.lambda$createDefaultUncaughtExceptionHandler$2(Utils.java:1053)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1055) [?:?]
        at 
java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1050) [?:?]
        at 
java.base/java.lang.Thread.dispatchUncaughtException(Thread.java:1997) [?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:199) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:106)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:128)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:285)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:139)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: org.apache.storm.thrift.TApplicationException: Internal error 
processing createStateInZookeeper
        at 
org.apache.storm.thrift.TServiceClient.receiveBase(TServiceClient.java:81) 
~[storm-shaded-deps-2.7.0.jar:2.7.0]
        at 
org.apache.storm.generated.Nimbus$Client.recv_createStateInZookeeper(Nimbus.java:1065)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.generated.Nimbus$Client.createStateInZookeeper(Nimbus.java:1052)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:136)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?] {code}
h3. Root Cause

This issue occurs due to a race condition when syncing the blobs. On some 
machines, the key we are trying to fetch information for during the process of 
creating the state in ZooKeeper for a recently downloaded blob might disappear. 
This results in a Thrift exception that is not being handled properly, causing 
the Nimbus process to crash.

The issue lies more specifically in this function:
{code:java}
public void createStateInZookeeper(String key) throws TException {
    try {
        IStormClusterState state = stormClusterState;
        BlobStore store = blobStore;
        NimbusInfo ni = nimbusHostPortInfo;
        if (store instanceof LocalFsBlobStore) {
            state.setupBlob(key, ni, getVersionForKey(key, ni, zkClient));
        }
        LOG.debug("Created state in zookeeper {} {} {}", state, store, ni);
    } catch (Exception e) {
        LOG.warn("Exception while creating state in zookeeper - key: " + key, 
e);
        if (e instanceof TException) {
            throw (TException) e;
        }
        throw new RuntimeException(e);
    }
} {code}
Here the *{{getVersionForKey}}* method can throw a 
{*}{{KeyNotFoundException}}{*}, which is not being handled properly. Instead, 
it is simply wrapped in a {{RuntimeException.}}

This exception then cascaded to the *blobSync* function, that doesn't handle 
the error, ultimately causing the main thread to terminate.

  was:
When deploying or terminating a topology in a distributed cluster, we 
occasionally encounter downtime on the Nimbus machines. Below is an example 
stack trace:
{code:java}
2025-01-07T08:56:48.088Z Utils [ERROR] Received error in thread 
BLOB-STORE-TIMER.. terminating server...
java.lang.Error: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:666) 
~[storm-client-2.7.0.jar:2.7.0]
        at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:670) 
~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.utils.Utils.lambda$createDefaultUncaughtExceptionHandler$2(Utils.java:1053)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1055) [?:?]
        at 
java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1050) [?:?]
        at 
java.base/java.lang.Thread.dispatchUncaughtException(Thread.java:1997) [?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:199) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:106)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
Internal error processing createStateInZookeeper
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:128)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:285)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: java.lang.RuntimeException: 
org.apache.storm.thrift.TApplicationException: Internal error processing 
createStateInZookeeper
        at 
org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:139)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
Caused by: org.apache.storm.thrift.TApplicationException: Internal error 
processing createStateInZookeeper
        at 
org.apache.storm.thrift.TServiceClient.receiveBase(TServiceClient.java:81) 
~[storm-shaded-deps-2.7.0.jar:2.7.0]
        at 
org.apache.storm.generated.Nimbus$Client.recv_createStateInZookeeper(Nimbus.java:1065)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.generated.Nimbus$Client.createStateInZookeeper(Nimbus.java:1052)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:136)
 ~[storm-client-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
 ~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174) 
~[storm-server-2.7.0.jar:2.7.0]
        at 
org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
~[storm-server-2.7.0.jar:2.7.0]
        at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
        at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?] {code}
h3. Root Cause

This issue occurs due to a race condition when syncing the blobs. On some 
machines, the key we are trying to fetch information for during the process of 
creating the state in ZooKeeper for a recently downloaded blob might disappear. 
This results in a Thrift exception that is not being handled properly, causing 
the Nimbus process to crash.

The issue lies more specifically in this function:
{code:java}
public void createStateInZookeeper(String key) throws TException {
    try {
        IStormClusterState state = stormClusterState;
        BlobStore store = blobStore;
        NimbusInfo ni = nimbusHostPortInfo;
        if (store instanceof LocalFsBlobStore) {
            state.setupBlob(key, ni, getVersionForKey(key, ni, zkClient));
        }
        LOG.debug("Created state in zookeeper {} {} {}", state, store, ni);
    } catch (Exception e) {
        LOG.warn("Exception while creating state in zookeeper - key: " + key, 
e);
        if (e instanceof TException) {
            throw (TException) e;
        }
        throw new RuntimeException(e);
    }
} {code}
Here the *{{getVersionForKey}}* method can throw a 
{*}{{KeyNotFoundException}}{*}, which is not being handled properly. Instead, 
it is simply wrapped in a {{RuntimeException. This exception then cascades to 
the *blobSync* function, that doesn't handle the error, ultimately causing the 
main thread to terminate.}}


> Nimbus down following topology deployment
> -----------------------------------------
>
>                 Key: STORM-4154
>                 URL: https://issues.apache.org/jira/browse/STORM-4154
>             Project: Apache Storm
>          Issue Type: Bug
>          Components: storm-server
>    Affects Versions: 2.7.0
>            Reporter: Diogo Pereira
>            Priority: Minor
>
> When deploying or terminating a topology in a distributed cluster, we 
> occasionally encounter downtime on the Nimbus machines. Below is an example 
> stack trace:
> {code:java}
> 2025-01-07T08:56:48.088Z Utils [ERROR] Received error in thread 
> BLOB-STORE-TIMER.. terminating server...
> java.lang.Error: java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
> Internal error processing createStateInZookeeper
>       at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:666) 
> ~[storm-client-2.7.0.jar:2.7.0]
>       at org.apache.storm.utils.Utils.handleUncaughtException(Utils.java:670) 
> ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.utils.Utils.lambda$createDefaultUncaughtExceptionHandler$2(Utils.java:1053)
>  ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1055) [?:?]
>       at 
> java.base/java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:1050) [?:?]
>       at 
> java.base/java.lang.Thread.dispatchUncaughtException(Thread.java:1997) [?:?]
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
> Internal error processing createStateInZookeeper
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:199) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: java.lang.RuntimeException: 
> org.apache.storm.thrift.TApplicationException: Internal error processing 
> createStateInZookeeper
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:106)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
> java.lang.RuntimeException: org.apache.storm.thrift.TApplicationException: 
> Internal error processing createStateInZookeeper
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:128)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
> org.apache.storm.thrift.TApplicationException: Internal error processing 
> createStateInZookeeper
>       at 
> org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:285)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
> Caused by: java.lang.RuntimeException: 
> org.apache.storm.thrift.TApplicationException: Internal error processing 
> createStateInZookeeper
>       at 
> org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:139)
>  ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?]
> Caused by: org.apache.storm.thrift.TApplicationException: Internal error 
> processing createStateInZookeeper
>       at 
> org.apache.storm.thrift.TServiceClient.receiveBase(TServiceClient.java:81) 
> ~[storm-shaded-deps-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.generated.Nimbus$Client.recv_createStateInZookeeper(Nimbus.java:1065)
>  ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.generated.Nimbus$Client.createStateInZookeeper(Nimbus.java:1052)
>  ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.NimbusBlobStore.createStateInZookeeper(NimbusBlobStore.java:136)
>  ~[storm-client-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.BlobStoreUtils.createStateInZookeeper(BlobStoreUtils.java:242)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.BlobStoreUtils.updateKeyForBlobStore(BlobStoreUtils.java:279)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.updateKeySetForBlobStore(LocalFsBlobStoreSynchronizer.java:125)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStoreSynchronizer.syncBlobs(LocalFsBlobStoreSynchronizer.java:84)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore.blobSync(LocalFsBlobStore.java:174)
>  ~[storm-server-2.7.0.jar:2.7.0]
>       at 
> org.apache.storm.blobstore.LocalFsBlobStore$1.run(LocalFsBlobStore.java:197) 
> ~[storm-server-2.7.0.jar:2.7.0]
>       at java.base/java.util.TimerThread.mainLoop(Timer.java:556) ~[?:?]
>       at java.base/java.util.TimerThread.run(Timer.java:506) ~[?:?] {code}
> h3. Root Cause
> This issue occurs due to a race condition when syncing the blobs. On some 
> machines, the key we are trying to fetch information for during the process 
> of creating the state in ZooKeeper for a recently downloaded blob might 
> disappear. This results in a Thrift exception that is not being handled 
> properly, causing the Nimbus process to crash.
> The issue lies more specifically in this function:
> {code:java}
> public void createStateInZookeeper(String key) throws TException {
>     try {
>         IStormClusterState state = stormClusterState;
>         BlobStore store = blobStore;
>         NimbusInfo ni = nimbusHostPortInfo;
>         if (store instanceof LocalFsBlobStore) {
>             state.setupBlob(key, ni, getVersionForKey(key, ni, zkClient));
>         }
>         LOG.debug("Created state in zookeeper {} {} {}", state, store, ni);
>     } catch (Exception e) {
>         LOG.warn("Exception while creating state in zookeeper - key: " + key, 
> e);
>         if (e instanceof TException) {
>             throw (TException) e;
>         }
>         throw new RuntimeException(e);
>     }
> } {code}
> Here the *{{getVersionForKey}}* method can throw a 
> {*}{{KeyNotFoundException}}{*}, which is not being handled properly. Instead, 
> it is simply wrapped in a {{RuntimeException.}}
> This exception then cascaded to the *blobSync* function, that doesn't handle 
> the error, ultimately causing the main thread to terminate.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to