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] Broker would lost bookie rack information in pulsar new version #23282

Open
3 tasks done
TakaHiR07 opened this issue Sep 11, 2024 · 0 comments
Open
3 tasks done
Labels
type/bug The PR fixed a bug or issue reported a bug

Comments

@TakaHiR07
Copy link
Contributor

TakaHiR07 commented Sep 11, 2024

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

  1. stop a bookie in a cluster
  2. restart broker
  3. restart the stopped bookie
  4. can find that bookie's rack information is lost, become /defaultRegion/defaultRack

What did you expect to see?

..

What did you see instead?

After upgrade to pulsar-3.0.6,observe that when bookie restart, some bookie's rack information become /defaultRegion/defaultRack,which is not correct.

After diving into code and error log, this issue is probably due to this pr, #22846. This pr made BookieRackAffinityMapping#watchAvailableBookies become async. However, I think this operation can not be async.

Let's see what happen when bookieClient construct in pulsar. we can see the code in https://github.com/apache/bookkeeper/blob/1f1df813b9b4efd410925caadfa45cfb17b811ba/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java#L409-L548

  • Firstly, it would execute initializeEnsemblePlacementPolicy(), then it would trigger RackAwarePolicy#initialize. pulsar's BookieRackAffinityMapping would be used as dnsResolver in bookieClient. So BookieRackAffinityMapping#setConf and BookieRackAffinityMapping#watchAvailableBookies is executed. register first listener in registrationClient. This listener is used to update rackInfo in broker memory.
    @Override
    public synchronized void setConf(Configuration conf) {
    super.setConf(conf);
    MetadataStore store;
    try {
    store = getMetadataStore(conf);
    } catch (MetadataException e) {
    throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list");
    }
    bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class);
    store.registerListener(this::handleUpdates);
    try {
    var racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH)
    .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new))
    .get();
    for (var bookieMapping : racksWithHost.values()) {
    for (String address : bookieMapping.keySet()) {
    bookieAddressListLastTime.add(BookieId.parse(address));
    }
    if (LOG.isDebugEnabled()) {
    LOG.debug("BookieRackAffinityMapping init, bookieAddressListLastTime {}",
    bookieAddressListLastTime);
    }
    }
    updateRacksWithHost(racksWithHost);
    } catch (ExecutionException | InterruptedException e) {
    LOG.error("Failed to update rack info. ", e);
    throw new RuntimeException(e);
    }
    watchAvailableBookies();
    }
    private void watchAvailableBookies() {
    BookieAddressResolver bookieAddressResolver = getBookieAddressResolver();
    if (bookieAddressResolver instanceof DefaultBookieAddressResolver) {
    try {
    Field field = DefaultBookieAddressResolver.class.getDeclaredField("registrationClient");
    field.setAccessible(true);
    RegistrationClient registrationClient = (RegistrationClient) field.get(bookieAddressResolver);
    registrationClient.watchWritableBookies(versioned -> {
    bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH)
    .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new))
    .thenAccept(this::updateRacksWithHost)
    .exceptionally(ex -> {
    LOG.error("Failed to update rack info. ", ex);
    return null;
    });
    });
    } catch (NoSuchFieldException | IllegalAccessException e) {
    LOG.error("Failed watch available bookies.", e);
    }
    }
    }
  • Secondly, this.bookieWatcher.initialBlockingBookieRead() is executed,registrationClient.watchWritableBookies would register second listener. This listener is used to trigger placementPolicy.onClusterChanged(), which update rackInfo in placementPolicy. https://github.com/apache/bookkeeper/blob/1f1df813b9b4efd410925caadfa45cfb17b811ba/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java#L223-L242

When we receive notification for bookie creation in metadataStore, it would go into this code block, execute first listener, and then second listener.

sequencer.sequential(() -> {
switch (n.getType()) {
case Created:
log.info("Bookie {} created. path: {}", bookieId, n.getPath());
if (path.startsWith(bookieReadonlyRegistrationPath)) {
return getReadOnlyBookies().thenAccept(bookies ->
readOnlyBookiesWatchers.forEach(w ->
executor.execute(() -> w.onBookiesChanged(bookies))));
}
return getWritableBookies().thenAccept(bookies ->
writableBookiesWatchers.forEach(w ->
executor.execute(() -> w.onBookiesChanged(bookies))));
case Modified:

When we execute second listener to do placementPolicy.onClusterChanged(), it would finally go into here, execute resolver.resolve(names). This resolver's implementation is BookieRackAffinityMapping#resolve. https://github.com/apache/bookkeeper/blob/1f1df813b9b4efd410925caadfa45cfb17b811ba/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java#L554-L585

Therefore, we can see that the second listener actually depend on the first listener. They must be executed in a sync way.

But now we change to async way. So when a bookie restart, broker would permanently lost the rack information of this bookie, causing serious problem.

We add a log in BookieRackAffinityMapping#updateRacksWithHost, and confirm that the problem occur once the async code is executed later.

14:38:18.628 [metadata-store-38-1] INFO  org.apache.pulsar.metadata.bookkeeper.PulsarRegistrationClient - Bookie ip1:port1 created. path: /ledgers/available/ip1:port1
14:38:18.629 [metadata-store-38-1] INFO  org.apache.pulsar.metadata.bookkeeper.PulsarRegistrationClient - Bookie ip1:port1 created. path: /ledgers/available/ip1:port1
14:38:18.635 [metadata-store-38-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=[]}]}
14:38:18.636 [metadata-store-38-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=[]}]}
14:38:18.637 [pulsar-registration-client-46-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
14:38:18.637 [pulsar-registration-client-63-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
14:38:18.637 [pulsar-registration-client-63-1] INFO  org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: /default-region/default-rack/ip1:port1
14:38:18.637 [pulsar-registration-client-46-1] INFO  org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: /default-region/default-rack/ip1:port1
14:38:18.638 [pulsar-registration-client-63-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
14:38:18.638 [pulsar-registration-client-46-1] WARN  org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for ip1, using default rack for it : /default-region/default-rack.
14:38:18.640 [metadata-store-38-1] INFO  org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping - trigger updateRacksWithHost

Anything else?

pulsar-2.9 do not have this issue.

Are you willing to submit a PR?

  • I'm willing to submit a PR!
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

No branches or pull requests

1 participant