-
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
Fix call sync method in an async callback when enabling geo replicator. #12590
Changes from all commits
cd76506
aa6130d
abd5360
53ba4b4
b9b13e6
7836fd3
803ca3e
b8f5b26
43d0e96
baaf6b0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -632,10 +632,7 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n | |
return; | ||
} | ||
|
||
List<CompletableFuture<Void>> createFutureList = new ArrayList<>(); | ||
|
||
CompletableFuture<Void> createLocalFuture = new CompletableFuture<>(); | ||
createFutureList.add(createLocalFuture); | ||
checkTopicExistsAsync(topicName).thenAccept(exists -> { | ||
if (exists) { | ||
log.warn("[{}] Failed to create already existing topic {}", clientAppId(), topicName); | ||
|
@@ -658,7 +655,13 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n | |
return null; | ||
}); | ||
|
||
FutureUtil.waitForAll(createFutureList).whenComplete((ignored, ex) -> { | ||
List<String> replicatedClusters = new ArrayList<>(); | ||
if (!createLocalTopicOnly && topicName.isGlobal() && isNamespaceReplicated(namespaceName)) { | ||
getNamespaceReplicatedClusters(namespaceName) | ||
.stream().filter(cluster -> !cluster.equals(pulsar().getConfiguration().getClusterName())) | ||
.forEach(replicatedClusters::add); | ||
} | ||
createLocalFuture.whenComplete((ignored, ex) -> { | ||
if (ex != null) { | ||
log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName, ex.getCause()); | ||
if (ex.getCause() instanceof RestException) { | ||
|
@@ -669,14 +672,20 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n | |
return; | ||
} | ||
|
||
if (!createLocalTopicOnly && topicName.isGlobal() && isNamespaceReplicated(namespaceName)) { | ||
getNamespaceReplicatedClusters(namespaceName) | ||
.stream() | ||
.filter(cluster -> !cluster.equals(pulsar().getConfiguration().getClusterName())) | ||
.forEach(cluster -> createFutureList.add( | ||
((TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster).topics()) | ||
if (!replicatedClusters.isEmpty()) { | ||
replicatedClusters.forEach(cluster -> { | ||
pulsar().getPulsarResources().getClusterResources().getClusterAsync(cluster) | ||
.thenAccept(clusterDataOp -> { | ||
((TopicsImpl) pulsar().getBrokerService() | ||
.getClusterPulsarAdmin(cluster, clusterDataOp).topics()) | ||
.createPartitionedTopicAsync( | ||
topicName.getPartitionedTopicName(), numPartitions, true))); | ||
topicName.getPartitionedTopicName(), numPartitions, true); | ||
}) | ||
.exceptionally(throwable -> { | ||
log.error("Failed to create partition topic in cluster {}.", cluster, throwable); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we fail the operation ? |
||
return null; | ||
}); | ||
}); | ||
} | ||
|
||
log.info("[{}] Successfully created partitions for topic {} in cluster {}", | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1052,6 +1052,7 @@ public void deleteTopicAuthenticationWithRetry(String topic, CompletableFuture<V | |
} | ||
|
||
private CompletableFuture<Optional<Topic>> createNonPersistentTopic(String topic) { | ||
CompletableFuture<Optional<Topic>> topicFuture = new CompletableFuture<>(); | ||
if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("Broker is unable to load non-persistent topic {}", topic); | ||
|
@@ -1061,43 +1062,56 @@ private CompletableFuture<Optional<Topic>> createNonPersistentTopic(String topic | |
} | ||
final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); | ||
NonPersistentTopic nonPersistentTopic = new NonPersistentTopic(topic, this); | ||
|
||
CompletableFuture<Optional<Topic>> future = nonPersistentTopic.initialize() | ||
.thenCompose(__ -> nonPersistentTopic.checkReplication()) | ||
.thenApply(__ -> { | ||
log.info("Created topic {}", nonPersistentTopic); | ||
long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs; | ||
pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs); | ||
addTopicToStatsMaps(TopicName.get(topic), nonPersistentTopic); | ||
return Optional.of(nonPersistentTopic); | ||
}); | ||
|
||
future.exceptionally((ex) -> { | ||
log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex); | ||
nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { | ||
pulsar.getExecutor().execute(() -> topics.remove(topic, future)); | ||
CompletableFuture<Void> isOwner = checkTopicNsOwnership(topic); | ||
isOwner.thenRun(() -> { | ||
nonPersistentTopic.initialize() | ||
.thenCompose(__ -> nonPersistentTopic.checkReplication()) | ||
.thenRun(() -> { | ||
log.info("Created topic {}", nonPersistentTopic); | ||
long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs; | ||
pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs); | ||
addTopicToStatsMaps(TopicName.get(topic), nonPersistentTopic); | ||
topicFuture.complete(Optional.of(nonPersistentTopic)); | ||
}).exceptionally(ex -> { | ||
log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex.getCause()); | ||
nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { | ||
pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); | ||
topicFuture.completeExceptionally(ex); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we run this after the execution of "topics.remove" ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No effect before and after There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good, just checking |
||
}); | ||
return null; | ||
}); | ||
|
||
}).exceptionally(e -> { | ||
log.warn("CheckTopicNsOwnership fail when createNonPersistentTopic! {}", topic, e.getCause()); | ||
// CheckTopicNsOwnership fail dont create nonPersistentTopic, when topic do lookup will find the correct | ||
// broker. When client get non-persistent-partitioned topic | ||
// metadata will the non-persistent-topic will be created. | ||
// so we should add checkTopicNsOwnership logic otherwise the topic will be created | ||
// if it dont own by this broker,we should return success | ||
// otherwise it will keep retrying getPartitionedTopicMetadata | ||
topicFuture.complete(Optional.of(nonPersistentTopic)); | ||
// after get metadata return success, we should delete this topic from this broker, because this topic not | ||
// owner by this broker and it don't initialize and checkReplication | ||
pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); | ||
return null; | ||
}); | ||
|
||
return future; | ||
return topicFuture; | ||
} | ||
|
||
private <T> CompletableFuture<T> futureWithDeadline() { | ||
return FutureUtil.createFutureWithTimeout(FUTURE_DEADLINE_TIMEOUT_DURATION, executor(), | ||
() -> FUTURE_DEADLINE_TIMEOUT_EXCEPTION); | ||
} | ||
|
||
public PulsarClient getReplicationClient(String cluster) { | ||
public PulsarClient getReplicationClient(String cluster, Optional<ClusterData> clusterDataOp) { | ||
PulsarClient client = replicationClients.get(cluster); | ||
if (client != null) { | ||
return client; | ||
} | ||
|
||
return replicationClients.computeIfAbsent(cluster, key -> { | ||
try { | ||
ClusterData data = pulsar.getPulsarResources().getClusterResources().getCluster(cluster) | ||
ClusterData data = clusterDataOp | ||
.orElseThrow(() -> new MetadataStoreException.NotFoundException(cluster)); | ||
ClientBuilder clientBuilder = PulsarClient.builder() | ||
.enableTcpNoDelay(false) | ||
|
@@ -1164,14 +1178,14 @@ private void configTlsSettings(ClientBuilder clientBuilder, String serviceUrl, | |
} | ||
} | ||
|
||
public PulsarAdmin getClusterPulsarAdmin(String cluster) { | ||
public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional<ClusterData> clusterDataOp) { | ||
PulsarAdmin admin = clusterAdmins.get(cluster); | ||
if (admin != null) { | ||
return admin; | ||
} | ||
return clusterAdmins.computeIfAbsent(cluster, key -> { | ||
try { | ||
ClusterData data = pulsar.getPulsarResources().getClusterResources().getCluster(cluster) | ||
ClusterData data = clusterDataOp | ||
.orElseThrow(() -> new MetadataStoreException.NotFoundException(cluster)); | ||
|
||
ServiceConfiguration conf = pulsar.getConfig(); | ||
|
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.
can you explain better this line ?
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.
We may receive a notification from the metastore
/admin/policies/public
, not a complete namespace pathThere 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.
works for me, probably adding a comment will help future readers of this code