Skip to content

KAFKA-14247: Consumer background thread base implementation#12672

Merged
vvcephei merged 45 commits intoapache:trunkfrom
philipnee:consumer-refactor-background-thread
Oct 20, 2022
Merged

KAFKA-14247: Consumer background thread base implementation#12672
vvcephei merged 45 commits intoapache:trunkfrom
philipnee:consumer-refactor-background-thread

Conversation

@philipnee
Copy link
Copy Markdown
Contributor

@philipnee philipnee commented Sep 21, 2022

1-pager: https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor

EventHandler runs a task to ingest the ApplicationEvents. The task runs on a background thread, and consumes events from the ApplicationEventQueue, and produces events to the BackgroundEventQueue.

The PR consist of the basic skeleton of the background thread with

  • network client
  • basic demonstration of the consumption logic

Continuation of #12663

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

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.

Are ConsumerRequestEvent/ConsumerResponseEvent and ApplicationEvent/BackgroundEvent the same thing?

@philipnee philipnee force-pushed the consumer-refactor-background-thread branch from 12ce01d to 7c7c072 Compare September 27, 2022 23:00
@philipnee philipnee changed the title KAFKA-14252: Consumer refactor background thread KAFKA-14252: Consumer background thread base implementation Sep 28, 2022
@philipnee philipnee force-pushed the consumer-refactor-background-thread branch from 0ae5431 to af2727e Compare September 30, 2022 18:55
@philipnee philipnee marked this pull request as ready for review September 30, 2022 18:55
@philipnee
Copy link
Copy Markdown
Contributor Author

@mhowlett @kirktrue Mind reviewing this PR? This is actually stack on top of : #12663 So some code might be outdated

@philipnee philipnee force-pushed the consumer-refactor-background-thread branch from 88e3d39 to 422eed4 Compare October 3, 2022 16:44
@philipnee philipnee force-pushed the consumer-refactor-background-thread branch from 28a5814 to 6cfa912 Compare October 4, 2022 18:53
Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

Thanks @philipnee for the PR. I made a first pass and left some comments.

log.debug("{} started", getClass());
while (running) {
runOnce();
time.sleep(retryBackoffMs);
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.

Instead of setting a sleep here, I think it's better to have a waitOnEmptyInputQueue function with a condition. Where the caller thread would notify when enqueuing a new item.

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.

@guozhangwang - I thought we wanted to maintain an active loop because we want to keep sending fetches and rebalance requests despite an empty ApplicationQueue.

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.

Yeah I think so -- please see my other comment below: #12672 (comment). But I still feel that instead of having a sleep for each iteration, we should just consider poll on the network client with a timeout, i.e. supposing our loop would look like:

  1. try to get some event from the queue, if yes, then handle it and potentially send the corresponding request;
  2. if the queue is empty, then try to send a fetch request.
  3. poll the network client in order to receive any responses ready-to-read on the socket.

Then our logic could be: if there's no new actions taken at step 1/2), i.e. we do not have any new items from the queue, and we do not yet need to send any new fetch/rebalance-related requests, then at step 3) we poll for a bit longer time until being notified by the caller that there's new items in the queue; otherwise, at step 3) we just poll without timeout and then immediately move on to the next iteration.

applicationEventQueue,
backgroundEventQueue);
assertTrue(eventHandler.add(new NoopApplicationEvent("hello-world")));
while (eventHandler.isEmpty()) { }
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 general comment: I think it worth adding a poll(timeout) in the EventHandler interface since 1) for testing, we do not need the while loop any more, and 2) in production code, for the caller thread scenarios I think there are cases where a poll(timeout) would be useful as well.


public enum EventType {
COMMIT,
NOOP,
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.

Ditto, I think we should keep the testing-related element out of the production code eventually. If we are going to remove it later then that's fine.

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.

Thanks, I think the NOOP event will be moved to the test package in the future PR; I left it here to elucidate the purpose of a few methods in the background thread.

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.

Thanks for clarifying this, makes sense.

/**
* Noop event. Intentionally left it here for demonstration purpose.
*/
public class NoopBackgroundEvent extends BackgroundEvent {
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.

Ditto here, I think we should move it to test package, not main package.

@philipnee philipnee force-pushed the consumer-refactor-background-thread branch from d17fe3b to 8f2c870 Compare October 11, 2022 18:57
@philipnee
Copy link
Copy Markdown
Contributor Author

Hey @guozhangwang , much thanks for the detail reviews, I tried to address some of the comments, please review them. In particular:

