[fix][client] Avoid recycling the same ConcurrentBitSetRecyclable among different threads#24725
Merged
BewareMyPower merged 3 commits intoapache:masterfrom Sep 11, 2025
Conversation
…oss different threads
3 tasks
Contributor
There was a problem hiding this comment.
Pull Request Overview
This PR fixes a concurrency issue where ConcurrentBitSetRecyclable objects were being recycled incorrectly when multiple threads called flush() simultaneously. The fix changes the data structure from ConcurrentHashMap to ConcurrentSkipListMap and uses atomic polling operations to prevent race conditions.
- Replace
ConcurrentHashMapwithConcurrentSkipListMapforpendingIndividualBatchIndexAcks - Use
pollFirstEntry()instead of iterator-based removal to ensure atomic operations - Add null check to handle concurrent removal scenarios
Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.
...nt/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java
Outdated
Show resolved
Hide resolved
dao-jun
approved these changes
Sep 11, 2025
nodece
approved these changes
Sep 11, 2025
Codecov Report✅ All modified and coverable lines are covered by tests. Additional details and impacted files@@ Coverage Diff @@
## master #24725 +/- ##
============================================
- Coverage 74.27% 74.14% -0.14%
+ Complexity 33569 33117 -452
============================================
Files 1896 1896
Lines 148111 148109 -2
Branches 17164 17163 -1
============================================
- Hits 110009 109814 -195
- Misses 29370 29519 +149
- Partials 8732 8776 +44
Flags with carried forward coverage won't be shown. Click here to find out more.
🚀 New features to boost your workflow:
|
manas-ctds
pushed a commit
to datastax/pulsar
that referenced
this pull request
Sep 12, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3) (cherry picked from commit ba50b60)
srinath-ctds
pushed a commit
to datastax/pulsar
that referenced
this pull request
Sep 12, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3) (cherry picked from commit ba50b60)
ganesh-ctds
pushed a commit
to datastax/pulsar
that referenced
this pull request
Sep 15, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3) (cherry picked from commit 498571d)
ganesh-ctds
pushed a commit
to datastax/pulsar
that referenced
this pull request
Sep 15, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3) (cherry picked from commit 498571d)
srinath-ctds
pushed a commit
to datastax/pulsar
that referenced
this pull request
Sep 15, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3) (cherry picked from commit 498571d)
nodece
pushed a commit
to nodece/pulsar
that referenced
this pull request
Sep 16, 2025
…ng different threads (apache#24725) (cherry picked from commit 14543d3)
KannarFr
pushed a commit
to CleverCloud/pulsar
that referenced
this pull request
Sep 22, 2025
…ng different threads (apache#24725)
walkinggo
pushed a commit
to walkinggo/pulsar
that referenced
this pull request
Oct 8, 2025
…ng different threads (apache#24725)
15 tasks
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Fixes #24724
Modifications
Change
pendingIndividualBatchIndexAcksto aConcurrentSkipListMapand callpollFirst()in the loop to ensure concurrentflushcall won't copy the reference to theConcurrentBitSetRecyclabletwice tonewMultiMessageAckCommonin different threads, which recycles the object.Verifying this change
It's hard to write a unit test because this race condition is hard to simulate.
Documentation
docdoc-requireddoc-not-neededdoc-completeMatching PR in forked repository
PR in forked repository:
Additional information
This PR is a simple fix on the issue that avoids refactorings. However, it might not be worth allowing
flushto be called concurrently. It's hard to test the race condition and could sendThe advantage is just to avoid coarse grained lock on all methods that access the following fields:
pendingIndividualAckspendingIndividualBatchIndexAckslastCumulativeAckHowever, from my perspective, it's a pre-mature optimization that makes code error-prone. Because acquiring a lock for all
acknowledgeAsynccalls should not be a bottle neck of consumer side, whose time consuming tasks are mainly the business logic that handles messages andreceivecalls that have many lock acquirements as well.The use of
ConcurrentBitSetRecyclableis also a pre-mature optimization that results to the bug described in #24724.As I've shared in https://lists.apache.org/thread/b5r13oz24y935p6o8tfwf578xk35wwpf, sharing a recyclable object among different threads is not a good practice, which introduces more overhead and bypasses the performance benefits of the thread-local stacks. There might be still other potential issues with the use of
ConcurrentBitSetRecyclable, I might open another PR for the code refactoring.