-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[improve][ml] Do not switch thread to execute asyncAddEntry's core logic #23940
[improve][ml] Do not switch thread to execute asyncAddEntry's core logic #23940
Conversation
I don't understand this part, could you pls explain more clear? |
@dao-jun I found this issue when I implemented the Kafka transaction. In Kafka, For example, assuming there are 3 ongoing writes, the flow looks like the following pseudo code: for (int i = 0; i < 3; i++) {
long nextOffset = interceptor.getIndex() + 1; // LEO
ongoingTxns.put(nextOffset, txn); // record the ongoing txn of this record batch
asyncAddEntry().thenAccept(__ -> ongoingTxns.remove(nextOffset));
} Assuming each record batch has only 1 message, ideally, after these 3 writes, The details above are beyond the scope of this PR but could help you understand the motivation. |
Thanks for your explain! |
Some shadow topic related tests failed, I will fix them |
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #23940 +/- ##
============================================
+ Coverage 73.57% 74.19% +0.61%
+ Complexity 32624 32236 -388
============================================
Files 1877 1853 -24
Lines 139502 143739 +4237
Branches 15299 16332 +1033
============================================
+ Hits 102638 106643 +4005
+ Misses 28908 28692 -216
- Partials 7956 8404 +448
Flags with carried forward coverage won't be shown. Click here to find out more.
|
@BewareMyPower Please also remove the comment "Jump to specific thread to avoid contention from writers writing from different threads" that is now obsolete: pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java Line 804 in cc7b381
|
@BewareMyPower This logic doesn't either make sense any more after the changes: pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java Lines 801 to 806 in cc7b381
|
@BewareMyPower There's a high chance that this change causes performance regressions: pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java Lines 809 to 811 in cc7b381
In Pulsar use cases, this could possibly happen when there's a large number of producers producing to a topic. |
Motivation
In protocol handler's implementation, sometimes it needs to record the current base offset before the actual
asyncAddEntry
call (or the wrappedPersistentTopic#publishMessages
call), for example:However, the existing implementation won't work as expected that even if the downstream protocol handler guarantees that
asyncAddEntry
is called in a single thread. Seepulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
Lines 804 to 809 in a19eaa2
First, the code above does not make sense because
internalAsyncAddEntry
itself is already synchronized.Second, to guarantee thread safety for concurrent add operations, switching to another thread is not more efficient than synchronizing the method call. Mostly, it's less efficient because
execute
involves with at least two method calls (offer
andpoll
) on aBlockingQueue
, as well as some other operations (like CAS). TakeArrayBlockingQueue
as example, both itsoffer
andpoll
implementations need to acquire the internal lock.Third, switching to another thread to execute the core logic is anti-intuitive, especially for modifying some fields that represent the current states. For example, to achieve the same goal at the beginning, I have to write the code like:
Modifications
Split
internalAsyncAddEntry
to two methodsbeforeAddEntryToQueue
andafterAddEntryToQueue
that are called before and after adding the operation topendingAddEntries
. Then simplify the code by throwing an exception and pass the exception to the callback. It's also more efficient becauseOpAddEntry#failed
won't be called in the synchronized block.Add
testBeforeAddEntry
to protect the change.Documentation
doc
doc-required
doc-not-needed
doc-complete
Matching PR in forked repository
PR in forked repository: