vinkal-chudgar opened a new pull request, #24962:
URL: https://github.com/apache/pulsar/pull/24962

   
   
   ### Motivation
   `BinaryProtoLookupService#getTopicsUnderNamespace` currently does not 
deduplicate concurrent requests with identical parameters (`namespace`, `mode`, 
`topicsPattern`, `topicsHash`). For each call, it creates a new 
`CompletableFuture` and immediately invokes the helper to obtain a connection 
and send a request, which can cause request amplification when multiple 
components query the same namespace simultaneously.
   
   **Current behavior in code**
   - `BinaryProtoLookupService#getTopicsUnderNamespace` allocates a new future 
and invokes the private helper unconditionally (no shared in-flight future).
   - Sibling methods, in the same class, already deduplicate:
      - BinaryProtoLookupService#getBroker uses 
lookupInProgress.computeIfAbsent(...) with removal in whenComplete(...).
       - BinaryProtoLookupService#getPartitionedTopicMetadata uses 
partitionedMetadataInProgress.computeIfAbsent(...) with removal in 
whenComplete(...).
   
   This inconsistency means `BinaryProtoLookupService#getTopicsUnderNamespace` 
can issue duplicate network requests for identical inputs, while 
`BinaryProtoLookupService#getBroker` and 
`BinaryProtoLookupService#getPartitionedTopicMetadata` do not.
   
   **Real impact scenarios in the current codebase**
   - Pattern consumers issue identical lookups at the same time.
     `PatternMultiTopicsConsumerImpl `calls 
`client.getLookup().getTopicsUnderNamespace(...)` during periodic rechecks. 
When multiple pattern consumers refresh at the same time, they make concurrent 
calls with identical (`namespace`, `mode`, `topicsPattern`, `topicsHash`). 
Without deduplication, each call sends a separate request to the broker
     
   -  Several pattern consumers start at once.
   During initial subscribe, PulsarClientImpl#patternTopicSubscribeAsync also 
calls getTopicsUnderNamespace(...). Creating several pattern consumers around 
the same time leads to multiple concurrent calls with identical parameters; 
without deduplication, each call becomes a separate broker request.
   
   **Effect**
   - Request amplification: multiple identical lookups to the broker.
   - Increased broker load: unnecessary processing of duplicate requests.
   - Client overhead: duplicate I/O and response handling.
   - Inconsistency: behavior diverges from `BinaryProtoLookupService#getBroker` 
and `BinaryProtoLookupService#getPartitionedTopicMetadata`, which already 
coalesce identical in-flight requests.
   
   **What this PR changes**
   This PR adds deduplication to 
`BinaryProtoLookupService#getTopicsUnderNamespace`. Concurrent calls with the 
same (`namespace`, `mode`, `topicsPattern`, `topicsHash`) now share one 
`CompletableFuture` and one network request, reducing duplicate broker traffic 
and client work. The implementation follows the same pattern already used by 
`BinaryProtoLookupService#getBroker` and 
`BinaryProtoLookupService#getPartitionedTopicMetadata` (coalesce via 
`computeIfAbsent`, remove on `whenComplete`), keeping retry/backoff logic 
unchanged.
   
   ### Modifications
   
   - Added deduplication map
     ```
     private final ConcurrentHashMap<TopicsUnderNamespaceKey, 
CompletableFuture<GetTopicsResult>>
               topicsUnderNamespaceInProgress = new ConcurrentHashMap<>();
     ```
   - Implemented composite key class
     Added TopicsUnderNamespaceKey (static inner class) to uniquely identify 
requests by (namespace, mode, topicsPattern, topicsHash)
     
   - Refactored `BinaryProtoLookupService#getTopicsUnderNamespace` method:
      `BinaryProtoLookupService#getTopicsUnderNamespace` now:
      - Uses `ConcurrentHashMap.computeIfAbsent(...)` to atomically coalesce 
identical in-flight requests and return the same `CompletableFuture`.
       - Registers `whenComplete(...)` on the shared future to remove the map 
entry after completion (success or failure), matching the pattern used by 
`BinaryProtoLookupService#getBroker` and 
`BinaryProtoLookupService#getPartitionedTopicMetadata`.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   **Additional validation:**
   
   - `BinaryProtoLookupServiceTest#testGetTopicsUnderNamespaceDeduplication`
     Verifies that two in-flight calls with the same (namespace, mode, 
topicsPattern, topicsHash) return the same CompletableFuture and only one call 
is made to the connection pool; after the shared future completes, a new call 
with the same parameters returns a new CompletableFuture and makes one more 
call to the connection pool.
   
   - 
`BinaryProtoLookupServiceTest#testGetTopicsUnderNamespaceDeduplicationDifferentHash`
  
     Verifies that calls with different `topicsHash` values are not combined: 
each returns a different `CompletableFuture` and makes a separate call to the 
connection pool. Cleanup is per key; completing one future does not affect the 
other in-flight call.
         
   (Tests are deterministic: no sleeps or timing assumptions; they do not 
execute the network request path; resources are closed/terminated.)
   
   **Personal CI Results**
   
   Tested in Personal CI fork: https://github.com/vinkal-chudgar/pulsar/pull/4
   
   Status: All checks have passed (50 successful checks, 2 skipped)
   
   ### Does this pull request potentially affect one of the following parts:
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   *If the box was checked, please highlight the changes*
   
   - [ ] Dependencies (add or upgrade a dependency)
   - [ ] The public API
   - [ ] The schema
   - [ ] The default values of configurations
   - [ ] The threading model
   - [ ] The binary protocol
   - [ ] The REST endpoints
   - [ ] The admin CLI options
   - [ ] The metrics
   - [ ] Anything that affects deployment
   
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [ ] `doc` <!-- Your PR contains doc changes. -->
   - [ ] `doc-required` <!-- Your PR changes impact docs and you will update 
later -->
   - [ ] `doc-not-needed` <!-- Your PR changes do not impact docs -->
   - [ ] `doc-complete` <!-- Docs have been already added -->
   
   ### Matching PR in forked repository
   
   PR in forked repository: https://github.com/vinkal-chudgar/pulsar/pull/4
   


-- 
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]

Reply via email to