Skip to content

KAFKA-14274 [6, 7]: Introduction of fetch request manager#14406

Merged
junrao merged 77 commits intoapache:trunkfrom
kirktrue:KAFKA-14274-3-introduce-fetch-request-manager-take-2
Oct 24, 2023
Merged

KAFKA-14274 [6, 7]: Introduction of fetch request manager#14406
junrao merged 77 commits intoapache:trunkfrom
kirktrue:KAFKA-14274-3-introduce-fetch-request-manager-take-2

Conversation

@kirktrue
Copy link
Contributor

Changes:

  1. Introduces FetchRequestManager that implements the RequestManager API for fetching messages from brokers. Unlike Fetcher, record decompression and deserialization is performed on the application thread inside CompletedFetch.
  2. Restructured the code so that objects owned by the background thread are not instantiated until the background thread runs (via Supplier) to ensure that there are no references available to the application thread.
  3. Ensuring resources are properly using Closeable and using IdempotentCloser to ensure they're only closed once.
  4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in the way the objects were built up for tests.

Changes:

1. Introduces FetchRequestManager that implements the RequestManager
   API for fetching messages from brokers. Unlike Fetcher, record
   decompression and deserialization is performed on the application
   thread inside CompletedFetch.
2. Restructured the code so that objects owned by the background thread
   are not instantiated until the background thread runs (via Supplier)
   to ensure that there are no references available to the
   application thread.
3. Ensuring resources are properly using Closeable and using
   IdempotentCloser to ensure they're only closed once.
4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in
   the way the objects were built up for tests.
@kirktrue
Copy link
Contributor Author

@philipnee Can you add the ctr label, please 🥺

@philipnee philipnee added the ctr Consumer Threading Refactor (KIP-848) label Sep 20, 2023
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kirktrue : Thanks for the PR. Made a pass of non-testing files. Left a few comments.

Changes:

1. Introduces FetchRequestManager that implements the RequestManager
   API for fetching messages from brokers. Unlike Fetcher, record
   decompression and deserialization is performed on the application
   thread inside CompletedFetch.
2. Restructured the code so that objects owned by the background thread
   are not instantiated until the background thread runs (via Supplier)
   to ensure that there are no references available to the
   application thread.
3. Ensuring resources are properly using Closeable and using
   IdempotentCloser to ensure they're only closed once.
4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in
   the way the objects were built up for tests.
…f github.com:kirktrue/kafka into KAFKA-14274-3-introduce-fetch-request-manager-take-2
Changes:

1. Introduced a new generic queue processing class that both the ApplicationEventProcessor and the BackgroundEventProcessor use.
2. Moved event processing to each iteration of the internal loop inside PrototypeAsyncConsumer.poll, instead of just at the start.
3. Removed the NOOP event types.
4. Introduced the CompletableEvent interface which will be used by the BackgroundEvent hierarchy soon
5. Provided missing documentation around the queue processors.
6. No longer returning a flag from the event processor method as it was not used anywhere.
Refactored the close method so that it's much cleaner. Also fixed a confusing doc change to refreshCommittedOffsetsIfNeeded().
FetchEvent now provides the Future to the FetchRequestManager which will directly respond with the results of the fetch. This allows the application thread to properly block in pollForFetches() as it waits for a response. A separate thread-safe blocking queue collects the results from the Future, as opposed to collecting the results from the future in the fetch buffer, as that is not thread safe and should not be used across threads.
1. Renamed DefaultEventHandler to ApplicationEventHandler
2. Renamed ErrorEventHandler to BackgroundEventHandler
3. Deleted EventHandler interface
4. Moved ApplicationEventHandler and BackgroundEventHandler (and their unit tests) into the "events" sub-package for consistency
5. Removed use of mocks in DefaultBackgroundThreadTest as they caused issues with various methods returning null
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kirktrue : Thanks for the updated PR. Added a few more comments.


