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
Make BookieId work with PulsarRegistrationDriver (second take) #17922
Conversation
/pulsar-bot rerun-failure-checks |
@Jason918 @codelipenghui @nicoloboschi @mattisonchao @michaeljmarshall you reviewed the first version of the patch. |
* Make BookieId work with PulsarRegistrationDriver * Switch to MetadataCache * checkstyle
5188f16
to
14701af
Compare
...r-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java
Show resolved
Hide resolved
@@ -153,4 +212,56 @@ private static Set<BookieId> convertToBookieAddresses(List<String> children) { | |||
} | |||
return newBookieAddrs; | |||
} | |||
|
|||
@Override | |||
public CompletableFuture<Versioned<BookieServiceInfo>> getBookieServiceInfo(BookieId bookieId) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am trying to get more sense from this issue, but I don't see this method in the stack of #17913. Call you point out the call path from metadata main thread?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
here (copied from #17913) we are waiting on the CompletableFuture returned by this method.
but without this PR the CF is expected to be completed by the main metadata store thread, that is already running the caller code (BookkeeperSchemaStorage.lambda$openLedger)
t java.util.concurrent.CompletableFuture.get(java.base@11.0.15.0.1/CompletableFuture.java:1998)
at org.apache.bookkeeper.common.concurrent.FutureUtils.result(FutureUtils.java:72)
at org.apache.bookkeeper.common.concurrent.FutureUtils.result(FutureUtils.java:61)
at org.apache.bookkeeper.client.DefaultBookieAddressResolver.resolve(DefaultBookieAddressResolver.java:43)
at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:532)
at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:658)
at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.initialize(DefaultPerChannelBookieClientPool.java:92)
at org.apache.bookkeeper.proto.BookieClientImpl.lookupClient(BookieClientImpl.java:217)
it is really complicated to see because there are so many indirections.
But the problem is here in the end.
We had the very same problem in BK when initially I implemented getBookieServiceInfo
in the BK core code base (you can take a look there)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see, getBookieServiceInfo
is called in a blocking way in DefaultBookieAddressResolver#resolve
.
|
||
public CompletableFuture<Versioned<BookieServiceInfo>> readBookieServiceInfoAsync(BookieId bookieId) { | ||
String asWritable = bookieRegistrationPath + "/" + bookieId; | ||
return bookieServiceInfoMetadataCache.get(asWritable) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You refer to this get
could be a blocking call.
My best guess is when the key is loading for the first time.
But the key call path should be like :
- org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl#get
- com.github.benmanes.caffeine.cache.AsyncLoadingCache#get
- org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl#readValueFromStore
and all of these should not contains any blocking operations.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
AsyncLoadingCache#get
this is nasty, I know, because even if it seems "async" the "completion" of the CompletableFuture is executed on the main metadata store thread.
In another operation that runs on the same thread you cannot wait for the completion of a CompletableFuture that wills be only completed on the same thread.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see, the THREAD (main-EventThread) is not blocked at this get
method actually. The get
method returns a CF without blocking, but the completion of the CF depends on the THREAD, and the THREAD is blocked at the completions of the CF. Thus the deadlock.
|
||
final CompletableFuture<Versioned<BookieServiceInfo>> readBookieInfoAsReadonlyBookie(BookieId bookieId) { | ||
String asReadonly = bookieReadonlyRegistrationPath + "/" + bookieId; | ||
return bookieServiceInfoMetadataCache.get(asReadonly) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it still necessary to get data from bookieServiceInfoMetadataCache
? It seems we already implemented a cache with bookieServiceInfoCache
, and we can read with store
directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Correct. I did it this way in order to leverage the built in support for SerDe of MetadataCache.
The overhead is not big and it is more elegant.
I can use raw reads and use the SerDe explicitly if you feel strong
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Correct. I did it this way in order to leverage the built in support for SerDe of MetadataCache. The overhead is not big and it is more elegant.
This make sense for me.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great work!
I think we have to cherry-pick this PR to branch-2.10 and 2.11, and this PR can also fix the ZK thread-blocking problem. |
Yes we can do it, to port it to branch 2.10 but there is no hurry. So I am not against porting it but I don't think it is worth, especially because it may introduce some little overhead and branch 2.10 is very stable |
Yes, I see. thanks! @eolivelli |
…e#17922) * Make BookieId work with PulsarRegistrationDriver (apache#17762) * Make BookieId work with PulsarRegistrationDriver * Switch to MetadataCache * checkstyle * Do not execute lookup on MetadataCache in the getBookieServiceInfo caller thread (cherry picked from commit 09f5eeb) (cherry picked from commit e3eb026)
…e#17922) * Make BookieId work with PulsarRegistrationDriver (apache#17762) * Make BookieId work with PulsarRegistrationDriver * Switch to MetadataCache * checkstyle * Do not execute lookup on MetadataCache in the getBookieServiceInfo caller thread (cherry picked from commit 09f5eeb) (cherry picked from commit e3eb026)
* Make BookieId work with PulsarRegistrationDriver (#17762) * Make BookieId work with PulsarRegistrationDriver * Switch to MetadataCache * checkstyle * Do not execute lookup on MetadataCache in the getBookieServiceInfo caller thread
Motivation
In Pulsar 2.10 the custom "BookieId" feature doesn't work anymore because in PulsarRegistrationDriver we are not implementing
getBookieServiceInfo
This patch also partially resolves #17759
Modifications
Implement
getBookieServiceInfo
This is a new version of #17762.
The main difference is the additional commit, in which we mimic the original code in BookKeeper
see https://github.com/apache/bookkeeper/blob/034ef8566ad037937a4d58a28f70631175744f53/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java#L183
We cannot perform reads from the MetadataStore in
getBookieServiceInfo
because it is very likely thatsomething will wait for the results of the returned CompletableFuture in a metadata store thread.
So we introduce a cache that is updated when the client scans the list of Bookies.
This works well in the original ZKRegistrationClient because the BookKeeper Client always looks for the available bookies
and this triggers the preloading of the local cache.
When we will upgrade to BK 4.15.1+ users will be able to leverage BP-41 flag to turn off BookKeeper Server address resolver
apache/bookkeeper#3356
Verifying this change
This change added tests
Documentation
doc-required
(Your PR needs to update docs and you will update later)
doc-not-needed
(Please explain why)
doc
(Your PR contains doc changes)
doc-complete
(Docs have been already added)
Matching PR in forked repository
PR in forked repository:eolivelli#17