This is an automated email from the ASF dual-hosted git repository.
penghui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git
The following commit(s) were added to refs/heads/master by this push:
new dcf01e9 Fix getInternalStats occasional lack of LeaderInfo (#13656)
dcf01e9 is described below
commit dcf01e910a4f8d30cd86ab95ab800952af92692a
Author: 包子 <[email protected]>
AuthorDate: Thu Jan 13 00:27:33 2022 +0800
Fix getInternalStats occasional lack of LeaderInfo (#13656)
### Motivation
#11704
AdminApiTest is flaky. The testGetPartitionedStatsInternal test method
fails sporadically.
The root cause it that `stateFuture` is returned, but
`availableBookiesFuture` is not accomplished, so we need ensure that `futures`
is all completed before `stateFuture.complete`.
https://github.com/apache/pulsar/blob/5faae1c4e683f4796ad34c1c612de0bcfe754d2a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L2176-L2178
---
.../org/apache/pulsar/broker/service/persistent/PersistentTopic.java | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 5c6f99a..990f2fc 100644
---
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -2023,9 +2023,10 @@ public class PersistentTopic extends AbstractTopic
implements Topic, AddEntryCal
stats.state = ml.getState().toString();
stats.ledgers = Lists.newArrayList();
- List<CompletableFuture<String>> futures = includeLedgerMetadata ?
Lists.newArrayList() : null;
+ List<CompletableFuture<String>> futures = Lists.newArrayList();
CompletableFuture<Set<String>> availableBookiesFuture =
brokerService.pulsar().getPulsarResources().getBookieResources().listAvailableBookiesAsync();
+ futures.add(availableBookiesFuture.handle((strings, throwable) ->
null));
availableBookiesFuture.whenComplete((bookies, e) -> {
if (e != null) {
log.error("[{}] Failed to fetch available bookies.", topic, e);
@@ -2038,7 +2039,7 @@ public class PersistentTopic extends AbstractTopic
implements Topic, AddEntryCal
info.size = li.getSize();
info.offloaded = li.hasOffloadContext() &&
li.getOffloadContext().getComplete();
stats.ledgers.add(info);
- if (futures != null) {
+ if (includeLedgerMetadata) {
futures.add(ml.getLedgerMetadata(li.getLedgerId()).handle((lMetadata, ex) -> {
if (ex == null) {
info.metadata = lMetadata;