if (!fetch.isEmpty()) {
// Notify the network thread to wake up and start the next round of fetching.
applicationEventHandler.wakeup();
Copy link
Contributor

Choose a reason for hiding this comment

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

This doesn't seem quite right. It's possible for a fetchRequest to return no data. In that case, if we don't wake up the the network thread, it may not be able to send the next fetch request for a long time.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The enqueuing of other events will also call wakeupNetworkThread, so it shouldn't be a problem. But I went ahead and changed it as requested.

/**
* Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the event from the queue.
*/
public void wakeup() {
Copy link
Contributor

Choose a reason for hiding this comment

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

wakeup => wakeupNetworkThread ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

Map<TopicPartition, Long> offsetResetTimestamps;

try {
offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp();
Copy link
Contributor

Choose a reason for hiding this comment

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

auto.offset.reset has 3 options: latest, earliest, none. It's possible for a user to choose none, which means "throw exception to the consumer if no previous offset is found for the consumer's group". But in that case, offsetFetcherUtils.getOffsetResetTimestamp just ignores that partition and won't reset it forever. We need to throw an exception so that the user knows.

It's a bit of weird configuration since it means a consumer can't really consume for the very first time with this option. However, I tested it out with the existing consumer and it does throw an exception.

bin/kafka-console-consumer.sh --consumer-property auto.offset.reset=none --bootstrap-server localhost:9092 --topic test
[2023-10-20 15:30:07,088] ERROR Error processing message, terminating consumer process:  (kafka.tools.ConsoleConsumer$)
org.apache.kafka.clients.consumer.NoOffsetForPartitionException: Undefined offset with no reset policy for partitions: [test-0]
	at org.apache.kafka.clients.consumer.internals.SubscriptionState.resetInitializingPositions(SubscriptionState.java:711)
	at org.apache.kafka.clients.consumer.KafkaConsumer.updateFetchPositions(KafkaConsumer.java:2459)
	at org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1243)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1198)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1178)
	at kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:473)
	at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:103)
	at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:77)
	at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:54)
	at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)

@kirktrue
Copy link
Contributor Author

FYI: I rebased against trunk to remove the conflicts.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kirktrue : Thanks for the updated PR. Just a couple of minor comments.

timer.update(pollTimer.currentTimeMs());
}

return collectFetch();
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we just return empty here instead of calling collectFetch() again since the caller is in a loop and can call this method to collect fetch again?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, we could update pollForFetches() to simply return Fetch.empty() at the end.

However, there's code that intentionally blocks (for a little bit) waiting for fetch data to arrive in the buffer:

try {
    fetchBuffer.awaitNotEmpty(pollTimer);
} catch (InterruptException e) {
    log.trace("Timeout during fetch", e);
} finally {
    timer.update(pollTimer.currentTimeMs());
}

return collectFetch();

Also, each loop through poll() executes updateAssignmentMetadataIfNeeded() before checking the fetch buffer for any data. That method does a lot of work (network I/O) that we'd ideally skip if we already have the data.

Perhaps we could update awaitNotEmpty() with a return flag so that the tail end of pollForFetches() can look something like this:

try {
    if (fetchBuffer.awaitNotEmpty(pollTimer))
        return collectFetch();
} catch (InterruptException e) {
    log.trace("Timeout during fetch", e);
} finally {
    timer.update(pollTimer.currentTimeMs());
}

return Fetch.empty();

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for the explanation, Kirk. We can just leave the code as it is then.

// positions, so a consumer with manually assigned partitions can avoid a coordinator
// dependence by always ensuring that assigned partitions have an initial position.
if (isCommittedOffsetsManagementEnabled() && !refreshCommittedOffsetsIfNeeded(timer))
return false;
Copy link
Contributor

Choose a reason for hiding this comment

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

Would it be clearer to rename refreshCommittedOffsetsIfNeeded to initWithCommittedOffsetsIfNeeded?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The refreshCommittedOffsetsIfNeeded() name is derived from existing code in KafkaConsumer that calls out to ConsumerCoordinator.refreshCommittedOffsetsIfNeeded(). If we change the method name in PrototypeAsyncConsumer, should we change ConsumerCoordinator's corresponding method name too?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, in both cases, the method sets offsets for partitions in initializing state.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated and a bit of cleanup.

…Needed

Also removed the check for null from ConsumerUtils.refreshCommittedOffsets() and put it on the ConsumerCoordinator.initWithCommittedOffsetsIfNeeded to check for null offsets map before calling.
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kirktrue : Thanks for the latest update. The PR LGTM now. Are the 30 test failures related?

@kirktrue
Copy link
Contributor Author

kirktrue commented Oct 23, 2023

Are the 30 test failures related?

No, I do not believe they are.

There are 8 "new" failures, of which two are the same test with different parameters. Here are the tests and any Jiras for flakiness:

  • kafka.server.DescribeClusterRequestTest.testDescribeClusterRequestExcludingClusterAuthorizedOperations: KAFKA-15419
  • kafka.server.DynamicBrokerReconfigurationTest.testLogCleanerConfig: KAFKA-7966, though the Jira is resolved 🤷‍♂️
  • o.a.k.common.network.SslVersionsTransportLayerTest.testTlsDefaults: KAFKA-9714, marked as "Critical" but open for 3 1/2 years
  • o.a.k.connect.integration.ConnectorRestartApiIntegrationTest.testMultiWorkerRestartOnlyConnector: KAFKA-15675, just filed (by me) with some pointers to a recent flakiness rate of 9%
  • o.a.k.streams.integration.StandbyTaskEOSMultiRebalanceIntegrationTest.shouldHonorEOSWhenUsingCachingAndStandbyReplicas: no Jira. Only around 2% flaky
  • o.a.k.tiered.storage.integration.TransactionsWithTieredStoreTest.testSendOffsetsWithGroupId: KAFKA-8003, though the Jira is filed against a different test, but test failures in testSendOffsetsWithGroupId are mentioned as being related
  • o.a.k.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated: KAFKA-8115, which is another old, 4.5 year old "Critical" issue

