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

KAFKA-17439: Make polling for new records an explicit action/event in the new consumer #17035

Merged
merged 27 commits into from
Oct 28, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
e984638
KAFKA-17439: Make polling for new records an explicit action/event in…
kirktrue Aug 28, 2024
b6af23b
Merge remote-tracking branch 'origin/trunk' into KAFKA-17439-poll-exp…
kirktrue Sep 11, 2024
335c249
Minor tweaks to FetchEvent documentation.
kirktrue Sep 11, 2024
2abd6a4
Update FetchRequestManager.java
kirktrue Sep 11, 2024
45bc8c0
Added unit tests to exercise poll, request-then-poll, and duplicate r…
kirktrue Sep 13, 2024
7265404
Updated FetchEvent to CreateFetchRequestsEvent and catching errors fr…
kirktrue Sep 13, 2024
5449549
Fixed spacing issue that checkstyle wasn't happy with
kirktrue Sep 13, 2024
f7a5940
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Sep 26, 2024
7acf732
Update KafkaConsumerTest.java
kirktrue Sep 27, 2024
7cda968
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Oct 1, 2024
de2c961
Update AsyncKafkaConsumer so that prefetch requests don't block
kirktrue Oct 1, 2024
a0beb14
Including fix for test that uses add() instead of addAndGet()
kirktrue Oct 1, 2024
af96038
Added requestGenerated back that somehow was removed
kirktrue Oct 1, 2024
09e951e
No longer need to wait for FETCH RPC in test since fetches don't happ…
kirktrue Oct 1, 2024
fbd147c
Testing that Future.get() throws a specific exception type
kirktrue Oct 1, 2024
4c7c1bf
Fixed spotless complaints
kirktrue Oct 4, 2024
3aed5e7
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Oct 7, 2024
b097ebe
Update AsyncKafkaConsumer.java
kirktrue Oct 7, 2024
62ccaa6
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Oct 17, 2024
01b5d7a
Reducing diff noise
kirktrue Oct 17, 2024
aae7e97
Updated test to check that CreateFetchRequests event uses addAndGet
kirktrue Oct 18, 2024
b942a46
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Oct 25, 2024
63abc91
Updates to suppress exceptions for pre-fetch and handle pollOnClose()
kirktrue Oct 25, 2024
a98e8b0
Updates to let pre-fetch be asynchronous
kirktrue Oct 25, 2024
d6e2241
Tweak to comment
kirktrue Oct 25, 2024
1e832e0
Merge branch 'trunk' into KAFKA-17439-poll-explicitly
kirktrue Oct 28, 2024
4071849
Updated comments for sendPrefetches to correctly reflect implementation
kirktrue Oct 28, 2024
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 @@ -53,6 +53,7 @@
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.FetchEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.PollEvent;
import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent;
Expand Down Expand Up @@ -696,6 +697,8 @@ public ConsumerRecords<K, V> poll(final Duration timeout) {
updateAssignmentMetadataIfNeeded(timer);
final Fetch<K, V> fetch = pollForFetches(timer);
if (!fetch.isEmpty()) {
sendFetches(timer);
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
Copy link
Member

Choose a reason for hiding this comment

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

at this point we may already have records in hand to return (consumed position updated), so we should be very careful to not throw any error here. But this sendFetches could throw interrupted because of the addAndGet right?

Shouldn't we just do a best effort to pipeline the next requests using add instead of addAndGet? It would achieve what we want, removing the risk of errors, and it would actually align better with what the classic does on this sendFetches + transmitSends:

* Poll for network IO in best-effort only trying to transmit the ready-to-send request
* Do not check any pending requests or metadata errors so that no exception should ever
* be thrown, also no wakeups be triggered and no interrupted exception either.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Done.


if (fetch.records().isEmpty()) {
log.trace("Returning empty records from `poll()` "
+ "since the consumer's position has advanced for at least one topic partition");
Expand Down Expand Up @@ -1528,6 +1531,9 @@ private Fetch<K, V> pollForFetches(Timer timer) {
return fetch;
}

// send any new fetches (won't resend pending fetches)
sendFetches(timer);
Copy link
Member

Choose a reason for hiding this comment

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

The actual poll now happens in here (addAndGet that will complete when the background has had one run, called fetchMgr.poll), so should the log line on ln 1538 "Polling for fetches with timeout..." be right before this?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

We're not polling for the incoming responses in sendFetches(), just enqueuing the outgoing requests. This mimics the ClassicKafkaConsumer in that the requests are enqueued in its sendFetches() but then toward the bottom of pollForFetches() client.poll() is invoked to wait for the results of the fetch requests.

Copy link
Member

Choose a reason for hiding this comment

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

well the sendFetches blocks until the CreateFetchRequestsEvent completes, and that only happens on fetchMgr.poll


So when the sendFetches completes we did poll the manager right? (and depending on time, maybe we did poll the client.poll too, which happens in the background right after polling all managers). That's why the log for "Polling for fetches" made sense to me before the sendFetches, but am I missing another poll happening after the log line maybe? (where it is now)

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

The two ConsumerDelegate implementations work differently:

  • AsyncKafkaConsumer: FetchRequestManager.poll() will complete the event's Future on the background thread before it exits, i.e. before the thread starts the network I/O. Completing the Future starts the application thread racing toward logging that message and the background thread racing toward starting network I/O. I'll admit—I haven't dug through the code to surmise the relative costs of each thread's work before either cross their respective finish lines to win.
  • ClassicKafkaConsumer: Fetcher.sendFetchesInternal() calls ConsumerNetworkClient.send() to enqueue the request, but then it calls NetworkClient.wakeup(). Since the same ConsumerNetworkClient instance used by the consumer is also used by AbstractCoordinator.HeartbeatThread, it's technically possible that the heartbeat thread's run() method could start network I/O when it calls NetworkClient.pollNoWakeup(). Granted, that's a race that the application thread is much more likely to win given that the heartbeat thread runs much less frequently.

Here are some points to consider:

  • The definition of the term "poll" as used in the log is open to interpretation. The term "poll" is everywhere, making its meaning ambiguous at any given point of use 😢
  • I agree there is a race condition (for both consumers, but more likely for the new consumer) that could result the the log message being emitted after the network I/O has commenced
  • For this to pose a problem to users, there needs to be other log entries that we're racing with, right?. We're trying to avoid the condition where the user is confused/mislead because the entries in the log are emitted in non-deterministic ordering.
  • The log line in question is only output at level TRACE, which I assume is very rare for users to enable.

Given the above, I'm of the opinion that it's an exercise in hair splitting to alter the logging. However, I could also just change it which would have been way less effort than researching, thinking, and composing this response 🤣

If we leave the log line as it is, what would the effect be for the user?

Copy link
Member

Choose a reason for hiding this comment

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

I surely didn't intend for you to put up that long response he he, sorry. It's not about the log line per-se, it's about the alignment on where the poll happens. The classic consumer logs "Polling for records", then calls client.poll. vs us here we do sendFetches (which triggers the client.poll async in the background thread because it blocks until we poll the fetch manager), then log "Polling for fetches...".

That's the diff I saw and just wanted to understand/align on where the poll happens: once we trigger sendFetches (blocking), the client.poll will happen in the background anytime, not controlled by the app thread. Agreed? If so I'm ok with leaving the log unchanged, understanding it could come out after the client.poll happened.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

That's the diff I saw and just wanted to understand/align on where the poll happens: once we trigger sendFetches (blocking), the client.poll will happen in the background anytime, not controlled by the app thread. Agreed?

Agreed—the background thread is going to move from calling each of the RequestManager’s poll() method to NetworkClient.poll() method without the intervention of the application thread.

If so I'm ok with leaving the log unchanged, understanding it could come out after the client.poll happened.

Thanks!


// We do not want to be stuck blocking in poll if we are missing some positions
// since the offset lookup may be backing off after a failure

Expand Down Expand Up @@ -1708,6 +1714,10 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd
offsetAndMetadata.leaderEpoch().ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(topicPartition, epoch));
}

private void sendFetches(Timer timer) {
applicationEventHandler.addAndGet(new FetchEvent(calculateDeadlineMs(timer)));
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
public boolean updateAssignmentMetadataIfNeeded(Timer timer) {
maybeThrowFencedInstanceException();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.function.BiConsumer;
import java.util.function.Predicate;
import java.util.stream.Collectors;
Expand All @@ -41,6 +42,7 @@
public class FetchRequestManager extends AbstractFetch implements RequestManager {

private final NetworkClientDelegate networkClientDelegate;
private CompletableFuture<Void> pendingFetchRequestFuture;

FetchRequestManager(final LogContext logContext,
final Time time,
Expand All @@ -65,16 +67,59 @@ protected void maybeThrowAuthFailure(Node node) {
networkClientDelegate.maybeThrowAuthFailure(node);
}

/**
* Request that a fetch request be issued to the cluster to pull down the next batch of records.
*
* <p/>
*
* The returned {@link CompletableFuture} is {@link CompletableFuture#complete(Object) completed} when the
* fetch request(s) have been created and enqueued into the network client's outgoing send buffer.
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
* It is <em>not completed</em> when the network client has received the data.
*
* @return Future for which the caller can wait to ensure that the requests have been enqueued
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
*/
public CompletableFuture<Void> requestFetch() {
CompletableFuture<Void> future = new CompletableFuture<>();

if (pendingFetchRequestFuture != null) {
// In this case, we have an outstanding fetch request, so chain the newly created future to be
// invoked when the outstanding fetch request is completed.
pendingFetchRequestFuture.whenComplete((value, exception) -> {
if (exception != null) {
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
future.completeExceptionally(exception);
} else {
future.complete(value);
}
});
} else {
pendingFetchRequestFuture = future;
}

return future;
}

/**
* {@inheritDoc}
*/
@Override
public PollResult poll(long currentTimeMs) {
return pollInternal(
if (pendingFetchRequestFuture == null) {
// If no explicit request for fetching has been issued, just short-circuit.
return PollResult.EMPTY;
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
}

try {
return pollInternal(
prepareFetchRequests(),
this::handleFetchSuccess,
this::handleFetchFailure
);
);
} finally {
// Completing the future here means that the caller knows that the fetch request logic has been
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
// performed. See FetchEvent for more detail.
pendingFetchRequestFuture.complete(null);
pendingFetchRequestFuture = null;
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ public enum Type {
COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, ALL_TOPICS_METADATA, SUBSCRIPTION_CHANGE,
UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED,
COMMIT_ON_CLOSE,
COMMIT_ON_CLOSE, FETCH,
SHARE_FETCH, SHARE_ACKNOWLEDGE_ASYNC, SHARE_ACKNOWLEDGE_SYNC,
SHARE_SUBSCRIPTION_CHANGE, SHARE_UNSUBSCRIBE,
SHARE_ACKNOWLEDGE_ON_CLOSE
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,10 @@ public void process(ApplicationEvent event) {
process((CommitOnCloseEvent) event);
return;

case FETCH:
process((FetchEvent) event);
return;

case SHARE_FETCH:
process((ShareFetchEvent) event);
return;
Expand Down Expand Up @@ -161,6 +165,11 @@ private void process(final PollEvent event) {
}
}

private void process(final FetchEvent event) {
CompletableFuture<Void> future = requestManagers.fetchRequestManager.requestFetch();
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
future.whenComplete(complete(event.future()));
}

private void process(final AsyncCommitEvent event) {
if (!requestManagers.commitRequestManager.isPresent()) {
return;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals.events;

import org.apache.kafka.clients.consumer.internals.FetchRequestManager;

/**
* {@code FetchEvent} is sent from the consumer to signal that we want to issue a fetch request for the partitions
* to which the consumer is currently subscribed.
*
* <p/>
*
* <em>Note</em>: this event is completed when the {@link FetchRequestManager} has finished performing the
* fetch request process. It does not mean that the requests are complete. It could be the case that no fetch
* requests were created. Also of note is that if any fetch requests were created.
kirktrue marked this conversation as resolved.
Show resolved Hide resolved
*/
public class FetchEvent extends CompletableApplicationEvent<Void> {

public FetchEvent(final long deadlineMs) {
super(Type.FETCH, deadlineMs);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.FetchEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.PollEvent;
import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent;
Expand Down Expand Up @@ -1858,6 +1859,7 @@ public void testEnsurePollEventSentOnConsumerPoll() {
consumer.subscribe(singletonList("topic1"));
consumer.poll(Duration.ofMillis(100));
verify(applicationEventHandler).add(any(PollEvent.class));
verify(applicationEventHandler).addAndGet(any(FetchEvent.class));
}

private Properties requiredConsumerConfigAndGroupId(final String groupId) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3660,6 +3660,7 @@ private Fetch<K, V> collectFetch() {
}

private int sendFetches() {
requestFetch();
NetworkClientDelegate.PollResult pollResult = poll(time.milliseconds());
networkClientDelegate.addAll(pollResult.unsentRequests);
return pollResult.unsentRequests.size();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ public void testApplicationEventIsProcessed(ApplicationEvent e) {
private static Stream<Arguments> applicationEvents() {
return Stream.of(
Arguments.of(new PollEvent(100)),
Arguments.of(new FetchEvent(calculateDeadlineMs(12345, 100))),
Arguments.of(new AsyncCommitEvent(new HashMap<>())),
Arguments.of(new SyncCommitEvent(new HashMap<>(), 500)),
Arguments.of(new ResetPositionsEvent(500)),
Expand Down