Skip to content
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

[Bug] Another issue result in broker lost bookie rack information in pulsar new version #23330

Open
3 tasks done
TakaHiR07 opened this issue Sep 23, 2024 · 2 comments
Open
3 tasks done
Labels
release/blocker Indicate the PR or issue that should block the release until it gets resolved type/bug The PR fixed a bug or issue reported a bug

Comments

@TakaHiR07
Copy link
Contributor

Search before asking

  • I searched in the issues and found nothing similar.

Read release policy

  • I understand that unsupported versions don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker.

Version

pulsar-3.0.6

Minimal reproduce step

restart broker many times

What did you expect to see?

when broker restart, can load rack information correctly

What did you see instead?

This is a supplement of #23282. After fix the previous issue, we found another issue would cause broker lost rack information. When broker restart many times, some broker may generate the following log, illustrate that it has lost rack information.

11:52:24.768 [ForkJoinPool.commonPool-worker-1] INFO  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Initialize rackaware ensemble placement policy @ null @ Unknown : org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping.
11:52:24.768 [ForkJoinPool.commonPool-worker-1] INFO  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Not weighted
11:52:24.768 [ForkJoinPool.commonPool-worker-1] INFO  org.apache.bookkeeper.client.BookKeeper - Weighted ledger placement is not enabled
11:52:24.775 [metadata-store-33-1] INFO  org.apache.pulsar.metadata.bookkeeper.PulsarRegistrationClient - Update BookieInfoCache (writable bookie) ip1:port1 -> BookieServiceInfo{properties={}, endpoints=[EndpointInfo{id=bookie, port=port1, host=ip1, protocol=bookie-rpc, auth=[], extensions=[]}]}
11:52:24.775 [metadata-store-33-1] INFO  org.apache.pulsar.metadata.bookkeeper.PulsarRegistrationClient - Update BookieInfoCache (writable bookie) ip1:port1 -> BookieServiceInfo{properties={}, endpoints=[EndpointInfo{id=bookie, port=port1, host=ip1, protocol=bookie-rpc, auth=[], extensions=[]}]}
11:52:24.796 [pulsar-registration-client-58-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
11:52:24.796 [pulsar-registration-client-58-1] INFO  org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: /default-region/default-rack/ip1:port1
11:52:24.797 [pulsar-registration-client-58-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
11:52:24.797 [pulsar-registration-client-58-1] INFO  org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: /default-region/default-rack/ip1:port2

Actually, the root reason is the same as #23282, two listeners in registrationClient must be executed in a sync way.

But the code implementation is async, when we restart broker, we do listener registering in PulsarRegistrationClient#watchWritableBookies. Notice that this is an async method. First listener is register in BookieRackAffinityMapping#watchAvailableBookies, using async way. Second listener is register in BookieWatcherImpl#initialBlockingBookieRead, using sync way.

That is the reason. In PulsarRegistrationClient#watchWritableBookies, two listeners is add into writableBookiesWatchers in a sync way. But when execute getWritableBookies().thenAcceptAsync(registrationListener::onBookiesChanged, executor), it is async.

It is easy to confirm this issue, just add some log in each part of PulsarRegistrationClient#watchWritableBookies. Since there are two listeners, We can see getWritableBookies() would be executed twice, in a async way.

@Override
public CompletableFuture<Void> watchWritableBookies(RegistrationListener registrationListener) {
writableBookiesWatchers.add(registrationListener);
return getWritableBookies()
.thenAcceptAsync(registrationListener::onBookiesChanged, executor);
}

Anything else?

By the way, previous version do not have this issue. Because previous version use ZKRegistrationClient as implementation of RegistrationClient, but not PulsarRegistrationClient. After PIP-45, pulsar change the implementation into PulsarRegistrationClient. Then would cause the issue.

Because ZKRegistrationClient#watchWritableBookies can ensure two listeners executed in a sync way. while PulsarRegistrationClient#watchWritableBookies can not.

Are you willing to submit a PR?

  • I'm willing to submit a PR!
@TakaHiR07 TakaHiR07 added the type/bug The PR fixed a bug or issue reported a bug label Sep 23, 2024
@lhotari lhotari added the release/blocker Indicate the PR or issue that should block the release until it gets resolved label Sep 23, 2024
@lhotari
Copy link
Member

lhotari commented Sep 23, 2024

@TakaHiR07 Thanks for reporting these issues and submitting the pull requests. Would you mind starting a dev mailing list thread to raise the awareness of these issues and so that we find a resolution?

@TakaHiR07
Copy link
Contributor Author

@TakaHiR07 Thanks for reporting these issues and submitting the pull requests. Would you mind starting a dev mailing list thread to raise the awareness of these issues and so that we find a resolution?

ok

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
release/blocker Indicate the PR or issue that should block the release until it gets resolved type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

No branches or pull requests

2 participants