There are 22 "existing" failures, which are all related to the verifyNoUnexpectedThreads check. kafka.server.DynamicBrokerReconfigurationTest is reporting that there are threads when it attempts to tear down the test harness. The remaining 21 failures all report the same unexpected threads in their check during test harness setup. There are other recent, unrelated pull requests that have experienced similar issues. In the other cases, I haven't seen that it's the DynamicBrokerReconfigurationTest that is the cause, though.

@kirktrue kirktrue closed this Oct 23, 2023
@kirktrue
Copy link
Contributor Author

Closing and reopening to trigger another Jenkins test run.

@kirktrue kirktrue reopened this Oct 23, 2023
@kirktrue
Copy link
Contributor Author

Ugh. One of the builds failed with:

ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from https://github.com/apache/kafka.git
	at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:1003)
	at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1245)
	at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1309)
	at org.jenkinsci.plugins.workflow.steps.scm.SCMStep.checkout(SCMStep.java:129)
	at org.jenkinsci.plugins.workflow.steps.scm.SCMStep$StepExecutionImpl.run(SCMStep.java:97)
	at org.jenkinsci.plugins.workflow.steps.scm.SCMStep$StepExecutionImpl.run(SCMStep.java:84)
	at org.jenkinsci.plugins.workflow.steps.SynchronousNonBlockingStepExecution.lambda$start$0(SynchronousNonBlockingStepExecution.java:47)
	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at java.base/java.lang.Thread.run(Thread.java:829)

No integration test failures due to threads, though 😏

@kirktrue
Copy link
Contributor Author

Closing and reopening to restart test.

@kirktrue kirktrue closed this Oct 24, 2023
@kirktrue kirktrue reopened this Oct 24, 2023
@kirktrue
Copy link
Contributor Author

Closing and reopening to restart test. Jenkins doesn't seem happy lately.

@kirktrue kirktrue closed this Oct 24, 2023
@kirktrue kirktrue reopened this Oct 24, 2023
@Override
public List<PartitionInfo> partitionsFor(String topic) {
throw new KafkaException("method not implemented");
return partitionsFor(topic, Duration.ofMillis(defaultApiTimeoutMs));
Copy link
Member

Choose a reason for hiding this comment

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

I had missed a detail here. We're only implementing this with the default timeout, but not the overloaded one below, that has the timeout provided by the user. Is there a reason or is it just that we missed the latter one? (same for the listTopics)

Copy link
Member

Choose a reason for hiding this comment

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

(I guess it will be all part of the integration with the metadata calls right?)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good call out. I don't know that the necessary topic plumbing code was written at that time. Would you mind filing a bug to resolve?

@junrao
Copy link
Contributor

junrao commented Oct 24, 2023

@kirktrue : The build for JDK 21 and Scala 2.13 failed this time. Did it succeed before?

@kirktrue
Copy link
Contributor Author

@junrao—yes. Here's a brief history for JDK 21, starting with the most recent build (build 74):

These same intra-Jenkins communication, git cloning, unexpected threads, and flaky tests affect all of the JDKs at random times 😢

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kirktrue : Thanks for explaining the test failures. The PR LGTM.

@junrao junrao merged commit 2b233bf into apache:trunk Oct 24, 2023
@kirktrue kirktrue deleted the KAFKA-14274-3-introduce-fetch-request-manager-take-2 branch October 25, 2023 00:14
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
Changes:

1. Introduces FetchRequestManager that implements the RequestManager
   API for fetching messages from brokers. Unlike Fetcher, record
   decompression and deserialization is performed on the application
   thread inside CompletedFetch.
2. Restructured the code so that objects owned by the background thread
   are not instantiated until the background thread runs (via Supplier)
   to ensure that there are no references available to the
   application thread.
3. Ensuring resources are properly using Closeable and using
   IdempotentCloser to ensure they're only closed once.
4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in
   the way the objects were built up for tests.

Reviewers: Philip Nee <pnee@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Jun Rao<junrao@gmail.com>
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)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants