lhotari opened a new pull request, #26025:
URL: https://github.com/apache/pulsar/pull/26025
Main Issue: #25294
### Motivation
When a topic is loaded, the broker first waits for the namespace's topic
policies cache to be
initialized. Initialization
(`SystemTopicBasedTopicPoliciesService#initPolicesCache`) reads the
namespace's `__change_events` system topic to the end via a recursive
`readNextAsync()` loop and
completes a **shared, per-namespace** future (`policyCacheInitMap`) that
every topic load in the
namespace awaits.
That read loop has **no timeout**. If the system-topic reader reconnects but
then stops making
progress — e.g. after `__change_events` is unloaded/moved and the
reconnected reader gets stuck (see
the compacted-read stuck-reader bug fixed in #25998) — the shared init
future stays pending forever.
There is a 60s `topicLoadTimeoutSeconds` on the individual topic-load
future, but its handler only
logs; it does not clear the poisoned `policyCacheInitMap` entry or close the
stuck reader. As a
result the per-namespace state stays poisoned and **every** topic in the
namespace remains stuck and
unloadable until the broker is restarted, as reported in #25294.
This is a defense-in-depth fix that is complementary to #25998 (which fixes
one concrete stuck-reader
trigger on the broker/cursor side): the timeout bounds the wait regardless
of *why* a reader is stuck,
so a single stuck reader can no longer take a whole namespace's topics down
until restart.
### Modifications
- Add `topicPoliciesCacheInitTimeoutSeconds` (default `60`, dynamic). It
bounds topic policies cache
initialization for a namespace. Set to `0` or a negative value to disable
(previous unbounded
behavior).
- In `SystemTopicBasedTopicPoliciesService#prepareInitPoliciesCacheAsync`,
schedule a timeout on the
per-namespace init future. If the timeout wins the race to complete the
future, the broker:
- fails the init future with a `TimeoutException` (so awaiting topic loads
fail fast and can retry),
- clears the cached state and closes the stuck `__change_events` reader via
`cleanPoliciesCacheInitMap(namespace, true)`, so a subsequent load
creates a fresh reader instead
of being pinned until restart,
- increments a new `pulsar.broker.topic.policies.cache.init.timeout.count`
OpenTelemetry counter.
The timeout task is cancelled as soon as initialization completes (success
or failure), so it adds
no overhead on the normal path.
### Verifying this change
This change added tests and can be verified as follows:
- Added
`SystemTopicBasedTopicPoliciesServiceTest#testPrepareInitPoliciesCacheAsyncTimesOutWhenReaderStuck`,
which spies the `__change_events` reader so it reports more events but
never delivers one (a stuck
reader), then asserts that `prepareInitPoliciesCacheAsync` fails with a
`TimeoutException` (instead of
hanging), that the poisoned `policyCacheInitMap` entry is cleared, and
that the stuck reader is closed.
Verified red without the fix (the future never completes) and green with
it.
- The full `SystemTopicBasedTopicPoliciesServiceTest` suite passes,
including the existing init/cleanup
tests, confirming the normal path (and cleanup call counts) is unaffected.
### Does this pull request potentially affect one of the following parts:
- [x] The default values of configurations *(new
`topicPoliciesCacheInitTimeoutSeconds`, default 60s; topic policies cache
initialization is now bounded by default instead of unbounded)*
- [x] The metrics *(new counter
`pulsar.broker.topic.policies.cache.init.timeout.count`)*
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]