  1. poll timeout logic
  2. InterruptException handling (A side question here, when the BT got interrupted, shouldn't we terminate instead of swallowing the exception?)
  3. Added a couple of tests to test the network client calls in a single poll.
  4. I also removed the interface, originally I thought it would be helpful to write stubbed tests.

I left the NOOP event there, but I agree we should move it once we've got an actual event implemented, which should happen soon.

@philipnee
Copy link
Copy Markdown
Contributor Author

cc John - @vvcephei

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.

I did another review of the code. I haven't gotten as far as the tests yet.

log.error("The background thread failed due to unexpected error",
t);
if (t instanceof RuntimeException)
this.exception.set(Optional.of((RuntimeException) t));
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.

Do we want to use KafkaException instead of RuntimeException?

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 could, there are other places in the code that uses RTE so trying to be consistent with the existing code.

}

public void close() {
this.wakeup();
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.

Does the order of these two lines (wakeup and running = false) want to be swapped?

close is called by the polling thread, right? If wakeup is called, it will throw an exception, but then the background thread might potentially loop back around to runOnce before the call to set running to false by the polling thread.

Sensor fetcherThrottleTimeSensor) {
this.applicationEventQueue = applicationEventQueue;
this.backgroundEventQueue = backgroundEventQueue;
ConsumerMetadata metadata = bootstrapMetadata(logContext,
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.

The call to bootstrapMetadata is blocking, right? Is that what we want? If so, can you add some comments (in the code) as to that behavior?

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.

It should be fine for this call to block, I think because all it does is create a Node object with id = -1 and an IP address. The actual metadata refresh happens during the background thread loop, not during this function call, since there are no network response deps to this call, so it shouldn't be blocking for too long.

@vvcephei
Copy link
Copy Markdown
Contributor

Hey, @philipnee, @guozhangwang isn't available for reviews right now, so I'll pick it up.

I reviewed it, and it all looks good to me. I had a large number of code convention corrections, so I just went ahead and applied the changes, rather than peppering you with nitpicks. I hope that's ok.

Generally:

  • please use final when it can be used
  • indent 4 spaces instead of 8
  • when a method call has too many args to fit within the line length limit, please break it up into one-arg-per-line, rather than just breaking the args at an arbitrary point.

I know that's all arbitrary, and that there are merits to doing things other ways; I'm just trying to keep these files in line with the rest of AK.

Thanks! Once the tests pass, I'll go ahead and merge.

@vvcephei
Copy link
Copy Markdown
Contributor

Unrelated test failure: [Build / JDK 17 and Scala 2.13 / org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-12672/41/testReport/junit/org.apache.kafka.controller/QuorumControllerTest/Build___JDK_17_and_Scala_2_13___testDelayedConfigurationOperations__/)

@philipnee
Copy link
Copy Markdown
Contributor Author

Thanks @vvcephei Will be better at the basic styling requirements in the future...

@vvcephei vvcephei merged commit 0a045d4 into apache:trunk Oct 20, 2022
guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…2672)

Adds skeleton of the background thread.

1-pager: https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor
Continuation of apache#12663

Reviewers: Guozhang Wang <guozhang@apache.org>, Kirk True <kirk@mustardgrain.com>, John Roesler <vvcephei@apache.org>
@philipnee philipnee added KIP-848 The Next Generation of the Consumer Rebalance Protocol ctr Consumer Threading Refactor (KIP-848) labels May 23, 2023
rutvijmehta-harness pushed a commit to rutvijmehta-harness/kafka that referenced this pull request Feb 9, 2024
…2672)

Adds skeleton of the background thread.

1-pager: https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor
Continuation of apache#12663

Reviewers: Guozhang Wang <guozhang@apache.org>, Kirk True <kirk@mustardgrain.com>, John Roesler <vvcephei@apache.org>
rutvijmehta-harness added a commit to rutvijmehta-harness/kafka that referenced this pull request Feb 9, 2024
…2672) (#76)

Adds skeleton of the background thread.

1-pager: https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor
Continuation of apache#12663

Reviewers: Guozhang Wang <guozhang@apache.org>, Kirk True <kirk@mustardgrain.com>, John Roesler <vvcephei@apache.org>

Co-authored-by: Philip Nee <pnee@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

ctr Consumer Threading Refactor (KIP-848) KIP-848 The Next Generation of the Consumer Rebalance Protocol

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants