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

[improve] Do not process acks in the Netty thread #22793

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
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 @@ -557,22 +557,29 @@ private CompletableFuture<Long> individualAckNormal(CommandAck ack, Map<String,

totalAckCount += ackedCount;
}
subscription.acknowledgeMessage(positionsAcked, AckType.Individual, properties);
CompletableFuture<Long> completableFuture = new CompletableFuture<>();
completableFuture.complete(totalAckCount);
if (isTransactionEnabled() && Subscription.isIndividualAckMode(subType)) {
completableFuture.whenComplete((v, e) -> positionsAcked.forEach(position -> {
//check if the position can remove from the consumer pending acks.
// the bit set is empty in pending ack handle.
if (((PositionImpl) position).getAckSet() != null) {
if (((PersistentSubscription) subscription)
.checkIsCanDeleteConsumerPendingAck((PositionImpl) position)) {
removePendingAcks((PositionImpl) position);

// we use acknowledgeMessageAsync because we don't want to perform the
// flush of the cursor (that may take much time in case of a long list of individuallyDeletedMessages)
// in the Netty eventloop thread
long totalCount = totalAckCount;
return subscription.acknowledgeMessageAsync(positionsAcked, AckType.Individual, properties)
.thenCompose(___ -> {
CompletableFuture<Long> completableFuture = new CompletableFuture<>();
completableFuture.complete(totalCount);
if (isTransactionEnabled() && Subscription.isIndividualAckMode(subType)) {
completableFuture.whenComplete((v, e) -> positionsAcked.forEach(position -> {
//check if the position can remove from the consumer pending acks.
// the bit set is empty in pending ack handle.
if (((PositionImpl) position).getAckSet() != null) {
if (((PersistentSubscription) subscription)
.checkIsCanDeleteConsumerPendingAck((PositionImpl) position)) {
removePendingAcks((PositionImpl) position);
}
}
}
}));
}
return completableFuture;
}));
}
return completableFuture;
});
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,12 @@ default void removeConsumer(Consumer consumer) throws BrokerServiceException {

void acknowledgeMessage(List<Position> positions, AckType ackType, Map<String, Long> properties);

default CompletableFuture<Void> acknowledgeMessageAsync(List<Position> positions,
AckType ackType, Map<String, Long> properties) {
acknowledgeMessage(positions, ackType, properties);
return CompletableFuture.completedFuture(null);
}

String getTopicName();

boolean isReplicated();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -367,6 +367,15 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) {
dispatcher.consumerFlow(consumer, additionalNumberOfMessages);
}

@Override
public CompletableFuture<Void> acknowledgeMessageAsync(List<Position> positions,
AckType ackType, Map<String, Long> properties) {
// which is the best thread ?
return CompletableFuture.runAsync(() -> {
acknowledgeMessage(positions, ackType, properties);
}, topic.getBrokerService().pulsar().getExecutor());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that an ordered executor (or pinned single thread executor) should be used so that all acknowledgements for a single subscription/cursor or topic get processed in a single thread.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lhotari something like topic.getBrokerService().getTopicOrderedExecutor().chooseThread(topicName)?
I'd love to avoid adding yet another executor.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dlg99 yes makes sense. I guess that the thread selection should involve the subscription name so that acknowledgements for a single subscription get handled by a single thread so that there would be less contention.

}

@Override
public void acknowledgeMessage(List<Position> positions, AckType ackType, Map<String, Long> properties) {
cursor.updateLastActive();
Expand Down
Loading