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

[Broker] Fix call sync method in async rest api for internalTerminatePartitionedTopic #13890

Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3264,55 +3264,73 @@ protected MessageId internalTerminate(boolean authoritative) {
}

protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) {
CompletableFuture<Void> future;
if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
validateTopicOwnership(topicName, authoritative);
validateTopicOperation(topicName, TopicOperation.TERMINATE);

Map<Integer, MessageId> messageIds = new ConcurrentHashMap<>();

PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false);

if (partitionMetadata.partitions == 0) {
throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a non-partitioned topic is "
+ "not allowed using partitioned-terminate, please use terminate commands.");
}
if (partitionMetadata.partitions > 0) {
final List<CompletableFuture<MessageId>> futures = Lists.newArrayList();
future = validateGlobalNamespaceOwnershipAsync(namespaceName);
} else {
future = CompletableFuture.completedFuture(null);
}

future.thenAccept(__ -> validateTopicOwnershipAsync(topicName, authoritative)
liudezhi2098 marked this conversation as resolved.
Show resolved Hide resolved
.thenCompose(unused -> validateTopicOperationAsync(topicName, TopicOperation.TERMINATE))
.thenCompose(unused -> getPartitionedTopicMetadataAsync(topicName, authoritative, false))
.thenAccept(partitionMetadata -> {
if (partitionMetadata.partitions == 0) {
String msg = "Termination of a non-partitioned topic is not allowed using partitioned-terminate"
+ ", please use terminate commands";
log.error("[{}] [{}] {}", clientAppId(), topicName, msg);
asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg));
}
if (partitionMetadata.partitions > 0) {
Map<Integer, MessageId> messageIds = new ConcurrentHashMap<>(partitionMetadata.partitions);
final List<CompletableFuture<MessageId>> futures =
Lists.newArrayListWithCapacity(partitionMetadata.partitions);

for (int i = 0; i < partitionMetadata.partitions; i++) {
TopicName topicNamePartition = topicName.getPartition(i);
try {
int finalI = i;
futures.add(pulsar().getAdminClient().topics()
.terminateTopicAsync(topicNamePartition.toString()).whenComplete((messageId, throwable) -> {
if (throwable != null) {
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition,
throwable);
asyncResponse.resume(new RestException(throwable));
}
messageIds.put(finalI, messageId);
}));
} catch (Exception e) {
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, e);
throw new RestException(e);
}
}
FutureUtil.waitForAll(futures).handle((result, exception) -> {
if (exception != null) {
Throwable t = exception.getCause();
if (t instanceof NotFoundException) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found"));
} else {
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t);
asyncResponse.resume(new RestException(t));
}
}
asyncResponse.resume(messageIds);
for (int i = 0; i < partitionMetadata.partitions; i++) {
TopicName topicNamePartition = topicName.getPartition(i);
try {
int finalI = i;
futures.add(pulsar().getAdminClient().topics()
.terminateTopicAsync(topicNamePartition.toString())
.whenComplete((messageId, throwable) -> {
if (throwable != null) {
log.error("[{}] Failed to terminate topic {}", clientAppId(),
topicNamePartition, throwable);
asyncResponse.resume(new RestException(throwable));
}
messageIds.put(finalI, messageId);
}));
} catch (Exception e) {
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, e);
throw new RestException(e);
}
}
FutureUtil.waitForAll(futures).handle((result, exception) -> {
if (exception != null) {
Throwable t = exception.getCause();
if (t instanceof NotFoundException) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found"));
} else {
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t);
asyncResponse.resume(new RestException(t));
}
}
asyncResponse.resume(messageIds);
return null;
});
}
}).exceptionally(e -> {
Throwable cause = e.getCause();
liudezhi2098 marked this conversation as resolved.
Show resolved Hide resolved
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, cause);
resumeAsyncResponseExceptionally(asyncResponse, cause);
return null;
})
).exceptionally(e -> {
Throwable cause = e.getCause();
log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, cause);
resumeAsyncResponseExceptionally(asyncResponse, cause);
return null;
});
}
});
}

protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, String subName,
Expand Down