Skip to content

Fix silent exception swallow in LocalFsBlobStore.prepare() leaving stormClusterState null#8416

Merged
rzo1 merged 2 commits intomasterfrom
fix/blob-store-cluster-state-init
Mar 3, 2026
Merged

Fix silent exception swallow in LocalFsBlobStore.prepare() leaving stormClusterState null#8416
rzo1 merged 2 commits intomasterfrom
fix/blob-store-cluster-state-init

Conversation

@jnioche
Copy link
Contributor

@jnioche jnioche commented Mar 2, 2026

Problem

In LocalFsBlobStore.prepare(), the call to ClusterUtils.mkStormClusterState() is wrapped in a try/catch that silently swallows any exception via e.printStackTrace():

try {
    this.stormClusterState = ClusterUtils.mkStormClusterState(conf, new ClusterStateContext(DaemonType.NIMBUS, conf));
} catch (Exception e) {
    e.printStackTrace();  // stormClusterState remains null
}

If this call fails — for example because ZooKeeper is unreachable at startup — stormClusterState stays null and prepare() returns successfully. The failure is only visible as a stack trace on stderr, which is easily lost in production log aggregation pipelines.

This creates a time-bomb: the blob store appears to initialise correctly, but every subsequent operation that touches the cluster state will throw a NullPointerException with no indication of the root cause:

  • startSyncBlobs()this.stormClusterState.blobstore(...)
  • setupBlobstore()state.activeKeys() (where state = stormClusterState)
  • blobSync()state.blobstore(...)

The NPE stack trace points into blob store internals, making it very hard to diagnose that the real problem was a ZooKeeper connection failure that happened earlier during prepare().

Fix

Rethrow the exception as a RuntimeException so that prepare() itself fails with a clear message and the original cause preserved:

try {
    this.stormClusterState = ClusterUtils.mkStormClusterState(conf, new ClusterStateContext(DaemonType.NIMBUS, conf));
} catch (Exception e) {
    throw new RuntimeException("Failed to initialize cluster state for LocalFsBlobStore", e);
}

This matches the pattern already used a few lines above in the same method for FileBlobStoreImpl initialisation, making the two failure modes consistent:

try {
    fbs = new FileBlobStoreImpl(baseDir, conf);
} catch (IOException e) {
    throw new RuntimeException(e);  // existing pattern
}

Nimbus now fails fast at startup with a meaningful error rather than entering a degraded state where blob operations crash with confusing NPEs.

If ClusterUtils.mkStormClusterState() threw (e.g. ZooKeeper unreachable),
the exception was caught and printed to stderr via e.printStackTrace(),
leaving stormClusterState null. Every subsequent blob store operation
(startSyncBlobs, setupBlobstore, blobSync) would then crash with a
NullPointerException rather than a meaningful error.

Rethrow as RuntimeException to match the existing pattern used a few
lines above for FileBlobStoreImpl initialization, and to surface the
root cause at startup instead of at first use.

Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
@rzo1
Copy link
Contributor

rzo1 commented Mar 3, 2026

@jnioche Looks like it needs an adjustment in

classname: org.apache.storm.localizer.AsyncLocalizerTest / testname: testKeyNotFoundException
org.opentest4j.AssertionFailedError: Unexpected exception type thrown, expected: <org.apache.storm.generated.KeyNotFoundException> but was: <java.lang.RuntimeException>
	at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:158)
	at org.junit.jupiter.api.AssertThrows.assertThrows(AssertThrows.java:68)
	at org.junit.jupiter.api.AssertThrows.assertThrows(AssertThrows.java:35)
	at org.junit.jupiter.api.Assertions.assertThrows(Assertions.java:3223)
	at org.apache.storm.localizer.AsyncLocalizerTest.testKeyNotFoundException(AsyncLocalizerTest.java:795)
Caused by: java.lang.RuntimeException: Failed to initialize cluster state for LocalFsBlobStore
	at org.apache.storm.blobstore.LocalFsBlobStore.prepare(LocalFsBlobStore.java:113)
	at org.apache.storm.blobstore.LocalFsBlobStore.prepare(LocalFsBlobStore.java:97)
	at org.apache.storm.localizer.AsyncLocalizerTest.lambda$testKeyNotFoundException$3(AsyncLocalizerTest.java:803)
	at org.junit.jupiter.api.AssertThrows.assertThrows(AssertThrows.java:54)
	... 3 more
Caused by: java.lang.RuntimeException: org.apache.storm.shade.org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /storm

Copy link
Contributor

@rzo1 rzo1 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see CI

…ore.prepare()

Both tests called prepare() without a reachable ZooKeeper, relying on the
now-fixed silent catch to let prepare() succeed with a null stormClusterState.

LocalFsBlobStoreSynchronizerTest: add InProcessZookeeper to @BeforeEach/@AfterEach
and pass its port in the conf used by initLocalFs(), matching the pattern
already established in LocalFsBlobStoreTest.

AsyncLocalizerTest.testKeyNotFoundException: wrap the test body in a
try-with-resources InProcessZookeeper and pass its port in the conf,
so that prepare() can initialise cluster state successfully before
getBlob() is called to exercise the KeyNotFoundException path.

Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
@jnioche
Copy link
Contributor Author

jnioche commented Mar 3, 2026

Sorry, had rushed this one a bit. About to push a fix

Root cause of the test failures: Both tests were accidentally relying on the silent exception swallow. They called prepare() without any ZooKeeper in the config, which caused mkStormClusterState to throw — but the old
e.printStackTrace() swallowed it and let the tests proceed with stormClusterState = null. The tests happened to pass because neither test's assertions touched cluster state.

@jnioche jnioche added the bug label Mar 3, 2026
@jnioche jnioche self-assigned this Mar 3, 2026
@rzo1 rzo1 added this to the 2.8.5 milestone Mar 3, 2026
@rzo1 rzo1 merged commit 2e7280b into master Mar 3, 2026
12 checks passed
@rzo1 rzo1 deleted the fix/blob-store-cluster-state-init branch March 3, 2026 09:14
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants