Search before reporting
Read release policy
User environment
All released versions with topic list watcher support
Issue Description
Pattern consumers (regex subscriptions) supports topic list watchers after PIP-145.
To avoid transporting topic listings when there are no changes, a hash is calculated on the client side of the current state and passed to the broker side. If the hash matches the current state, the broker responds with an empty listing. When change events are sent from the broker to the client, the broker adds the hash so that the client can detect whether there's a state drift and a reconcilation operation would be needed to synchronize state.
PIP-145 also modified LookupService so that the hash can be passed when using the getTopicsUnderNamespace method.
|
/** |
|
* Returns all the topics that matches {@param topicPattern} for a given namespace. |
|
* |
|
* Note: {@param topicPattern} it relate to the topic name(without the partition suffix). For example: |
|
* - There is a partitioned topic "tp-a" with two partitions. |
|
* - tp-a-partition-0 |
|
* - tp-a-partition-1 |
|
* - If {@param topicPattern} is "tp-a", the consumer will subscribe to the two partitions. |
|
* - if {@param topicPattern} is "tp-a-partition-0", the consumer will subscribe nothing. |
|
* |
|
* @param namespace : namespace-name |
|
* @return |
|
*/ |
|
CompletableFuture<GetTopicsResult> getTopicsUnderNamespace(NamespaceName namespace, Mode mode, |
|
String topicPattern, String topicsHash); |
The problem that exist is that the topic listing results in topic list watcher uses org.apache.pulsar.broker.namespace.NamespaceService#getListOfPersistentTopics
and lookup service backend uses org.apache.pulsar.broker.namespace.NamespaceService#getListOfUserTopics .
The difference is that getListOfUserTopics filters out system topics:
https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java#L1535-L1537
Due to this difference in original topics used to calculate the hash, it won't be helpful at all.
Error messages
Reproducing the issue
This problem doesn't cause an actual impact in currently released clients since the hash of the topic list watcher results is ignored. Resolving this issue is relevant for #25188 changes.
Additional information
No response
Are you willing to submit a PR?
Search before reporting
Read release policy
User environment
All released versions with topic list watcher support
Issue Description
Pattern consumers (regex subscriptions) supports topic list watchers after PIP-145.
To avoid transporting topic listings when there are no changes, a hash is calculated on the client side of the current state and passed to the broker side. If the hash matches the current state, the broker responds with an empty listing. When change events are sent from the broker to the client, the broker adds the hash so that the client can detect whether there's a state drift and a reconcilation operation would be needed to synchronize state.
PIP-145 also modified LookupService so that the hash can be passed when using the getTopicsUnderNamespace method.
pulsar/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java
Lines 149 to 163 in 52a4d5e
The problem that exist is that the topic listing results in topic list watcher uses org.apache.pulsar.broker.namespace.NamespaceService#getListOfPersistentTopics
and lookup service backend uses org.apache.pulsar.broker.namespace.NamespaceService#getListOfUserTopics .
The difference is that getListOfUserTopics filters out system topics:
https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java#L1535-L1537
Due to this difference in original topics used to calculate the hash, it won't be helpful at all.
Error messages
Reproducing the issue
This problem doesn't cause an actual impact in currently released clients since the hash of the topic list watcher results is ignored. Resolving this issue is relevant for #25188 changes.
Additional information
No response
Are you willing to submit a PR?