Skip to content

KAFKA-15696: Refactor closing consumer#14937

Merged
lucasbru merged 14 commits intoapache:trunkfrom
philipnee:refactor-closing-consumer
Dec 19, 2023
Merged

KAFKA-15696: Refactor closing consumer#14937
lucasbru merged 14 commits intoapache:trunkfrom
philipnee:refactor-closing-consumer

Conversation

@philipnee
Copy link
Copy Markdown
Contributor

@philipnee philipnee commented Dec 6, 2023

We drives the consumer closing via events, and rely on the still-alived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shutdown, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. tell the background thread to perform n events and 2. block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

  • AsynckafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
  • AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
  • CommitRequestManager: signalClose()
  • FetchRequestManagerTest: changes due to change in pollOnClose()
  • ApplicationEventProcessor: handle commitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we timeout on the application thread)

@Test
public void testGroupIdNotNullAndValid() {
// close the default consumer
shutDown();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Isn't this anyways going to happen in afterAll?

Copy link
Copy Markdown
Contributor Author

@philipnee philipnee Dec 6, 2023

Choose a reason for hiding this comment

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

The test spins up another consumer so we should shutdown the BeforeEach one.

@lucasbru
Copy link
Copy Markdown
Member

lucasbru commented Dec 6, 2023

Yes, I think using events is much clearer. @kirktrue do you agree with this approach? Then I'd suggest we close the other PR and continue with this one.

@philipnee philipnee force-pushed the refactor-closing-consumer branch from 3207f21 to 2b590bf Compare December 6, 2023 18:20
final AtomicReference<Throwable> firstException) {
try {
applicationEventHandler.addAndGet(event, timer);
} catch (TimeoutException e) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

We don't really throw timeout exceptions during closing because if user tries to close with 0 duration then all ops will be timedout. The current implementation just polls, but since we cannot directly polls the client, we need to either wait till the future is completed or times out and keep going.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, we have issue with timeouts of 0 elsewhere. There's a Jira somewhere to solve it, but it's not been designed/fixed.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@kirktrue and I discussed the potential tasks for dealing with zero timeout. This needs to be examined perhaps after the preview. So we will spin off a jira ticket for this specific issue.

@philipnee philipnee changed the title Refactor closing consumer KAFKA-15696: Refactor closing consumer Dec 6, 2023
@philipnee philipnee marked this pull request as ready for review December 6, 2023 23:48
@philipnee philipnee force-pushed the refactor-closing-consumer branch from 0158de2 to a2e3ed1 Compare December 6, 2023 23:48
@philipnee
Copy link
Copy Markdown
Contributor Author

Hi @kirktrue - I rewrote the previous PR based on your feedback. I thought driving the close via event is a better and clearer pattern, so thanks for the suggestions. Would you have time to take a look at this PR?

@lucasbru - Thanks for reviewing the PR - I've decided according to your suggestion to use Kirk's approach to close the consumer. Let me know what do you think.

@@ -274,79 +269,18 @@ private void closeInternal(final Duration timeout) {
}

void cleanup() {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

really not much to do when shutting down the network thread - we will try one more time to send the unsent and poll the network client to make sure all requests and sent

Copy link
Copy Markdown
Contributor

@kirktrue kirktrue left a comment

Choose a reason for hiding this comment

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

This is really tricky, @philipnee 😞

I think we need to resolve the behavioral ambiguity around a user invoking close(0) ASAHP.

cc @lianetm @AndrewJSchofield @lucasbru @cadonna

} catch (Exception e) {
log.error("Unexpected error during shutdown. Proceed with closing.", e);
} finally {
networkClientDelegate.awaitPendingRequests(timer);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Network requests are tied to the CompletableApplicationEvents, right? Can we just rely on the events to wait for their network I/O to complete via the addAndGet() method.?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can there be other requests (not tied to the closing application events) that we want to wait for as long as we still have time?

Copy link
Copy Markdown
Contributor Author

@philipnee philipnee Dec 7, 2023

Choose a reason for hiding this comment

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

I think we actually don't need this here because runAtClose is already closing after checking the code. if the timer runs out, then we don't need to poll again. if all request are completed before timer runs out, then we don't need to repoll again.

case COMMIT:
log.debug("Sending unsent commit before closing.");
sendUnsentCommit();
event.future().complete(null);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This is a bit of a different pattern than our other CompletableApplicationEvents. In the other events, we completed the Future when the response was processed. In these events, we're completing them just after sending off the request. Is that truly what we want?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Yeah, as long as our timeout did not expire, we probably want to wait for the response, right?

Comment on lines +121 to +127
case PREP_CLOSING:
processPrepClosingEvent((ConsumerCloseApplicationEvent) event);
return;

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Any reason we can't have these as separate types like the other events?

Comment on lines +253 to +274
private void sendUnsentCommit() {
if (!requestManagers.commitRequestManager.isPresent())
return;
NetworkClientDelegate.PollResult res = requestManagers.commitRequestManager.get().pollOnClose();
if (res.unsentRequests.isEmpty())
return;
// NetworkThread will continue to poll the networkClientDelegate
networkClientDelegate.addAll(res);
}

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'm not quite understanding why this needs to be done as a special case. Why can't we rely on the normal runOnce() invocation to poll() the request managers?

private final Logger log;
private final ConsumerMetadata metadata;
private final RequestManagers requestManagers;
private final NetworkClientDelegate networkClientDelegate;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'm uncomfortable with introducing the NetworkClientDelegate at this layer. It's centralized in ConsumerNetworkThread for the reason that we can reason on where the various network I/O is performed.

*/
package org.apache.kafka.clients.consumer.internals.events;

public class ConsumerCloseApplicationEvent extends CompletableApplicationEvent<Void> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'm happy to have a superclass for 'close' events, but having a type and a task gets a bit muddy, doesn't it?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why not just have separate event types as per the rest of the codebase?

final AtomicReference<Throwable> firstException) {
try {
applicationEventHandler.addAndGet(event, timer);
} catch (TimeoutException e) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, we have issue with timeouts of 0 elsewhere. There's a Jira somewhere to solve it, but it's not been designed/fixed.

Copy link
Copy Markdown
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

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

I had a look and left a few comments, but @kirktrue asked all the right questions already

// Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until
// all requests have received a response.
do {
while (timer.notExpired() && !requestFutures.stream().allMatch(Future::isDone)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Why are you changing this back?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

if the close timer has expired, should we proceed with closing without sending the request? I'm undecided on this. @kirktrue wdyt?

return EMPTY;

List<NetworkClientDelegate.UnsentRequest> requests = pendingRequests.drainOnClose();
System.out.print("ddraining + " + requests);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

remove

}
}

private CompletableFuture<Void> onLeavePrepare() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

not sure if that is the best name to describe what it does

} catch (Exception e) {
log.error("Unexpected error during shutdown. Proceed with closing.", e);
} finally {
networkClientDelegate.awaitPendingRequests(timer);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can there be other requests (not tied to the closing application events) that we want to wait for as long as we still have time?

case COMMIT:
log.debug("Sending unsent commit before closing.");
sendUnsentCommit();
event.future().complete(null);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Yeah, as long as our timeout did not expire, we probably want to wait for the response, right?

droppedPartitions.addAll(subscriptions.assignedPartitions());
if (!subscriptions.hasAutoAssignedPartitions() || droppedPartitions.isEmpty())
return CompletableFuture.completedFuture(null);
// TODO: Invoke rebalanceListener via KAFKA-15276
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@kirktrue - I am not 100% sure what is the right way to invoke the listener. Are we returning a completable future? The current implementation blocks on listener invocation, which means where we need to do future.get(forever). If the listener is broken in some way, then we are stuck here.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we merge it without resolving this comment?

final Timer timer) {
// These are the optional outgoing requests at the
List<NetworkClientDelegate.PollResult> pollResults = requestManagers.stream()
requestManagers.stream()
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

see the comment in the FetchRequestManager

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we merge it without resolving this comment?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is not a conflict actually - this is just some changes to how fetch request manager closes

*/
@Override
public PollResult pollOnClose() {
// TODO: move the logic to poll to handle signal close
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I added a method signalClose() to the interface. I wonder if we should keep letting the network thread poll the network client as usual, until we actually invoke close. This means, close will do very little but check if there are any pending requests.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Yes, using the normal poll loop sounds like a good idea. We should still probably sendUnsentRequests once when the timeout has passed.

final String groupId = "consumerGroupA";
final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
final LinkedBlockingQueue<BackgroundEvent> backgroundEventQueue = new LinkedBlockingQueue<>();
try (final AsyncKafkaConsumer<String, String> consumer =
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

removing the try to close the consumer with 0 timeout. @lucasbru

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Please be aware that all these changes will be unnecessary/conflicting once we move to mocks, which is rather soon I believe, see:

#14930

We will get lots of conflicts and I think whoever has to resolve the conflicts will probably just skip the changes from this PR, because the commit logic in the background thread will not be executed anymore in this test.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

ok makes sense, I can revert these changes.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I didn't mean to revert the changes, but discard them when we merge with the asyncKafkaConsumer refactoring. Is it possible these changes were required to not run OOM?

One option would be to rebase this PR on the refactoring, which can possibly resolve these OOMs.

ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer);

// the network is polled during the last state of clean up.
networkClientDelegate.poll(time.timer(1));
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Does kirk need to confirm this change before we can merge the PR?

@philipnee
Copy link
Copy Markdown
Contributor Author

Hi @kirktrue @lucasbru - Sorry about the huge PR. But I've addressed most of your comments if not all (apologize if not). I've left comments on places where a bit more discussions are needed. LMK if you have any questions.

@philipnee philipnee force-pushed the refactor-closing-consumer branch 2 times, most recently from c0e3780 to 86e6943 Compare December 7, 2023 23:13
Copy link
Copy Markdown
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

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

I had a look, seems like we are still iterating on the design. Left some comments

final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
metadata,
applicationEventQueue,
applicationEventQueue,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

indentation

logContext,
metadata,
applicationEventQueue,
applicationEventQueue,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

indentation

@@ -178,27 +171,11 @@ static void runAtClose(final Collection<Optional<? extends RequestManager>> requ
final NetworkClientDelegate networkClientDelegate,
final Timer timer) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Seems timer is now completely unused.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

noted, let's get rid of runAtClose all together in the future PR.

private void sendUnsentRequests(final Timer timer) {
// Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until
// all requests have received a response.
while (!networkClientDelegate.unsentRequests().isEmpty() && timer.notExpired()) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Closing with timeout 0 would mean we don't send any closing requests, right? I think we should poll nevertheless, so we should check the timer at the end.

I think if we'd use the normal poll loop as long as timeout > 0, this function may not need to check the timer anyway, since it's only used if the time ran out and there are still unsent requests.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

sorry, to reiterate on your comment, perhaps your suggestion is: if time has run out, we do client.poll(0) to try to send and process the request one last time. if the time hasn't run out and there are still request to be sent, we continue to poll until all requests are sent and timer runs out. Is this what you meant?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Well, if all requests are sent, I wouldn't timeout. But otherwise, yes.

*/
@Override
public PollResult pollOnClose() {
// TODO: move the logic to poll to handle signal close
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Yes, using the normal poll loop sounds like a good idea. We should still probably sendUnsentRequests once when the timeout has passed.

@philipnee philipnee force-pushed the refactor-closing-consumer branch 2 times, most recently from 739bd4f to afb605d Compare December 11, 2023 21:29
Copy link
Copy Markdown
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

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

Approved from my side with minor comments, but needs to be rebased, approved by Kirk and pass CI.

closeQuietly(fetchBuffer, "Failed to close the fetch buffer", firstException);
closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed shutting down network thread", firstException);
closeTimer.update();
// Ensure all async commit callbacks are invoked
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

misplaced comment

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

might not even need this comment.

droppedPartitions.addAll(subscriptions.assignedPartitions());
if (!subscriptions.hasAutoAssignedPartitions() || droppedPartitions.isEmpty())
return CompletableFuture.completedFuture(null);
// TODO: Invoke rebalanceListener via KAFKA-15276
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we merge it without resolving this comment?

final Timer timer) {
// These are the optional outgoing requests at the
List<NetworkClientDelegate.PollResult> pollResults = requestManagers.stream()
requestManagers.stream()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we merge it without resolving this comment?

ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer);

// the network is polled during the last state of clean up.
networkClientDelegate.poll(time.timer(1));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Does kirk need to confirm this change before we can merge the PR?

@philipnee philipnee force-pushed the refactor-closing-consumer branch from afb605d to 0fc4a0a Compare December 12, 2023 19:57
assertTrue(applicationEventsQueue.isEmpty());
}

@Test
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

removed because they are irrelevant now

Copy link
Copy Markdown
Contributor

@kirktrue kirktrue left a comment

Choose a reason for hiding this comment

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

Thanks for the PR, @philipnee. Please make sure to file those follow-up Jiras 😉

@lucasbru
Copy link
Copy Markdown
Member

Restarting build as previous was aborted. @philipnee have you run the integration tests on this one?

@philipnee
Copy link
Copy Markdown
Contributor Author

Sorry @lucasbru - forgot to post the integration results but I did. Let me do that for the record.

@lucasbru
Copy link
Copy Markdown
Member

@lucasbru
Copy link
Copy Markdown
Member

@philipnee If tests are flaky in the PR that are also flaky on trunk, I don't think this is blocking the merge given the current state of the CI. The problem is that the builds fail completely, so there is no way to tell whether this PR introduces new problems or not. I think this may also be caused by other changes on trunk. For now, I think we have to restart CI and hope for the situation to improve, or debug the build failures ourselves. I looked into it briefly (checking the last few runs on trunk), and it seemed to me that a lot of failures of Gradle Executors followed the execution of TransactionsWithTieredStoreTest.

@lucasbru
Copy link
Copy Markdown
Member

@philipnee I think part of the reason why things are running particularly bad this weekend might be a change in transaction tests. I proposed a revert here. #15029 I will retrigger the CI for this PR for now. Let's see if we get the revert-PR merged, then we can rebase this PR and hopefully get stable runs again.

@philipnee
Copy link
Copy Markdown
Contributor Author

Thanks @lucasbru

@philipnee philipnee force-pushed the refactor-closing-consumer branch from d09d3be to 3dc624d Compare December 17, 2023 17:29
@philipnee philipnee force-pushed the refactor-closing-consumer branch from d22c9ef to 1786208 Compare December 19, 2023 00:36
@philipnee
Copy link
Copy Markdown
Contributor Author

hey @lucasbru - Seems like your fix fixed the long running/oom issue with the test. The testExpandingTopicSubscriptions is on the flaky side, I've observed that in other PRs.

@philipnee
Copy link
Copy Markdown
Contributor Author

Hey @lucasbru - I think your patch fixed the long running/oom issue with the async consumer test. The tests finished within 3.5hr in this commit: 1786208
However, there are still quite a few flaky tests in the PlaintextConsumerText. Namely..
testExpandingTopicSubscriptions
testShrinkingTopicSubscriptions
testFetchOutOfRangeOffsetResetConfigLatest

They are observed sparsely in other builds I've seen. So I'm disabling them in the latest commit: 11a3ae6

List<ConsumerPartitionAssignor> assignors,
String groupId,
String clientId) {
return new AsyncKafkaConsumer<>(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

@philipnee I removed this on purpose from the test file to use the normal constructor and not just test a mock construction of the class. I'm sure there could have been a better way than to reintroduce the constructor that I remove. However, since we are otherwise not converging with this PR, I am going to merge this. Please consider following up with a PR that removes this constructor again

@lucasbru lucasbru merged commit 5e37ec8 into apache:trunk Dec 19, 2023
lucasbru pushed a commit to lucasbru/kafka that referenced this pull request Dec 19, 2023
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
lucasbru pushed a commit that referenced this pull request Dec 20, 2023
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
gaurav-narula pushed a commit to gaurav-narula/kafka that referenced this pull request Jan 24, 2024
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.

This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548

When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.

Here I outline the changes in detail

AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)

Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

consumer ctr Consumer Threading Refactor (KIP-848)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants