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

[cherry-pick][branch-2.9] make getList async #16221 #18811

Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -825,4 +825,21 @@ protected void validatePersistencePolicies(PersistencePolicies persistence) {
persistence.getBookkeeperAckQuorum()));

}

/**
* Check current exception whether is redirect exception.
*
* @param ex The throwable.
* @return Whether is redirect exception
*/
protected static boolean isRedirectException(Throwable ex) {
Throwable realCause = FutureUtil.unwrapCompletionException(ex);
return realCause instanceof WebApplicationException
&& ((WebApplicationException) realCause).getResponse().getStatus()
== Status.TEMPORARY_REDIRECT.getStatusCode();
}

protected static String getPartitionedTopicNotFoundErrorMessage(String topic) {
return String.format("Partitioned Topic %s not found", topic);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,19 @@ protected List<String> internalGetPartitionedTopicList() {
return getPartitionedTopicList(TopicDomain.getEnum(domain()));
}

protected CompletableFuture<List<String>> internalGetListAsync() {
return validateNamespaceOperationAsync(namespaceName, NamespaceOperation.GET_TOPICS)
.thenCompose(__ -> namespaceResources().namespaceExistsAsync(namespaceName))
.thenAccept(exists -> {
if (!exists) {
throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
}
})
.thenCompose(__ -> topicResources().listPersistentTopicsAsync(namespaceName))
.thenApply(topics -> topics.stream().filter(topic ->
!isTransactionInternalName(TopicName.get(topic))).collect(Collectors.toList()));
}

protected Map<String, Set<AuthAction>> internalGetPermissionsOnTopic() {
// This operation should be reading from zookeeper and it should be allowed without having admin privileges
validateAdminAccessForTenant(namespaceName.getTenant());
Expand Down Expand Up @@ -3701,17 +3714,22 @@ private CompletableFuture<Topic> topicNotFoundReasonAsync(TopicName topicName) {

return getPartitionedTopicMetadataAsync(
TopicName.get(topicName.getPartitionedTopicName()), false, false)
.thenApply(partitionedTopicMetadata -> {
.thenAccept(partitionedTopicMetadata -> {
if (partitionedTopicMetadata == null || partitionedTopicMetadata.partitions == 0) {
final String topicErrorType = partitionedTopicMetadata
== null ? "has no metadata" : "has zero partitions";
throw new RestException(Status.NOT_FOUND, String.format(
"Partitioned Topic not found: %s %s", topicName.toString(), topicErrorType));
} else if (!internalGetList().contains(topicName.toString())) {
}
})
.thenCompose(__ -> internalGetListAsync())
.thenApply(topics -> {
if (!topics.contains(topicName.toString())) {
throw new RestException(Status.NOT_FOUND, "Topic partitions were not yet created");
}
throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found");
});
throw new RestException(Status.NOT_FOUND,
getPartitionedTopicNotFoundErrorMessage(topicName.toString()));
});
}

private Topic getOrCreateTopic(TopicName topicName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,14 +73,16 @@ public class PersistentTopics extends PersistentTopicsBase {
@ApiResponse(code = 404, message = "Namespace doesn't exist")})
public void getList(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
try {
validateNamespaceName(property, cluster, namespace);
asyncResponse.resume(internalGetList());
} catch (WebApplicationException wae) {
asyncResponse.resume(wae);
} catch (Exception e) {
asyncResponse.resume(new RestException(e));
}
validateNamespaceName(property, cluster, namespace);
internalGetListAsync()
.thenAccept(asyncResponse::resume)
.exceptionally(ex -> {
if (!isRedirectException(ex)) {
log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, ex);
}
resumeAsyncResponseExceptionally(asyncResponse, ex);
return null;
});
}

@GET
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,14 +95,16 @@ public void getList(
@PathParam("tenant") String tenant,
@ApiParam(value = "Specify the namespace", required = true)
@PathParam("namespace") String namespace) {
try {
validateNamespaceName(tenant, namespace);
asyncResponse.resume(internalGetList());
} catch (WebApplicationException wae) {
asyncResponse.resume(wae);
} catch (Exception e) {
asyncResponse.resume(new RestException(e));
}
validateNamespaceName(tenant, namespace);
internalGetListAsync()
.thenAccept(asyncResponse::resume)
.exceptionally(ex -> {
if (!isRedirectException(ex)) {
log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, ex);
}
resumeAsyncResponseExceptionally(asyncResponse, ex);
return null;
});
}

@GET
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -797,8 +797,9 @@ public void persistentTopics() throws Exception {
.createPolicies(NamespaceName.get(property, cluster, namespace), new Policies());

AsyncResponse response = mock(AsyncResponse.class);

persistentTopics.getList(response, property, cluster, namespace);
verify(response, times(1)).resume(Lists.newArrayList());
verify(response, timeout(5000).times(1)).resume(Lists.newArrayList());
// create topic
assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList());
response = mock(AsyncResponse.class);
Expand Down