lhotari opened a new issue, #25997: URL: https://github.com/apache/pulsar/issues/25997
### Search before reporting - [x] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar. ### Read release policy - [x] I understand that [unsupported versions](https://pulsar.apache.org/contribute/release-policy/#supported-versions) don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker. ### User environment - master branch (unreleased), regression introduced by #25929 (merged 2026-06-04, commit 17ead1d691c) - Java client (`pulsar-client`), affects any client embedding it — observed with the broker's internal client - Reproduced on GitHub Actions CI (ubuntu) and locally on macOS (Darwin 24.6, JDK 21) ### Issue Description #25929 (PIP-473 P5.3) added an `explicitHostURI` field to `ConnectionHandler` so the v5 transaction coordinator's metadata-store discovery can re-dial the known leader on retry instead of falling back to the service URL. However, the change also persists the broker-assigned redirect URL from PIP-307 unload notifications into the same field: - `connectionClosed(cnx, initialConnectionDelayMs, hostUrl)` stores `hostUrl` (the `assignedBrokerServiceUrl` from `CommandCloseProducer`/`CommandCloseConsumer`) into `explicitHostURI`: https://github.com/apache/pulsar/blob/67ec614b8be/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java#L227-L233 - `reconnectLater()` then re-dials `explicitHostURI` directly on **every** subsequent retry, bypassing topic lookup: https://github.com/apache/pulsar/blob/67ec614b8be/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java#L205-L217 - Nothing ever clears the field on failure, so a producer/consumer that was redirected once is pinned to that broker address permanently. Before #25929, the redirect URL was honored for the immediate reconnect attempt only; if connecting to the assigned broker failed, the next retry went through a normal topic lookup and recovered. The assigned broker in a PIP-307 redirect can be wrong or stale. Concrete case that now fails: when an `ExtensibleLoadManagerImpl` leader steps down (`playFollower()` → `closeInternalTopics()`), the disconnects it sends for the internal topics carry `assignedBrokerLookupData` pointing at **itself** (the channel-table assignment for the `pulsar/system` bundle is stale at that moment). Pre-#25929 this was harmless — one failed reconnect, then lookup found the new owner. Post-#25929 every internal-topic client (service-unit-state table view readers/writers, load-data store clients) is pinned to the old leader and rejects forever with `ServiceUnitNotReadyException`, until topic ownership coincidentally returns to the pinned broker. This is the cause of the new `ExtensibleLoadManagerImplTest.testRoleChange` flakiness on master, e.g. https://github.com/apache/pulsar/actions/runs/27284412071/job/80587641224 — the test's 30s await times out while the clients are wedged. The regression is not test-specific: any broker-initiated redirect to a broker that fails or refuses the topic (unload races, stale ownership views, broker restarts) now wedges the client instead of recovering via lookup. ### Error messages ``` 2026-06-10T14:59:27,294 - WARN - [pulsar-io-17-4:BrokerService] - Namespace bundle (pulsar/system/0x00000000_0xffffffff) for topic (persistent://pulsar/system/loadbalancer-service-unit-state) not served by this instance:localhost:38741. Please redo the lookup. 2026-06-10T14:59:27,294 - WARN - [pulsar-io-17-2:ClientCnx] - Received error from server {channel=[id: 0xcd4919c7, L:/127.0.0.1:59288 - R:localhost/127.0.0.1:44799], message=...ServiceUnitNotReadyException...} 2026-06-10T14:59:27,294 - WARN - [pulsar-io-17-2:ConnectionHandler] - Could not get connection to broker - Will try again {delaySec=0.096, ...} (repeats for the full 30s window — 226 rejections, every retry re-dialing the same stale address on the same pooled connection, no lookup in between) ``` The client-side pinning is initiated by the redirect: ``` 00:19:10,925 - INFO - [pulsar-load-manager-15-1:Producer] - Disconnecting producer {assignedBrokerLookupData=Optional[BrokerLookupData[brokerId=localhost:51736, ...]]} <- the stepping-down leader redirects clients to itself 00:19:10,929 - INFO - [pulsar-io-17-4:ConnectionHandler] - Closed connection - Will try again {hostUrl=pulsar://localhost:51734, ...} ``` ### Reproducing the issue On current master, run `ExtensibleLoadManagerImplTest.testRoleChange` repeatedly, e.g. temporarily set `invocationCount = 20` on the test and run: ``` ./gradlew :pulsar-broker:test --tests "org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImplTest.testRoleChange" -PexcludedTestGroups='' ``` Locally this fails 1-2 invocations per 20 (each failing invocation shows the 30s wedge described above). With the `connectionClosed()` persistence of `hostUrl` into `explicitHostURI` removed (restoring the pre-#25929 lookup fallback while keeping the one-shot redirect and the TC discovery re-dial, which sets `explicitHostURI` via `grabCnx(URI, boolean)`), the same runs pass 80/80 invocations. A focused client-level reproduction: connect a `ConnectionHandler`-based producer/consumer normally, then deliver a broker disconnect carrying an unreachable `assignedBrokerServiceUrl` — post-#25929 the handler never recovers; pre-#25929 it recovers on the next retry via lookup. ### Additional information - `git blame` attributes the `explicitHostURI` field, the `reconnectLater()` re-dial, and the `connectionClosed()` persistence to 17ead1d691c (#25929). - The TC discovery use case does not need the `connectionClosed()` persistence: `TransactionMetaStoreHandler` sets `explicitHostURI` itself via `grabCnx(URI, boolean)` on (re)discovery, and TC connections never receive `CloseProducer`/`CloseConsumer` redirects. - Separately from this client regression, the broker-side behavior of redirecting internal-topic clients to the stepping-down leader itself (stale `assignedBrokerLookupData` for the `pulsar/system` bundle in `closeInternalTopics()`) is questionable on its own and could be improved (e.g. omit the lookup data when it equals the local broker), but it predates #25929 and was harmless with the lookup fallback in place. ### Are you willing to submit a PR? - [x] I'm willing to submit a PR\! -- 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]
