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

[fix][broker][branch-2.10] Replace sync method call in async call chain to prevent ZK event thread deadlock #19539

Merged
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 @@ -2414,78 +2414,82 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse,
protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String subName, boolean authoritative,
MessageIdImpl messageId, boolean isExcluded, int batchIndex) {
CompletableFuture<Void> ret;
if (topicName.isGlobal()) {
ret = validateGlobalNamespaceOwnershipAsync(namespaceName);
// If the topic name is a partition name, no need to get partition topic metadata again
if (!topicName.isPartitioned()) {
ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false)
.thenCompose(topicMetadata -> {
if (topicMetadata.partitions > 0) {
log.warn("[{}] Not supported operation on partitioned-topic {} {}",
clientAppId(), topicName, subName);
throw new CompletionException(new RestException(Status.METHOD_NOT_ALLOWED,
"Reset-cursor at position is not allowed for partitioned-topic"));
} else {
return CompletableFuture.completedFuture(null);
}
});
} else {
ret = CompletableFuture.completedFuture(null);
}
if (topicName.isGlobal()) {
ret = ret.thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(namespaceName));
}
ret.thenAccept(__ -> {
log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), topicName,
subName, messageId);
// If the topic name is a partition name, no need to get partition topic metadata again
if (!topicName.isPartitioned()
&& getPartitionedTopicMetadata(topicName, authoritative, false).partitions > 0) {
log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), topicName,
subName);
asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED,
"Reset-cursor at position is not allowed for partitioned-topic"));
return;
} else {
validateTopicOwnershipAsync(topicName, authoritative)
.thenCompose(ignore ->
validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName))
.thenCompose(ignore -> getTopicReferenceAsync(topicName))
.thenAccept(topic -> {
if (topic == null) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found"));
return;
}
PersistentSubscription sub = ((PersistentTopic) topic).getSubscription(subName);
if (sub == null) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found"));
return;
}
CompletableFuture<Integer> batchSizeFuture = new CompletableFuture<>();
getEntryBatchSize(batchSizeFuture, (PersistentTopic) topic, messageId, batchIndex);
batchSizeFuture.thenAccept(bi -> {
PositionImpl seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex,
messageId);
sub.resetCursor(seekPosition).thenRun(() -> {
log.info("[{}][{}] successfully reset cursor on subscription {}"
+ " to position {}", clientAppId(),
topicName, subName, messageId);
asyncResponse.resume(Response.noContent().build());
}).exceptionally(ex -> {
Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex);
log.warn("[{}][{}] Failed to reset cursor on subscription {}"
+ " to position {}", clientAppId(),
topicName, subName, messageId, t);
if (t instanceof SubscriptionInvalidCursorPosition) {
asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED,
"Unable to find position for position specified: "
+ t.getMessage()));
} else if (t instanceof SubscriptionBusyException) {
asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED,
"Failed for Subscription Busy: " + t.getMessage()));
} else {
resumeAsyncResponseExceptionally(asyncResponse, t);
}
return null;
});
}).exceptionally(e -> {
asyncResponse.resume(e);
validateTopicOwnershipAsync(topicName, authoritative)
.thenCompose(ignore ->
validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName))
.thenCompose(ignore -> getTopicReferenceAsync(topicName))
.thenAccept(topic -> {
if (topic == null) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found"));
return;
}
PersistentSubscription sub = ((PersistentTopic) topic).getSubscription(subName);
if (sub == null) {
asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found"));
return;
}
CompletableFuture<Integer> batchSizeFuture = new CompletableFuture<>();
getEntryBatchSize(batchSizeFuture, (PersistentTopic) topic, messageId, batchIndex);
batchSizeFuture.thenAccept(bi -> {
PositionImpl seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex,
messageId);
sub.resetCursor(seekPosition).thenRun(() -> {
log.info("[{}][{}] successfully reset cursor on subscription {}"
+ " to position {}", clientAppId(),
topicName, subName, messageId);
asyncResponse.resume(Response.noContent().build());
}).exceptionally(ex -> {
Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex);
log.warn("[{}][{}] Failed to reset cursor on subscription {}"
+ " to position {}", clientAppId(),
topicName, subName, messageId, t);
if (t instanceof SubscriptionInvalidCursorPosition) {
asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED,
"Unable to find position for position specified: "
+ t.getMessage()));
} else if (t instanceof SubscriptionBusyException) {
asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED,
"Failed for Subscription Busy: " + t.getMessage()));
} else {
resumeAsyncResponseExceptionally(asyncResponse, t);
}
return null;
});
}).exceptionally(ex -> {
// If the exception is not redirect exception we need to log it.
if (!isRedirectException(ex)) {
log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}",
clientAppId(), topicName, subName, messageId, ex.getCause());
}
resumeAsyncResponseExceptionally(asyncResponse, ex.getCause());
return null;
});
}
}).exceptionally(e -> {
asyncResponse.resume(e);
return null;
});
}).exceptionally(ex -> {
// If the exception is not redirect exception we need to log it.
if (!isRedirectException(ex)) {
log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}",
clientAppId(), topicName, subName, messageId, ex.getCause());
}
resumeAsyncResponseExceptionally(asyncResponse, ex.getCause());
return null;
});
}).exceptionally(ex -> {
// If the exception is not redirect exception we need to log it.
if (!isRedirectException(ex)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,22 @@ public void testResetCursorOnPosition(String namespaceName) throws Exception {
consumer.close();
}

@Test
public void shouldNotSupportResetOnPartitionedTopic() throws PulsarAdminException, PulsarClientException {
final String partitionedTopicName = "persistent://prop-xyz/ns1/" + BrokerTestUtil.newUniqueName("parttopic");
admin.topics().createPartitionedTopic(partitionedTopicName, 4);
@Cleanup
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(partitionedTopicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Shared).subscribe();
try {
admin.topics().resetCursor(partitionedTopicName, "my-sub", MessageId.earliest);
fail();
} catch (PulsarAdminException.NotAllowedException e) {
assertTrue(e.getMessage().contains("Reset-cursor at position is not allowed for partitioned-topic"),
"Condition doesn't match. Actual message:" + e.getMessage());
}
}

private void publishMessagesOnPersistentTopic(String topicName, int messages, int startIdx) throws Exception {
Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topicName)
Expand Down