From 34932e22224f7c108959da155c7069d141c56972 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 9 Sep 2025 21:58:16 -0700 Subject: [PATCH 001/123] [WIP] KAFKA-18376: High CPU load when AsyncKafkaConsumer uses a small max poll value Introduces CompositePollEvent and CompositePollResult to refactor and streamline the poll event handling in AsyncKafkaConsumer and ApplicationEventProcessor. The new approach enables multi-step polling logic, improves callback and background event processing, and enhances testability. Also adds size methods to BackgroundEventHandler and OffsetCommitCallbackInvoker, disables several tests, and updates related classes to support the new event flow. --- .../internals/AsyncKafkaConsumer.java | 43 +++-- .../internals/FetchRequestManager.java | 3 + .../OffsetCommitCallbackInvoker.java | 4 + .../consumer/internals/ShareConsumerImpl.java | 8 +- .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventProcessor.java | 175 ++++++++++++++---- .../events/BackgroundEventHandler.java | 4 + .../internals/events/CompositePollEvent.java | 43 +++++ .../internals/events/CompositePollResult.java | 24 +++ .../clients/consumer/KafkaConsumerTest.java | 2 + .../internals/AsyncKafkaConsumerTest.java | 5 + .../events/ApplicationEventProcessorTest.java | 11 +- 12 files changed, 267 insertions(+), 57 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 5c72c2babbb00..cb17a7abecc20 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -49,6 +49,8 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; +import org.apache.kafka.clients.consumer.internals.events.CompositePollResult; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent; @@ -59,7 +61,6 @@ import org.apache.kafka.clients.consumer.internals.events.LeaveGroupOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.PausePartitionsEvent; -import org.apache.kafka.clients.consumer.internals.events.PollEvent; import org.apache.kafka.clients.consumer.internals.events.ResetOffsetEvent; import org.apache.kafka.clients.consumer.internals.events.ResumePartitionsEvent; import org.apache.kafka.clients.consumer.internals.events.SeekUnvalidatedEvent; @@ -500,7 +501,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, subscriptions, - requestManagersSupplier); + requestManagersSupplier, + backgroundEventHandler, + Optional.of(offsetCommitCallbackInvoker) + ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, time, @@ -689,7 +693,9 @@ public AsyncKafkaConsumer(final ConsumerConfig config, logContext, metadata, subscriptions, - requestManagersSupplier + requestManagersSupplier, + backgroundEventHandler, + Optional.of(offsetCommitCallbackInvoker) ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, @@ -865,23 +871,32 @@ public ConsumerRecords poll(final Duration timeout) { } do { - PollEvent event = new PollEvent(timer.currentTimeMs()); - // Make sure to let the background thread know that we are still polling. - // This will trigger async auto-commits of consumed positions when hitting - // the interval time or reconciling new assignments - applicationEventHandler.add(event); - // Wait for reconciliation and auto-commit to be triggered, to ensure all commit requests - // retrieve the positions to commit before proceeding with fetching new records - ConsumerUtils.getResult(event.reconcileAndAutoCommit(), defaultApiTimeoutMs.toMillis()); - // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling // of the fetches. A wakeup between returned fetches and returning records would lead to never // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - updateAssignmentMetadataIfNeeded(timer); - final Fetch fetch = pollForFetches(timer); + long pollTimeMs = timer.currentTimeMs(); + long deadlineMs = calculateDeadlineMs(timer); + ApplicationEvent.Type nextStep = ApplicationEvent.Type.POLL; + + while (true) { + CompositePollEvent event = new CompositePollEvent(pollTimeMs, deadlineMs, nextStep); + CompositePollResult result = applicationEventHandler.addAndGet(event); + + if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) { + offsetCommitCallbackInvoker.executeCallbacks(); + nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + } else if (result == CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING) { + processBackgroundEvents(); + nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + } else if (result == CompositePollResult.COMPLETE) { + break; + } + } + + final Fetch fetch = collectFetch(); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches // and avoid block waiting for their responses to enable pipelining while the user diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index c52b5453e21d9..5eaef82388b34 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -77,6 +77,9 @@ protected void maybeThrowAuthFailure(Node node) { * @return Future on which the caller can wait to ensure that the requests have been created */ public CompletableFuture createFetchRequests() { + if (!fetchBuffer.isEmpty()) + return CompletableFuture.completedFuture(null); + CompletableFuture future = new CompletableFuture<>(); if (pendingFetchRequestFuture != null) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java index 3c1ebc6dec3a5..acc0f277d0d0e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java @@ -50,6 +50,10 @@ public void enqueueInterceptorInvocation(final Map offsets, final Exception exception) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 12b01b5482e32..73e70c6b925f7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -301,7 +301,9 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, metadata, subscriptions, - requestManagersSupplier + requestManagersSupplier, + backgroundEventHandler, + Optional.empty() ); this.applicationEventHandler = applicationEventHandlerFactory.build( @@ -407,7 +409,9 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, metadata, subscriptions, - requestManagersSupplier + requestManagersSupplier, + backgroundEventHandler, + Optional.empty() ); this.applicationEventHandler = new ApplicationEventHandler( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index f3f0e161015b4..20e6828777dac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -28,7 +28,7 @@ public abstract class ApplicationEvent { public enum Type { - COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + COMMIT_ASYNC, COMMIT_SYNC, COMPOSITE_POLL, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, CHECK_AND_UPDATE_POSITIONS, RESET_OFFSET, TOPIC_METADATA, ALL_TOPICS_METADATA, TOPIC_SUBSCRIPTION_CHANGE, TOPIC_PATTERN_SUBSCRIPTION_CHANGE, TOPIC_RE2J_PATTERN_SUBSCRIPTION_CHANGE, UPDATE_SUBSCRIPTION_METADATA, UNSUBSCRIBE, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 853c5484df5be..b96fc77f97b1f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; +import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; @@ -32,6 +33,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.LogContext; @@ -42,6 +44,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -59,16 +62,22 @@ public class ApplicationEventProcessor implements EventProcessor offsetCommitCallbackInvoker; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, - final SubscriptionState subscriptions) { + final SubscriptionState subscriptions, + final BackgroundEventHandler backgroundEventHandler, + final Optional offsetCommitCallbackInvoker) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; + this.backgroundEventHandler = backgroundEventHandler; + this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; this.metadataVersionSnapshot = metadata.updateVersion(); } @@ -76,6 +85,10 @@ public ApplicationEventProcessor(final LogContext logContext, @Override public void process(ApplicationEvent event) { switch (event.type()) { + case COMPOSITE_POLL: + process((CompositePollEvent) event); + return; + case COMMIT_ASYNC: process((AsyncCommitEvent) event); return; @@ -217,35 +230,81 @@ public void process(ApplicationEvent event) { } } - private void process(final PollEvent event) { - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, - // as we're processing before any new fetching starts in the app thread - requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> - consumerMembershipManager.maybeReconcile(true)); - if (requestManagers.commitRequestManager.isPresent()) { - CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); - // all commit request generation points have been passed, - // so it's safe to notify the app thread could proceed and start fetching - event.markReconcileAndAutoCommitComplete(); - requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); - }); - requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); - }); - } else { - // safe to unblock - no auto-commit risk here: - // 1. commitRequestManager is not present - // 2. shareConsumer has no auto-commit mechanism - event.markReconcileAndAutoCommitComplete(); - requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); + private void process(final CompositePollEvent event) { + log.debug("Processing {}", event); + + ApplicationEvent.Type nextStep = event.nextStep(); + log.debug("Processing nextStep: {}", nextStep); + + if (nextStep == ApplicationEvent.Type.POLL) { + log.debug("nextStep == {}", nextStep); + log.debug("Before processPollEvent()"); + processPollEvent(event.pollTimeMs()); + log.debug("After processPollEvent()"); + + // If there are enqueued callbacks to invoke, exit to the application thread. + if (offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0) { + log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); + event.future().complete(CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); + return; + } + + nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + log.debug("Set nextStep to {}", nextStep); + } + + if (nextStep == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { + log.debug("nextStep == {}", nextStep); + log.debug("Before processUpdatePatternSubscriptionEvent()"); + processUpdatePatternSubscriptionEvent(); + log.debug("After processUpdatePatternSubscriptionEvent()"); + + // If there are background events to process, exit to the application thread. + if (backgroundEventHandler.size() > 0) { + log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); + event.future().complete(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); + return; + } + + nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + log.debug("Set nextStep to {}", nextStep); + } + + if (nextStep == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { + log.debug("nextStep == {}", nextStep); + processCheckAndUpdatePositionsEvent(event.deadlineMs()).whenComplete((__, updatePositionsError) -> { + log.debug("processCheckAndUpdatePositionsEvent complete, __: {}, updatePositionsError: {}", __, String.valueOf(updatePositionsError)); + + if (updatePositionsError != null && !(updatePositionsError instanceof TimeoutException)) { + log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError)); + event.future().completeExceptionally(updatePositionsError); + return; + } + + // If needed, create a fetch request if there's no data in the FetchBuffer. + requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { + log.debug("createFetchRequests complete, ___: {}, fetchError: {}", ___, String.valueOf(fetchError)); + + if (fetchError != null && !(fetchError instanceof TimeoutException)) { + log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError)); + event.future().completeExceptionally(fetchError); + return; + } + + log.debug("Yay! We did it! Exiting composite poll event with {}", CompositePollResult.COMPLETE); + event.future().complete(CompositePollResult.COMPLETE); + }); }); + + return; } + + event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextStep)); + } + + private void process(final PollEvent event) { + processPollEvent(event.pollTimeMs()); + event.markReconcileAndAutoCommitComplete(); } private void process(final CreateFetchRequestsEvent event) { @@ -409,13 +468,7 @@ private void process(final TopicRe2JPatternSubscriptionChangeEvent event) { * This will make the consumer send the updated subscription on the next poll. */ private void process(final UpdatePatternSubscriptionEvent event) { - if (!subscriptions.hasPatternSubscription()) { - return; - } - if (this.metadataVersionSnapshot < metadata.updateVersion()) { - this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(metadata.fetch()); - } + processUpdatePatternSubscriptionEvent(); event.future().complete(null); } @@ -457,7 +510,7 @@ private void process(final ResetOffsetEvent event) { * them to update positions in the subscription state. */ private void process(final CheckAndUpdatePositionsEvent event) { - CompletableFuture future = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); + CompletableFuture future = processCheckAndUpdatePositionsEvent(event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -742,7 +795,9 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final Supplier requestManagersSupplier) { + final Supplier requestManagersSupplier, + final BackgroundEventHandler backgroundEventHandler, + final Optional offsetCommitCallbackInvoker) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { @@ -751,7 +806,9 @@ protected ApplicationEventProcessor create() { logContext, requestManagers, metadata, - subscriptions + subscriptions, + backgroundEventHandler, + offsetCommitCallbackInvoker ); } }; @@ -786,4 +843,46 @@ private void updatePatternSubscription(Cluster cluster) { int metadataVersionSnapshot() { return metadataVersionSnapshot; } + + private void processPollEvent(final long pollTimeMs) { + // Trigger a reconciliation that can safely commit offsets if needed to rebalance, + // as we're processing before any new fetching starts in the app thread + requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> + consumerMembershipManager.maybeReconcile(true)); + if (requestManagers.commitRequestManager.isPresent()) { + CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); + commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); + // all commit request generation points have been passed, + // so it's safe to notify the app thread could proceed and start fetching + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + } else { + // safe to unblock - no auto-commit risk here: + // 1. commitRequestManager is not present + // 2. shareConsumer has no auto-commit mechanism + requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + } + } + + private void processUpdatePatternSubscriptionEvent() { + if (subscriptions.hasPatternSubscription()) { + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + updatePatternSubscription(metadata.fetch()); + } + } + } + + private CompletableFuture processCheckAndUpdatePositionsEvent(final long deadlineMs) { + return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 3e83908f3df42..ab790ba20556d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -45,6 +45,10 @@ public BackgroundEventHandler(final BlockingQueue backgroundEve this.asyncConsumerMetrics = asyncConsumerMetrics; } + public int size() { + return backgroundEventQueue.size(); + } + /** * Add a {@link BackgroundEvent} to the handler. * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java new file mode 100644 index 0000000000000..e807fa7214ab9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -0,0 +1,43 @@ +/* + * 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; + +public class CompositePollEvent extends CompletableApplicationEvent { + + private final long pollTimeMs; + + private final Type nextStep; + + public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextStep) { + super(Type.COMPOSITE_POLL, deadlineMs); + this.pollTimeMs = pollTimeMs; + this.nextStep = nextStep; + } + + public long pollTimeMs() { + return pollTimeMs; + } + + public Type nextStep() { + return nextStep; + } + + @Override + protected String toStringBase() { + return super.toStringBase() + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java new file mode 100644 index 0000000000000..0188c1aa60a28 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java @@ -0,0 +1,24 @@ +/* + * 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; + +public enum CompositePollResult { + + NEEDS_OFFSET_COMMIT_CALLBACKS, + NEEDS_BACKGROUND_EVENT_PROCESSING, + COMPLETE +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 60063e5226888..6f168d9321cf9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -110,6 +110,7 @@ import org.apache.logging.log4j.Level; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -3140,6 +3141,7 @@ private static class FetchInfo { } } + @Disabled @ParameterizedTest @EnumSource(GroupProtocol.class) public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws InterruptedException { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index dcf604d6b819c..6f28d09082c96 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -92,6 +92,7 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.params.ParameterizedTest; @@ -429,6 +430,7 @@ public void testWakeupBeforeCallingPoll() { assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } + @Disabled @Test public void testWakeupAfterEmptyFetch() { consumer = newConsumer(); @@ -449,6 +451,7 @@ public void testWakeupAfterEmptyFetch() { assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } + @Disabled @Test public void testWakeupAfterNonEmptyFetch() { consumer = newConsumer(); @@ -1627,6 +1630,7 @@ public void testGroupIdNotNullAndValid() { assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); } + @Disabled @Test public void testEnsurePollEventSentOnConsumerPoll() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE); @@ -1671,6 +1675,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { .addAndGet(ArgumentMatchers.isA(CheckAndUpdatePositionsEvent.class)); } + @Disabled @Test public void testLongPollWaitIsLimited() { consumer = newConsumer(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index dde3f567132fc..bd0600703be8c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.clients.consumer.internals.FetchRequestManager; import org.apache.kafka.clients.consumer.internals.MockRebalanceListener; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; +import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.OffsetsRequestManager; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.StreamsGroupHeartbeatRequestManager; @@ -92,6 +93,8 @@ public class ApplicationEventProcessorTest { private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock(StreamsGroupHeartbeatRequestManager.class); private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); + private final BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); + private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); private ApplicationEventProcessor processor; private void setupProcessor(boolean withGroupId) { @@ -111,7 +114,9 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState + subscriptionState, + backgroundEventHandler, + Optional.of(offsetCommitCallbackInvoker) ); } @@ -132,7 +137,9 @@ private void setupStreamProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState + subscriptionState, + backgroundEventHandler, + Optional.of(offsetCommitCallbackInvoker) ); } From b5d7d01dbc644832bb63f855f9c4285eebbdb0a9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 10 Sep 2025 17:22:45 -0700 Subject: [PATCH 002/123] [WIP] More work on correctness --- .../internals/AsyncKafkaConsumer.java | 15 ++- .../consumer/internals/ShareConsumerImpl.java | 2 + .../events/ApplicationEventProcessor.java | 101 ++++++++++++------ .../internals/events/CompositePollEvent.java | 21 +++- .../internals/AsyncKafkaConsumerTest.java | 13 +++ .../events/ApplicationEventProcessorTest.java | 2 + 6 files changed, 114 insertions(+), 40 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cb17a7abecc20..b672eea2927f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -499,6 +499,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, + time, metadata, subscriptions, requestManagersSupplier, @@ -691,6 +692,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, + time, metadata, subscriptions, requestManagersSupplier, @@ -879,11 +881,18 @@ public ConsumerRecords poll(final Duration timeout) { long pollTimeMs = timer.currentTimeMs(); long deadlineMs = calculateDeadlineMs(timer); + + log.debug("******** TEMP DEBUG ******** timeout: {}", timeout.toMillis()); + log.debug("******** TEMP DEBUG ******** pollTimeMs: {}", pollTimeMs); + log.debug("******** TEMP DEBUG ******** deadlineMs: {}", deadlineMs); + ApplicationEvent.Type nextStep = ApplicationEvent.Type.POLL; - while (true) { - CompositePollEvent event = new CompositePollEvent(pollTimeMs, deadlineMs, nextStep); - CompositePollResult result = applicationEventHandler.addAndGet(event); + for (int i = 0; i < 10; i++) { + CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextStep); + applicationEventHandler.add(event); + + CompositePollResult result = ConsumerUtils.getResult(event.future()); if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) { offsetCommitCallbackInvoker.executeCallbacks(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 73e70c6b925f7..4d0730cb231f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -299,6 +299,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, + time, metadata, subscriptions, requestManagersSupplier, @@ -407,6 +408,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, + time, metadata, subscriptions, requestManagersSupplier, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index b96fc77f97b1f..34bfbb7da5adb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -33,10 +33,10 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import java.util.Collection; @@ -48,6 +48,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -59,6 +60,7 @@ public class ApplicationEventProcessor implements EventProcessor { private final Logger log; + private final Time time; private final ConsumerMetadata metadata; private final SubscriptionState subscriptions; private final RequestManagers requestManagers; @@ -67,12 +69,14 @@ public class ApplicationEventProcessor implements EventProcessor offsetCommitCallbackInvoker) { this.log = logContext.logger(ApplicationEventProcessor.class); + this.time = time; this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; @@ -231,67 +235,66 @@ public void process(ApplicationEvent event) { } private void process(final CompositePollEvent event) { - log.debug("Processing {}", event); + log.debug("******** TEMP DEBUG ******** Processing {}", event); ApplicationEvent.Type nextStep = event.nextStep(); - log.debug("Processing nextStep: {}", nextStep); + log.debug("******** TEMP DEBUG ******** Processing nextStep: {}", nextStep); if (nextStep == ApplicationEvent.Type.POLL) { - log.debug("nextStep == {}", nextStep); - log.debug("Before processPollEvent()"); + log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + log.debug("******** TEMP DEBUG ******** Before processPollEvent()"); processPollEvent(event.pollTimeMs()); - log.debug("After processPollEvent()"); + log.debug("******** TEMP DEBUG ******** After processPollEvent()"); // If there are enqueued callbacks to invoke, exit to the application thread. - if (offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0) { - log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); - event.future().complete(CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); + RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; + + if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS)) return; - } nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - log.debug("Set nextStep to {}", nextStep); + log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep); } if (nextStep == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("nextStep == {}", nextStep); - log.debug("Before processUpdatePatternSubscriptionEvent()"); + log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + log.debug("******** TEMP DEBUG ******** Before processUpdatePatternSubscriptionEvent()"); processUpdatePatternSubscriptionEvent(); - log.debug("After processUpdatePatternSubscriptionEvent()"); + log.debug("******** TEMP DEBUG ******** After processUpdatePatternSubscriptionEvent()"); // If there are background events to process, exit to the application thread. - if (backgroundEventHandler.size() > 0) { - log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); - event.future().complete(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); + RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0; + + if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING)) return; - } nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - log.debug("Set nextStep to {}", nextStep); + log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep); } if (nextStep == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.debug("nextStep == {}", nextStep); - processCheckAndUpdatePositionsEvent(event.deadlineMs()).whenComplete((__, updatePositionsError) -> { - log.debug("processCheckAndUpdatePositionsEvent complete, __: {}, updatePositionsError: {}", __, String.valueOf(updatePositionsError)); + log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + + long nowMs = time.milliseconds(); + long timeoutMs = event.deadlineMs() - nowMs; + + log.debug("******** TEMP DEBUG ******** deadlineMs: {}", event.deadlineMs()); + log.debug("******** TEMP DEBUG ******** nowMs: {}", nowMs); + log.debug("******** TEMP DEBUG ******** timeoutMs: {}", timeoutMs); - if (updatePositionsError != null && !(updatePositionsError instanceof TimeoutException)) { - log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError)); - event.future().completeExceptionally(updatePositionsError); + CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()) + .orTimeout(timeoutMs, TimeUnit.MILLISECONDS); + + updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + if (maybeFailCompositePoll(event.future(), updatePositionsError)) return; - } // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - log.debug("createFetchRequests complete, ___: {}, fetchError: {}", ___, String.valueOf(fetchError)); - - if (fetchError != null && !(fetchError instanceof TimeoutException)) { - log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError)); - event.future().completeExceptionally(fetchError); + if (maybeFailCompositePoll(event.future(), fetchError)) return; - } - log.debug("Yay! We did it! Exiting composite poll event with {}", CompositePollResult.COMPLETE); + log.debug("******** TEMP DEBUG ******** Exiting composite poll event with {}", CompositePollResult.COMPLETE); event.future().complete(CompositePollResult.COMPLETE); }); }); @@ -302,6 +305,29 @@ private void process(final CompositePollEvent event) { event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextStep)); } + private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, + CompletableFuture future, + CompositePollResult nextStep) { + if (test.requiresApplicationThread()) + return false; + + log.debug("******** TEMP DEBUG ******** Pausing composite poll at step {}", nextStep); + future.complete(nextStep); + return true; + } + + private boolean maybeFailCompositePoll(CompletableFuture future, Throwable t) { + if (t == null) + return false; + + if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) + return false; + + log.debug("******** TEMP DEBUG ******** Failing composite poll event", t); + future.completeExceptionally(t); + return true; + } + private void process(final PollEvent event) { processPollEvent(event.pollTimeMs()); event.markReconcileAndAutoCommitComplete(); @@ -793,6 +819,7 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, + final Time time, final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, @@ -804,6 +831,7 @@ protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); return new ApplicationEventProcessor( logContext, + time, requestManagers, metadata, subscriptions, @@ -885,4 +913,11 @@ private void processUpdatePatternSubscriptionEvent() { private CompletableFuture processCheckAndUpdatePositionsEvent(final long deadlineMs) { return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs); } + + private interface RequiresApplicationThreadExecution { + + boolean requiresApplicationThread(); + } + + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index e807fa7214ab9..309940b15b378 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,16 +16,25 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class CompositePollEvent extends CompletableApplicationEvent { +import java.util.concurrent.CompletableFuture; - private final long pollTimeMs; +public class CompositePollEvent extends ApplicationEvent { + private final long deadlineMs; + private final long pollTimeMs; private final Type nextStep; + private final CompletableFuture future; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextStep) { - super(Type.COMPOSITE_POLL, deadlineMs); + super(Type.COMPOSITE_POLL); + this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextStep = nextStep; + this.future = new CompletableFuture<>(); + } + + public long deadlineMs() { + return deadlineMs; } public long pollTimeMs() { @@ -36,8 +45,12 @@ public Type nextStep() { return nextStep; } + public CompletableFuture future() { + return future; + } + @Override protected String toStringBase() { - return super.toStringBase() + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep + ", future=" + future; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 6f28d09082c96..4875f6e00d36d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -411,6 +411,7 @@ public void testCommittedExceptionThrown() { assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); } + @Disabled @Test public void testWakeupBeforeCallingPoll() { consumer = newConsumer(); @@ -478,6 +479,7 @@ public void testWakeupAfterNonEmptyFetch() { assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); } + @Disabled @Test public void testCommitInRebalanceCallback() { consumer = newConsumer(); @@ -513,6 +515,7 @@ public void onPartitionsAssigned(final Collection partitions) { assertTrue(callbackExecuted.get()); } + @Disabled @Test public void testClearWakeupTriggerAfterPoll() { consumer = newConsumer(); @@ -665,6 +668,7 @@ private CompletableApplicationEvent addAndGetLastEnqueuedEvent() { return allValues.get(allValues.size() - 1); } + @Disabled @Test public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer = newConsumer(); @@ -1198,12 +1202,14 @@ public void testNoInterceptorCommitAsyncFailed() { assertEquals(0, MockConsumerInterceptor.ON_COMMIT_COUNT.get()); } + @Disabled @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { consumer = newConsumer(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); } + @Disabled @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { // Create consumer without group id so committed offsets are not used for updating positions @@ -1449,6 +1455,7 @@ public void testStreamRebalanceData() { * callback execution does not immediately errors. Instead, those errors are forwarded to the * application event thread for the {@link ConsumerMembershipManager} to handle. */ + @Disabled @ParameterizedTest @MethodSource("listenerCallbacksInvokeSource") public void testListenerCallbacksInvoke(List methodNames, @@ -1536,6 +1543,7 @@ private static Stream listenerCallbacksInvokeSource() { ); } + @Disabled @Test public void testBackgroundError() { final String groupId = "consumerGroupA"; @@ -1552,6 +1560,7 @@ public void testBackgroundError() { assertEquals(expectedException.getMessage(), exception.getMessage()); } + @Disabled @Test public void testMultipleBackgroundErrors() { final String groupId = "consumerGroupA"; @@ -1795,6 +1804,7 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { * Tests that calling {@link Thread#interrupt()} before {@link KafkaConsumer#poll(Duration)} * causes {@link InterruptException} to be thrown. */ + @Disabled @Test public void testPollThrowsInterruptExceptionIfInterrupted() { consumer = newConsumer(); @@ -1835,6 +1845,7 @@ void testReaperInvokedInUnsubscribe() { verify(backgroundEventReaper).reap(time.milliseconds()); } + @Disabled @Test void testReaperInvokedInPoll() { consumer = newConsumer(); @@ -1894,6 +1905,7 @@ public void testSeekToEnd() { assertEquals(AutoOffsetResetStrategy.LATEST, resetOffsetEvent.offsetResetStrategy()); } + @Disabled @Test public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer = newConsumer(); @@ -1955,6 +1967,7 @@ public void testSubscribeToRe2JPatternGeneratesEvent() { // SubscriptionPattern is supported as of ConsumerGroupHeartbeatRequest v1. Clients using subscribe // (SubscribePattern) against older broker versions should get UnsupportedVersionException on poll after subscribe + @Disabled @Test public void testSubscribePatternAgainstBrokerNotSupportingRegex() throws InterruptedException { final Properties props = requiredConsumerConfig(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index bd0600703be8c..e213ada3e64a6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -112,6 +112,7 @@ private void setupProcessor(boolean withGroupId) { ); processor = new ApplicationEventProcessor( new LogContext(), + time, requestManagers, metadata, subscriptionState, @@ -135,6 +136,7 @@ private void setupStreamProcessor(boolean withGroupId) { ); processor = new ApplicationEventProcessor( new LogContext(), + time, requestManagers, metadata, subscriptionState, From d4802c78e3b91f2d702ad88a76ca4764b7e8777d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 10 Sep 2025 21:16:46 -0700 Subject: [PATCH 003/123] Re-enabling tests in AsyncKafkaConsumer --- .../internals/AsyncKafkaConsumer.java | 2 +- .../internals/AsyncKafkaConsumerTest.java | 52 ++++++++++--------- 2 files changed, 28 insertions(+), 26 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b672eea2927f2..3a1af95bd9b1e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -892,7 +892,7 @@ public ConsumerRecords poll(final Duration timeout) { CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextStep); applicationEventHandler.add(event); - CompositePollResult result = ConsumerUtils.getResult(event.future()); + CompositePollResult result = ConsumerUtils.getResult(event.future(), defaultApiTimeoutMs.toMillis()); if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) { offsetCommitCallbackInvoker.executeCallbacks(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 4875f6e00d36d..b4a06b1f0f95e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -42,8 +42,9 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; +import org.apache.kafka.clients.consumer.internals.events.CompositePollResult; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent; 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; @@ -92,7 +93,6 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.params.ParameterizedTest; @@ -411,7 +411,6 @@ public void testCommittedExceptionThrown() { assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); } - @Disabled @Test public void testWakeupBeforeCallingPoll() { consumer = newConsumer(); @@ -427,11 +426,11 @@ public void testWakeupBeforeCallingPoll() { consumer.wakeup(); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } - @Disabled @Test public void testWakeupAfterEmptyFetch() { consumer = newConsumer(); @@ -448,11 +447,11 @@ public void testWakeupAfterEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } - @Disabled @Test public void testWakeupAfterNonEmptyFetch() { consumer = newConsumer(); @@ -473,13 +472,13 @@ public void testWakeupAfterNonEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); // the previously ignored wake-up should not be ignored in the next call assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); } - @Disabled @Test public void testCommitInRebalanceCallback() { consumer = newConsumer(); @@ -511,11 +510,11 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); consumer.poll(Duration.ZERO); assertTrue(callbackExecuted.get()); } - @Disabled @Test public void testClearWakeupTriggerAfterPoll() { consumer = newConsumer(); @@ -534,6 +533,7 @@ public void testClearWakeupTriggerAfterPoll() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); consumer.poll(Duration.ZERO); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); @@ -668,7 +668,6 @@ private CompletableApplicationEvent addAndGetLastEnqueuedEvent() { return allValues.get(allValues.size() - 1); } - @Disabled @Test public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer = newConsumer(); @@ -681,6 +680,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @@ -1202,14 +1202,12 @@ public void testNoInterceptorCommitAsyncFailed() { assertEquals(0, MockConsumerInterceptor.ON_COMMIT_COUNT.get()); } - @Disabled @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { consumer = newConsumer(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); } - @Disabled @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { // Create consumer without group id so committed offsets are not used for updating positions @@ -1455,7 +1453,6 @@ public void testStreamRebalanceData() { * callback execution does not immediately errors. Instead, those errors are forwarded to the * application event thread for the {@link ConsumerMembershipManager} to handle. */ - @Disabled @ParameterizedTest @MethodSource("listenerCallbacksInvokeSource") public void testListenerCallbacksInvoke(List methodNames, @@ -1484,6 +1481,8 @@ public void testListenerCallbacksInvoke(List listenerCallbacksInvokeSource() { ); } - @Disabled @Test public void testBackgroundError() { final String groupId = "consumerGroupA"; @@ -1555,12 +1553,12 @@ public void testBackgroundError() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); assertEquals(expectedException.getMessage(), exception.getMessage()); } - @Disabled @Test public void testMultipleBackgroundErrors() { final String groupId = "consumerGroupA"; @@ -1575,6 +1573,7 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); assertEquals(expectedException1.getMessage(), exception.getMessage()); @@ -1639,7 +1638,6 @@ public void testGroupIdNotNullAndValid() { assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); } - @Disabled @Test public void testEnsurePollEventSentOnConsumerPoll() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE); @@ -1659,9 +1657,9 @@ public void testEnsurePollEventSentOnConsumerPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(PollEvent.class)); - verify(applicationEventHandler).add(any(CreateFetchRequestsEvent.class)); + verify(applicationEventHandler).add(any(CompositePollEvent.class)); } private Properties requiredConsumerConfigAndGroupId(final String groupId) { @@ -1678,13 +1676,10 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); consumer.poll(Duration.ZERO); - - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(CheckAndUpdatePositionsEvent.class)); } - @Disabled @Test public void testLongPollWaitIsLimited() { consumer = newConsumer(); @@ -1716,6 +1711,7 @@ public void testLongPollWaitIsLimited() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); assertEquals(2, returnedRecords.count()); @@ -1804,7 +1800,6 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { * Tests that calling {@link Thread#interrupt()} before {@link KafkaConsumer#poll(Duration)} * causes {@link InterruptException} to be thrown. */ - @Disabled @Test public void testPollThrowsInterruptExceptionIfInterrupted() { consumer = newConsumer(); @@ -1821,6 +1816,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { try { Thread.currentThread().interrupt(); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); assertThrows(InterruptException.class, () -> consumer.poll(Duration.ZERO)); } finally { // clear interrupted state again since this thread may be reused by JUnit @@ -1845,7 +1841,6 @@ void testReaperInvokedInUnsubscribe() { verify(backgroundEventReaper).reap(time.milliseconds()); } - @Disabled @Test void testReaperInvokedInPoll() { consumer = newConsumer(); @@ -1854,6 +1849,7 @@ void testReaperInvokedInPoll() { consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); } @@ -1905,7 +1901,6 @@ public void testSeekToEnd() { assertEquals(AutoOffsetResetStrategy.LATEST, resetOffsetEvent.offsetResetStrategy()); } - @Disabled @Test public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer = newConsumer(); @@ -1918,6 +1913,7 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer.assign(singleton(new TopicPartition("topic1", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); + markResultForCompositePollEvent(CompositePollResult.COMPLETE); consumer.poll(Duration.ZERO); verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); @@ -1925,7 +1921,6 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer.subscribe(Pattern.compile("t*")); consumer.poll(Duration.ZERO); - verify(applicationEventHandler).addAndGet(any(UpdatePatternSubscriptionEvent.class)); } @Test @@ -1967,7 +1962,6 @@ public void testSubscribeToRe2JPatternGeneratesEvent() { // SubscriptionPattern is supported as of ConsumerGroupHeartbeatRequest v1. Clients using subscribe // (SubscribePattern) against older broker versions should get UnsupportedVersionException on poll after subscribe - @Disabled @Test public void testSubscribePatternAgainstBrokerNotSupportingRegex() throws InterruptedException { final Properties props = requiredConsumerConfig(); @@ -2235,4 +2229,12 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); } + + private void markResultForCompositePollEvent(CompositePollResult result) { + doAnswer(invocation -> { + CompositePollEvent event = invocation.getArgument(0); + event.future().complete(result); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); + } } From d3fa910d10cfe1147e646beac4ec033090af78dc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 15 Sep 2025 14:06:41 -0700 Subject: [PATCH 004/123] Minor clean up from design review --- .../internals/AsyncKafkaConsumer.java | 54 +++++++++---------- .../events/ApplicationEventProcessor.java | 42 +++++++-------- .../internals/events/CompositePollEvent.java | 23 +++++--- .../internals/events/CompositePollResult.java | 24 --------- .../internals/AsyncKafkaConsumerTest.java | 33 ++++++------ 5 files changed, 78 insertions(+), 98 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 3a1af95bd9b1e..0bf049caaaa12 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -50,7 +50,6 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; -import org.apache.kafka.clients.consumer.internals.events.CompositePollResult; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent; @@ -878,33 +877,7 @@ public ConsumerRecords poll(final Duration timeout) { // of the fetches. A wakeup between returned fetches and returning records would lead to never // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - - long pollTimeMs = timer.currentTimeMs(); - long deadlineMs = calculateDeadlineMs(timer); - - log.debug("******** TEMP DEBUG ******** timeout: {}", timeout.toMillis()); - log.debug("******** TEMP DEBUG ******** pollTimeMs: {}", pollTimeMs); - log.debug("******** TEMP DEBUG ******** deadlineMs: {}", deadlineMs); - - ApplicationEvent.Type nextStep = ApplicationEvent.Type.POLL; - - for (int i = 0; i < 10; i++) { - CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextStep); - applicationEventHandler.add(event); - - CompositePollResult result = ConsumerUtils.getResult(event.future(), defaultApiTimeoutMs.toMillis()); - - if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) { - offsetCommitCallbackInvoker.executeCallbacks(); - nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - } else if (result == CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING) { - processBackgroundEvents(); - nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - } else if (result == CompositePollResult.COMPLETE) { - break; - } - } - + prepareFetch(timer); final Fetch fetch = collectFetch(); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -932,6 +905,31 @@ public ConsumerRecords poll(final Duration timeout) { } } + private void prepareFetch(Timer timer) { + long pollTimeMs = timer.currentTimeMs(); + long deadlineMs = calculateDeadlineMs(timer); + + ApplicationEvent.Type nextEventType = ApplicationEvent.Type.POLL; + + while (true) { + CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); + applicationEventHandler.add(event); + CompositePollEvent.State state = ConsumerUtils.getResult(event.future(), defaultApiTimeoutMs.toMillis()); + + if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { + offsetCommitCallbackInvoker.executeCallbacks(); + nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { + processBackgroundEvents(); + nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + } else if (state == CompositePollEvent.State.COMPLETE) { + break; + } else { + throw new IllegalStateException("Unexpected state: " + state); + } + } + } + /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 34bfbb7da5adb..03cc5780910db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -237,11 +237,11 @@ public void process(ApplicationEvent event) { private void process(final CompositePollEvent event) { log.debug("******** TEMP DEBUG ******** Processing {}", event); - ApplicationEvent.Type nextStep = event.nextStep(); - log.debug("******** TEMP DEBUG ******** Processing nextStep: {}", nextStep); + ApplicationEvent.Type nextEventType = event.nextEventType(); + log.debug("******** TEMP DEBUG ******** Processing nextEventType: {}", nextEventType); - if (nextStep == ApplicationEvent.Type.POLL) { - log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + if (nextEventType == ApplicationEvent.Type.POLL) { + log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); log.debug("******** TEMP DEBUG ******** Before processPollEvent()"); processPollEvent(event.pollTimeMs()); log.debug("******** TEMP DEBUG ******** After processPollEvent()"); @@ -249,15 +249,15 @@ private void process(final CompositePollEvent event) { // If there are enqueued callbacks to invoke, exit to the application thread. RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; - if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS)) + if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) return; - nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep); + nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + log.debug("******** TEMP DEBUG ******** Set nextEventType to {}", nextEventType); } - if (nextStep == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { + log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); log.debug("******** TEMP DEBUG ******** Before processUpdatePatternSubscriptionEvent()"); processUpdatePatternSubscriptionEvent(); log.debug("******** TEMP DEBUG ******** After processUpdatePatternSubscriptionEvent()"); @@ -265,15 +265,15 @@ private void process(final CompositePollEvent event) { // If there are background events to process, exit to the application thread. RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0; - if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING)) + if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) return; - nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep); + nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + log.debug("******** TEMP DEBUG ******** Set nextEventType to {}", nextEventType); } - if (nextStep == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep); + if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { + log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); long nowMs = time.milliseconds(); long timeoutMs = event.deadlineMs() - nowMs; @@ -294,25 +294,25 @@ private void process(final CompositePollEvent event) { if (maybeFailCompositePoll(event.future(), fetchError)) return; - log.debug("******** TEMP DEBUG ******** Exiting composite poll event with {}", CompositePollResult.COMPLETE); - event.future().complete(CompositePollResult.COMPLETE); + log.debug("******** TEMP DEBUG ******** Exiting composite poll event with {}", CompositePollEvent.State.COMPLETE); + event.future().complete(CompositePollEvent.State.COMPLETE); }); }); return; } - event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextStep)); + event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextEventType)); } private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, - CompletableFuture future, - CompositePollResult nextStep) { + CompletableFuture future, + CompositePollEvent.State state) { if (test.requiresApplicationThread()) return false; - log.debug("******** TEMP DEBUG ******** Pausing composite poll at step {}", nextStep); - future.complete(nextStep); + log.debug("******** TEMP DEBUG ******** Pausing composite poll at state {}", state); + future.complete(state); return true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 309940b15b378..b423982770ec1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -20,16 +20,23 @@ public class CompositePollEvent extends ApplicationEvent { + public enum State { + + OFFSET_COMMIT_CALLBACKS_REQUIRED, + BACKGROUND_EVENT_PROCESSING_REQUIRED, + COMPLETE + } + private final long deadlineMs; private final long pollTimeMs; - private final Type nextStep; - private final CompletableFuture future; + private final Type nextEventType; + private final CompletableFuture future; - public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextStep) { + public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { super(Type.COMPOSITE_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; - this.nextStep = nextStep; + this.nextEventType = nextEventType; this.future = new CompletableFuture<>(); } @@ -41,16 +48,16 @@ public long pollTimeMs() { return pollTimeMs; } - public Type nextStep() { - return nextStep; + public Type nextEventType() { + return nextEventType; } - public CompletableFuture future() { + public CompletableFuture future() { return future; } @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep + ", future=" + future; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", future=" + future; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java deleted file mode 100644 index 0188c1aa60a28..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollResult.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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; - -public enum CompositePollResult { - - NEEDS_OFFSET_COMMIT_CALLBACKS, - NEEDS_BACKGROUND_EVENT_PROCESSING, - COMPLETE -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index b4a06b1f0f95e..83618f9fb6ea8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -43,7 +43,6 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; -import org.apache.kafka.clients.consumer.internals.events.CompositePollResult; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; @@ -426,7 +425,7 @@ public void testWakeupBeforeCallingPoll() { consumer.wakeup(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -447,7 +446,7 @@ public void testWakeupAfterEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -472,7 +471,7 @@ public void testWakeupAfterNonEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); // the previously ignored wake-up should not be ignored in the next call @@ -510,7 +509,7 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); consumer.poll(Duration.ZERO); assertTrue(callbackExecuted.get()); } @@ -533,7 +532,7 @@ public void testClearWakeupTriggerAfterPoll() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ZERO); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); @@ -680,7 +679,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS); + markResultForCompositePollEvent(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @@ -1481,7 +1480,7 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); assertEquals(expectedException.getMessage(), exception.getMessage()); @@ -1573,7 +1572,7 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); assertEquals(expectedException1.getMessage(), exception.getMessage()); @@ -1657,7 +1656,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ofMillis(100)); verify(applicationEventHandler).add(any(CompositePollEvent.class)); } @@ -1676,7 +1675,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ZERO); } @@ -1711,7 +1710,7 @@ public void testLongPollWaitIsLimited() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); assertEquals(2, returnedRecords.count()); @@ -1816,7 +1815,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { try { Thread.currentThread().interrupt(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); assertThrows(InterruptException.class, () -> consumer.poll(Duration.ZERO)); } finally { // clear interrupted state again since this thread may be reused by JUnit @@ -1849,7 +1848,7 @@ void testReaperInvokedInPoll() { consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); } @@ -1913,7 +1912,7 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer.assign(singleton(new TopicPartition("topic1", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollResult.COMPLETE); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ZERO); verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); @@ -2230,7 +2229,7 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); } - private void markResultForCompositePollEvent(CompositePollResult result) { + private void markResultForCompositePollEvent(CompositePollEvent.State result) { doAnswer(invocation -> { CompositePollEvent event = invocation.getArgument(0); event.future().complete(result); From dbc4773a341f674553effd00504a34afaad5c24a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 15 Sep 2025 14:46:46 -0700 Subject: [PATCH 005/123] Updates to fix inverted logic in maybeInterruptCompositePoll() --- .../events/ApplicationEventProcessor.java | 12 ++-- .../internals/AsyncKafkaConsumerTest.java | 59 +++++++++++++++++-- 2 files changed, 59 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 03cc5780910db..6881ff7c7dc4a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -249,7 +249,7 @@ private void process(final CompositePollEvent event) { // If there are enqueued callbacks to invoke, exit to the application thread. RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; - if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) + if (maybeInterruptCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; @@ -265,7 +265,7 @@ private void process(final CompositePollEvent event) { // If there are background events to process, exit to the application thread. RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0; - if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) + if (maybeInterruptCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; @@ -305,10 +305,10 @@ private void process(final CompositePollEvent event) { event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextEventType)); } - private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, - CompletableFuture future, - CompositePollEvent.State state) { - if (test.requiresApplicationThread()) + private boolean maybeInterruptCompositePoll(RequiresApplicationThreadExecution test, + CompletableFuture future, + CompositePollEvent.State state) { + if (!test.requiresApplicationThread()) return false; log.debug("******** TEMP DEBUG ******** Pausing composite poll at state {}", state); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 83618f9fb6ea8..7e2f0cba909bb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -109,6 +109,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Map; @@ -509,7 +510,13 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + markResultForCompositePollEvent( + List.of( + CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, + CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, + CompositePollEvent.State.COMPLETE + ) + ); consumer.poll(Duration.ZERO); assertTrue(callbackExecuted.get()); } @@ -535,6 +542,7 @@ public void testClearWakeupTriggerAfterPoll() { markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ZERO); + markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -679,7 +687,13 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); + markResultForCompositePollEvent( + List.of( + CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, + CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, + CompositePollEvent.State.COMPLETE + ) + ); assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @@ -1480,7 +1494,12 @@ public void testListenerCallbacksInvoke(List { CompositePollEvent event = invocation.getArgument(0); - event.future().complete(result); + + if (Thread.currentThread().isInterrupted()) + event.future().completeExceptionally(new InterruptException("Test interrupt")); + else + event.future().complete(state); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); + } + + private void markResultForCompositePollEvent(Collection states) { + LinkedList statesQueue = new LinkedList<>(states); + + doAnswer(invocation -> { + CompositePollEvent.State state = statesQueue.poll(); + + if (state == null) + throw new IllegalStateException("The array of " + CompositePollEvent.State.class.getSimpleName() + " did not provide enough values"); + + CompositePollEvent event = invocation.getArgument(0); + + if (Thread.currentThread().isInterrupted()) + event.future().completeExceptionally(new InterruptException("Test interrupt")); + else + event.future().complete(state); return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } From 09f8cb57cfa437de57c69a8c9e3d7a4c3aaa63b2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 15 Sep 2025 14:50:37 -0700 Subject: [PATCH 006/123] Add documentation for RequiresApplicationThreadExecution Added a Javadoc comment to the RequiresApplicationThreadExecution interface to clarify its purpose and usage, specifically regarding the need to interrupt CompositePollEvent processing when requiresApplicationThread() returns true. --- .../internals/events/ApplicationEventProcessor.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 6881ff7c7dc4a..53c432e196847 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -914,10 +914,13 @@ private CompletableFuture processCheckAndUpdatePositionsEvent(final lon return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs); } + /** + * This interface exists mostly to make the code more intuitive. When {@link #requiresApplicationThread()} + * returns true, the {@link CompositePollEvent} processing needs to be interrupted so that processing + * can return to the application thread. + */ private interface RequiresApplicationThreadExecution { boolean requiresApplicationThread(); } - - } From 5e794ce079dc8d78a212cf90a9dea399ff31fc97 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 15 Sep 2025 15:57:06 -0700 Subject: [PATCH 007/123] Inject NetworkClientDelegate via supplier for ApplicationEventProcessor to check metadata errors Refactors AsyncKafkaConsumer, ShareConsumerImpl, and ApplicationEventProcessor to inject NetworkClientDelegate using a supplier method. Adds a static supplier factory to NetworkClientDelegate for deferred instantiation. Updates related tests and construction logic to support the new dependency injection approach. --- .../internals/AsyncKafkaConsumer.java | 4 ++- .../internals/NetworkClientDelegate.java | 30 +++++++++++++++++++ .../consumer/internals/ShareConsumerImpl.java | 4 ++- .../events/ApplicationEventProcessor.java | 19 +++++++++++- .../clients/consumer/KafkaConsumerTest.java | 2 -- .../events/ApplicationEventProcessorTest.java | 5 +++- 6 files changed, 58 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 0bf049caaaa12..db4eebb43ca1e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -502,6 +502,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker) ); @@ -660,7 +661,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, new RebalanceCallbackMetricsManager(metrics) ); ApiVersions apiVersions = new ApiVersions(); - Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( + Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier( time, config, logContext, @@ -695,6 +696,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker) ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 3c280e39d0279..374ff96fd5e8b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -471,4 +471,34 @@ protected NetworkClientDelegate create() { } }; } + + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ + public static Supplier supplier(final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client, + final Metadata metadata, + final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, + final AsyncConsumerMetrics asyncConsumerMetrics) { + return new CachedSupplier<>() { + @Override + protected NetworkClientDelegate create() { + return new NetworkClientDelegate( + time, + config, + logContext, + client, + metadata, + backgroundEventHandler, + notifyMetadataErrorsViaErrorQueue, + asyncConsumerMetrics + ); + } + }; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 4d0730cb231f0..0d62216d40446 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -303,6 +303,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.empty() ); @@ -387,7 +388,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { backgroundEventQueue, time, asyncConsumerMetrics); final Supplier networkClientDelegateSupplier = - () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics); + NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, @@ -412,6 +413,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.empty() ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 53c432e196847..c914405b73a68 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; @@ -35,8 +36,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.LogContext; - import org.apache.kafka.common.utils.Time; + import org.slf4j.Logger; import java.util.Collection; @@ -57,6 +58,7 @@ * An {@link EventProcessor} that is created and executes in the {@link ConsumerNetworkThread network thread} * which processes {@link ApplicationEvent application events} generated by the application thread. */ +@SuppressWarnings({"ClassFanOutComplexity"}) public class ApplicationEventProcessor implements EventProcessor { private final Logger log; @@ -64,6 +66,7 @@ public class ApplicationEventProcessor implements EventProcessor offsetCommitCallbackInvoker; private int metadataVersionSnapshot; @@ -71,6 +74,7 @@ public class ApplicationEventProcessor implements EventProcessor exception = networkClientDelegate.getAndClearMetadataError(); + + if (exception.isPresent()) { + log.debug("******** TEMP DEBUG ******** Metadata error: {} from network client delegate {}", exception, networkClientDelegate.hashCode()); + event.future().completeExceptionally(exception.get()); + return; + } + ApplicationEvent.Type nextEventType = event.nextEventType(); log.debug("******** TEMP DEBUG ******** Processing nextEventType: {}", nextEventType); @@ -823,16 +836,20 @@ public static Supplier supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, + final Supplier networkClientDelegateSupplier, final BackgroundEventHandler backgroundEventHandler, final Optional offsetCommitCallbackInvoker) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); + NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); + return new ApplicationEventProcessor( logContext, time, requestManagers, + networkClientDelegate, metadata, subscriptions, backgroundEventHandler, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 6f168d9321cf9..60063e5226888 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -110,7 +110,6 @@ import org.apache.logging.log4j.Level; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -3141,7 +3140,6 @@ private static class FetchInfo { } } - @Disabled @ParameterizedTest @EnumSource(GroupProtocol.class) public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws InterruptedException { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index e213ada3e64a6..00d901a7a190c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -82,7 +82,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@SuppressWarnings("ClassDataAbstractionCoupling") +@SuppressWarnings({"ClassDataAbstractionCoupling", "ClassFanOutComplexity"}) public class ApplicationEventProcessorTest { private final Time time = new MockTime(); private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); @@ -95,6 +95,7 @@ public class ApplicationEventProcessorTest { private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); private final BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); + private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private ApplicationEventProcessor processor; private void setupProcessor(boolean withGroupId) { @@ -114,6 +115,7 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), time, requestManagers, + networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, @@ -138,6 +140,7 @@ private void setupStreamProcessor(boolean withGroupId) { new LogContext(), time, requestManagers, + networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, From 464d5bafb9c6268624b1ae244d4a878208e31c81 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 16 Sep 2025 13:47:15 -0700 Subject: [PATCH 008/123] Removed the verbose logging --- .../events/ApplicationEventProcessor.java | 30 ++++--------------- 1 file changed, 5 insertions(+), 25 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index c914405b73a68..fbf2d793e9a06 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -240,61 +240,42 @@ public void process(ApplicationEvent event) { } private void process(final CompositePollEvent event) { - log.debug("******** TEMP DEBUG ******** Processing {}", event); - Optional exception = networkClientDelegate.getAndClearMetadataError(); if (exception.isPresent()) { - log.debug("******** TEMP DEBUG ******** Metadata error: {} from network client delegate {}", exception, networkClientDelegate.hashCode()); event.future().completeExceptionally(exception.get()); return; } ApplicationEvent.Type nextEventType = event.nextEventType(); - log.debug("******** TEMP DEBUG ******** Processing nextEventType: {}", nextEventType); if (nextEventType == ApplicationEvent.Type.POLL) { - log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); - log.debug("******** TEMP DEBUG ******** Before processPollEvent()"); processPollEvent(event.pollTimeMs()); - log.debug("******** TEMP DEBUG ******** After processPollEvent()"); // If there are enqueued callbacks to invoke, exit to the application thread. RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; - if (maybeInterruptCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) + if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - log.debug("******** TEMP DEBUG ******** Set nextEventType to {}", nextEventType); } if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); - log.debug("******** TEMP DEBUG ******** Before processUpdatePatternSubscriptionEvent()"); processUpdatePatternSubscriptionEvent(); - log.debug("******** TEMP DEBUG ******** After processUpdatePatternSubscriptionEvent()"); // If there are background events to process, exit to the application thread. RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0; - if (maybeInterruptCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) + if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - log.debug("******** TEMP DEBUG ******** Set nextEventType to {}", nextEventType); } if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.debug("******** TEMP DEBUG ******** nextEventType == {}", nextEventType); - long nowMs = time.milliseconds(); long timeoutMs = event.deadlineMs() - nowMs; - - log.debug("******** TEMP DEBUG ******** deadlineMs: {}", event.deadlineMs()); - log.debug("******** TEMP DEBUG ******** nowMs: {}", nowMs); - log.debug("******** TEMP DEBUG ******** timeoutMs: {}", timeoutMs); - CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()) .orTimeout(timeoutMs, TimeUnit.MILLISECONDS); @@ -307,7 +288,6 @@ private void process(final CompositePollEvent event) { if (maybeFailCompositePoll(event.future(), fetchError)) return; - log.debug("******** TEMP DEBUG ******** Exiting composite poll event with {}", CompositePollEvent.State.COMPLETE); event.future().complete(CompositePollEvent.State.COMPLETE); }); }); @@ -318,9 +298,9 @@ private void process(final CompositePollEvent event) { event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextEventType)); } - private boolean maybeInterruptCompositePoll(RequiresApplicationThreadExecution test, - CompletableFuture future, - CompositePollEvent.State state) { + private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, + CompletableFuture future, + CompositePollEvent.State state) { if (!test.requiresApplicationThread()) return false; From d253b847e65f57dda3876292b7f11cdf5a2d2ccc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 16 Sep 2025 17:24:50 -0700 Subject: [PATCH 009/123] Work in progress to get past most of the integration test issues --- .../internals/AsyncKafkaConsumer.java | 27 +++-- .../consumer/internals/ShareConsumerImpl.java | 14 ++- .../events/ApplicationEventProcessor.java | 107 +++++++++++++----- .../events/CompletableEventReaper.java | 66 ++++++++--- .../internals/events/CompositePollEvent.java | 20 +--- .../internals/AsyncKafkaConsumerTest.java | 24 ++-- .../events/ApplicationEventProcessorTest.java | 9 +- 7 files changed, 173 insertions(+), 94 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index db4eebb43ca1e..bd4a62c85d518 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -497,20 +497,21 @@ public AsyncKafkaConsumer(final ConsumerConfig config, memberStateListener, streamsRebalanceData ); + final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, - time, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker) + Optional.of(offsetCommitCallbackInvoker), + applicationEventReaper ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, time, applicationEventQueue, - new CompletableEventReaper(logContext), + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -690,20 +691,21 @@ public AsyncKafkaConsumer(final ConsumerConfig config, memberStateListener, Optional.empty() ); + final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, - time, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker) + Optional.of(offsetCommitCallbackInvoker), + applicationEventReaper ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, - new CompletableEventReaper(logContext), + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -910,13 +912,18 @@ public ConsumerRecords poll(final Duration timeout) { private void prepareFetch(Timer timer) { long pollTimeMs = timer.currentTimeMs(); long deadlineMs = calculateDeadlineMs(timer); - ApplicationEvent.Type nextEventType = ApplicationEvent.Type.POLL; while (true) { CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); - applicationEventHandler.add(event); - CompositePollEvent.State state = ConsumerUtils.getResult(event.future(), defaultApiTimeoutMs.toMillis()); + CompositePollEvent.State state; + + try { + state = applicationEventHandler.addAndGet(event); + } catch (TimeoutException e) { + // Timeouts are OK, there's just no data to return on this pass. + break; + } if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { offsetCommitCallbackInvoker.executeCallbacks(); @@ -930,6 +937,8 @@ private void prepareFetch(Timer timer) { throw new IllegalStateException("Unexpected state: " + state); } } + + timer.update(); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 0d62216d40446..08767c397e41d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -297,22 +297,23 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { clientTelemetryReporter, metrics ); + final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, - time, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, backgroundEventHandler, - Optional.empty() + Optional.empty(), + applicationEventReaper ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, time, applicationEventQueue, - new CompletableEventReaper(logContext), + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -407,22 +408,23 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metrics ); + final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, - time, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, backgroundEventHandler, - Optional.empty() + Optional.empty(), + applicationEventReaper ); this.applicationEventHandler = new ApplicationEventHandler( logContext, time, applicationEventQueue, - new CompletableEventReaper(logContext), + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index fbf2d793e9a06..eaff3af8d0090 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -36,7 +36,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; @@ -49,7 +48,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -62,32 +60,56 @@ public class ApplicationEventProcessor implements EventProcessor { private final Logger log; - private final Time time; private final ConsumerMetadata metadata; private final SubscriptionState subscriptions; private final RequestManagers requestManagers; private final NetworkClientDelegate networkClientDelegate; - private final BackgroundEventHandler backgroundEventHandler; - private final Optional offsetCommitCallbackInvoker; + private final RequiresApplicationThreadExecution backgroundEventProcessingRequiredTest; + private final RequiresApplicationThreadExecution offsetCommitCallbackInvocationRequiredTest; + private final CompletableEventReaper applicationEventReaper; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, - final Time time, final RequestManagers requestManagers, final NetworkClientDelegate networkClientDelegate, final ConsumerMetadata metadata, final SubscriptionState subscriptions, final BackgroundEventHandler backgroundEventHandler, - final Optional offsetCommitCallbackInvoker) { + final Optional offsetCommitCallbackInvoker, + final CompletableEventReaper applicationEventReaper) { this.log = logContext.logger(ApplicationEventProcessor.class); - this.time = time; this.requestManagers = requestManagers; this.networkClientDelegate = networkClientDelegate; this.metadata = metadata; this.subscriptions = subscriptions; - this.backgroundEventHandler = backgroundEventHandler; - this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; + this.applicationEventReaper = applicationEventReaper; this.metadataVersionSnapshot = metadata.updateVersion(); + + // If there are background events to process, exit to the application thread. + this.backgroundEventProcessingRequiredTest = new RequiresApplicationThreadExecution() { + @Override + public boolean requiresApplicationThread() { + return backgroundEventHandler.size() > 0; + } + + @Override + public CompositePollEvent.State targetState() { + return CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED; + } + }; + + // If there are enqueued callbacks to invoke, exit to the application thread. + this.offsetCommitCallbackInvocationRequiredTest = new RequiresApplicationThreadExecution() { + @Override + public boolean requiresApplicationThread() { + return offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; + } + + @Override + public CompositePollEvent.State targetState() { + return CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED; + } + }; } @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @@ -252,10 +274,7 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.POLL) { processPollEvent(event.pollTimeMs()); - // If there are enqueued callbacks to invoke, exit to the application thread. - RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; - - if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED)) + if (maybePauseCompositePoll(offsetCommitCallbackInvocationRequiredTest, event.future())) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; @@ -264,22 +283,22 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { processUpdatePatternSubscriptionEvent(); - // If there are background events to process, exit to the application thread. - RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0; - - if (maybePauseCompositePoll(test, event.future(), CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED)) + if (maybePauseCompositePoll(backgroundEventProcessingRequiredTest, event.future())) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; } if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - long nowMs = time.milliseconds(); - long timeoutMs = event.deadlineMs() - nowMs; - CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()) - .orTimeout(timeoutMs, TimeUnit.MILLISECONDS); + // This is a bit tricky... The CompositePollEvent should be "paused" from being reaped while the code + // for new CheckAndUpdatePositionsEvent is in flight. + applicationEventReaper.pause(event); + CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); + applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + applicationEventReaper.resume(event); + if (maybeFailCompositePoll(event.future(), updatePositionsError)) return; @@ -299,13 +318,13 @@ private void process(final CompositePollEvent event) { } private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, - CompletableFuture future, - CompositePollEvent.State state) { + CompletableFuture future) { if (!test.requiresApplicationThread()) return false; - log.debug("******** TEMP DEBUG ******** Pausing composite poll at state {}", state); - future.complete(state); + CompositePollEvent.State targetState = test.targetState(); + log.debug("******** TEMP DEBUG ******** Pausing composite poll to process logic for target state {}", targetState); + future.complete(targetState); return true; } @@ -812,13 +831,13 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, - final Time time, final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, final Supplier networkClientDelegateSupplier, final BackgroundEventHandler backgroundEventHandler, - final Optional offsetCommitCallbackInvoker) { + final Optional offsetCommitCallbackInvoker, + final CompletableEventReaper applicationEventReaper) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { @@ -827,13 +846,13 @@ protected ApplicationEventProcessor create() { return new ApplicationEventProcessor( logContext, - time, requestManagers, networkClientDelegate, metadata, subscriptions, backgroundEventHandler, - offsetCommitCallbackInvoker + offsetCommitCallbackInvoker, + applicationEventReaper ); } }; @@ -919,5 +938,33 @@ private CompletableFuture processCheckAndUpdatePositionsEvent(final lon private interface RequiresApplicationThreadExecution { boolean requiresApplicationThread(); + + CompositePollEvent.State targetState(); + } + + private static class CompositePollPsuedoEvent implements CompletableEvent { + + private final CompletableFuture future; + private final long deadlineMs; + + public CompositePollPsuedoEvent(CompletableFuture future, long deadlineMs) { + this.future = future; + this.deadlineMs = deadlineMs; + } + + @Override + public CompletableFuture future() { + return future; + } + + @Override + public long deadlineMs() { + return deadlineMs; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{future=" + future + ", deadlineMs=" + deadlineMs + '}'; + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index b4440de06264b..34e21a77ee20c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -25,8 +25,9 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; @@ -41,13 +42,13 @@ public class CompletableEventReaper { private final Logger log; /** - * List of tracked events that are candidates for expiration. + * Tracked events that are candidates for expiration. The key is the event and the value is a flag for paused. */ - private final List> tracked; + private final Map, Boolean> tracked; public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); - this.tracked = new ArrayList<>(); + this.tracked = new HashMap<>(); } /** @@ -56,7 +57,36 @@ public CompletableEventReaper(LogContext logContext) { * @param event Event to track */ public void add(CompletableEvent event) { - tracked.add(Objects.requireNonNull(event, "Event to track must be non-null")); + put(event, false, false, "add"); + } + + /** + * Pauses an event to exclude it as a candidate for reaps. + * + * @param event Event to pause + */ + public void pause(CompletableEvent event) { + put(event, true, true, "pause"); + } + + /** + * Resumes tracking an event for reaping. + * + * @param event Event to resume + */ + public void resume(CompletableEvent event) { + put(event, false, true, "resume"); + } + + private void put(CompletableEvent event, boolean paused, boolean checkExists, String verb) { + Objects.requireNonNull(event, "Event to " + verb + " must be non-null"); + + if (checkExists && !tracked.containsKey(event)) + throw new IllegalArgumentException("The event " + event + " was not previously added; cannot " + verb); + + tracked.put(event, paused); + + log.debug("Event {} was {}-d with paused set to {}", event, verb, paused); } /** @@ -85,15 +115,23 @@ public void add(CompletableEvent event) { */ public long reap(long currentTimeMs) { int count = 0; + List> unpausedEvents = new ArrayList<>(tracked.size()); - Iterator> iterator = tracked.iterator(); + for (Map.Entry, Boolean> entry : tracked.entrySet()) { + boolean isPaused = entry.getValue(); - while (iterator.hasNext()) { - CompletableEvent event = iterator.next(); + if (isPaused) { + // Don't reap "paused" events + continue; + } + unpausedEvents.add(entry.getKey()); + } + + for (CompletableEvent event : unpausedEvents) { if (event.future().isDone()) { // Remove any events that are already complete. - iterator.remove(); + tracked.remove(event); continue; } @@ -115,7 +153,7 @@ public long reap(long currentTimeMs) { count++; // Remove the events so that we don't hold a reference to it. - iterator.remove(); + tracked.remove(event); } return count; @@ -143,7 +181,7 @@ public long reap(long currentTimeMs) { public long reap(Collection events) { Objects.requireNonNull(events, "Event queue to reap must be non-null"); - long trackedExpiredCount = completeEventsExceptionallyOnClose(tracked); + long trackedExpiredCount = completeEventsExceptionallyOnClose(tracked.keySet()); tracked.clear(); long eventExpiredCount = completeEventsExceptionallyOnClose(events); @@ -157,7 +195,7 @@ public int size() { } public boolean contains(CompletableEvent event) { - return event != null && tracked.contains(event); + return event != null && tracked.containsKey(event); } public List> uncompletedEvents() { @@ -165,7 +203,9 @@ public List> uncompletedEvents() { // path of the ConsumerNetworkThread loop. List> events = new ArrayList<>(); - for (CompletableEvent event : tracked) { + for (Map.Entry, Boolean> entry : tracked.entrySet()) { + CompletableEvent event = entry.getKey(); + if (!event.future().isDone()) events.add(event); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index b423982770ec1..a54e70bb78bfa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,9 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import java.util.concurrent.CompletableFuture; - -public class CompositePollEvent extends ApplicationEvent { +public class CompositePollEvent extends CompletableApplicationEvent { public enum State { @@ -27,21 +25,13 @@ public enum State { COMPLETE } - private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; - private final CompletableFuture future; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { - super(Type.COMPOSITE_POLL); - this.deadlineMs = deadlineMs; + super(Type.COMPOSITE_POLL, deadlineMs); this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; - this.future = new CompletableFuture<>(); - } - - public long deadlineMs() { - return deadlineMs; } public long pollTimeMs() { @@ -52,12 +42,8 @@ public Type nextEventType() { return nextEventType; } - public CompletableFuture future() { - return future; - } - @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", future=" + future; + return super.toStringBase() + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 7e2f0cba909bb..b27eb18f59736 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1677,7 +1677,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { markReconcileAndAutoCommitCompleteForPollEvent(); markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(CompositePollEvent.class)); + verify(applicationEventHandler).addAndGet(any(CompositePollEvent.class)); } private Properties requiredConsumerConfigAndGroupId(final String groupId) { @@ -2255,14 +2255,11 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { private void markResultForCompositePollEvent(CompositePollEvent.State state) { doAnswer(invocation -> { - CompositePollEvent event = invocation.getArgument(0); - if (Thread.currentThread().isInterrupted()) - event.future().completeExceptionally(new InterruptException("Test interrupt")); - else - event.future().complete(state); - return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); + throw new InterruptException("Test interrupt"); + + return state; + }).when(applicationEventHandler).addAndGet(ArgumentMatchers.isA(CompositePollEvent.class)); } private void markResultForCompositePollEvent(Collection states) { @@ -2274,13 +2271,10 @@ private void markResultForCompositePollEvent(Collection Date: Wed, 17 Sep 2025 12:14:14 -0700 Subject: [PATCH 010/123] Clean up logic related to metadata errors that can happen along any step of CompositePollEvent --- .../events/ApplicationEventProcessor.java | 54 +++++++++++++------ 1 file changed, 37 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index eaff3af8d0090..358071d8dcde1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -262,19 +262,17 @@ public void process(ApplicationEvent event) { } private void process(final CompositePollEvent event) { - Optional exception = networkClientDelegate.getAndClearMetadataError(); - - if (exception.isPresent()) { - event.future().completeExceptionally(exception.get()); + if (maybeFailCompositePoll(event)) return; - } ApplicationEvent.Type nextEventType = event.nextEventType(); if (nextEventType == ApplicationEvent.Type.POLL) { processPollEvent(event.pollTimeMs()); - if (maybePauseCompositePoll(offsetCommitCallbackInvocationRequiredTest, event.future())) + if (maybeFailCompositePoll(event)) + return; + else if (maybePauseCompositePoll(event, offsetCommitCallbackInvocationRequiredTest)) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; @@ -283,7 +281,9 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { processUpdatePatternSubscriptionEvent(); - if (maybePauseCompositePoll(backgroundEventProcessingRequiredTest, event.future())) + if (maybeFailCompositePoll(event)) + return; + else if (maybePauseCompositePoll(event, backgroundEventProcessingRequiredTest)) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; @@ -297,16 +297,19 @@ private void process(final CompositePollEvent event) { applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + // Make sure to resume the CompositePollEvent *before* checking for failure so that it is assured + // to be resumed. applicationEventReaper.resume(event); - if (maybeFailCompositePoll(event.future(), updatePositionsError)) + if (maybeFailCompositePoll(event, updatePositionsError)) return; // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event.future(), fetchError)) + if (maybeFailCompositePoll(event, fetchError)) return; + log.trace("Completing CompositePollEvent {}", event); event.future().complete(CompositePollEvent.State.COMPLETE); }); }); @@ -317,29 +320,46 @@ private void process(final CompositePollEvent event) { event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextEventType)); } - private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test, - CompletableFuture future) { + private boolean maybePauseCompositePoll(CompositePollEvent event, RequiresApplicationThreadExecution test) { if (!test.requiresApplicationThread()) return false; CompositePollEvent.State targetState = test.targetState(); - log.debug("******** TEMP DEBUG ******** Pausing composite poll to process logic for target state {}", targetState); - future.complete(targetState); + log.trace("Pausing CompositePollEvent {} to process logic for target state {}", event, targetState); + event.future().complete(targetState); return true; } - private boolean maybeFailCompositePoll(CompletableFuture future, Throwable t) { + private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { + if (maybeFailCompositePoll(event)) + return true; + if (t == null) return false; - if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) + if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { + log.trace("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); return false; + } - log.debug("******** TEMP DEBUG ******** Failing composite poll event", t); - future.completeExceptionally(t); + log.trace("Failing CompositePollEvent {}", event, t); + event.future().completeExceptionally(t); return true; } + private boolean maybeFailCompositePoll(CompositePollEvent event) { + Optional exception = networkClientDelegate.getAndClearMetadataError(); + + if (exception.isPresent()) { + Exception e = exception.get(); + log.trace("Failing CompositePollEvent {} with error from NetworkClient", event, e); + event.future().completeExceptionally(e); + return true; + } + + return false; + } + private void process(final PollEvent event) { processPollEvent(event.pollTimeMs()); event.markReconcileAndAutoCommitComplete(); From 40f6754810b4625248533d5593821b33175d3d1d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 17 Sep 2025 12:31:11 -0700 Subject: [PATCH 011/123] Minor updates for CompletableEventReaper logging --- .../events/CompletableEventReaper.java | 55 +++++++++++++------ 1 file changed, 39 insertions(+), 16 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 34e21a77ee20c..3a676b81490d0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -39,12 +39,32 @@ */ public class CompletableEventReaper { + private enum State { + ACTIVE, + PAUSED + } + + private enum Modification { + + ADD(State.ACTIVE, "added"), + PAUSE(State.PAUSED, "paused"), + RESUME(State.ACTIVE, "resumed"); + + private final State state; + private final String verb; + + Modification(State state, String verb) { + this.state = state; + this.verb = verb; + } + } + private final Logger log; /** - * Tracked events that are candidates for expiration. The key is the event and the value is a flag for paused. + * Tracked events that are candidates for expiration. */ - private final Map, Boolean> tracked; + private final Map, State> tracked; public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); @@ -57,7 +77,7 @@ public CompletableEventReaper(LogContext logContext) { * @param event Event to track */ public void add(CompletableEvent event) { - put(event, false, false, "add"); + put(event, Modification.ADD); } /** @@ -66,7 +86,7 @@ public void add(CompletableEvent event) { * @param event Event to pause */ public void pause(CompletableEvent event) { - put(event, true, true, "pause"); + put(event, Modification.PAUSE); } /** @@ -75,18 +95,21 @@ public void pause(CompletableEvent event) { * @param event Event to resume */ public void resume(CompletableEvent event) { - put(event, false, true, "resume"); + put(event, Modification.RESUME); } - private void put(CompletableEvent event, boolean paused, boolean checkExists, String verb) { - Objects.requireNonNull(event, "Event to " + verb + " must be non-null"); + private void put(CompletableEvent event, Modification modification) { + Objects.requireNonNull(event, "Event must be non-null"); - if (checkExists && !tracked.containsKey(event)) - throw new IllegalArgumentException("The event " + event + " was not previously added; cannot " + verb); - - tracked.put(event, paused); + if (modification == Modification.ADD) { + if (tracked.containsKey(event)) + throw new IllegalArgumentException("The event " + event + " was previously added, so it cannot be " + modification.verb + " again"); + } else if (!tracked.containsKey(event)) { + throw new IllegalArgumentException("The event " + event + " was not previously added, so it cannot be " + modification.verb); + } - log.debug("Event {} was {}-d with paused set to {}", event, verb, paused); + tracked.put(event, modification.state); + log.trace("Event {} was {} and is now in state {}", event, modification.verb, modification.state); } /** @@ -117,10 +140,10 @@ public long reap(long currentTimeMs) { int count = 0; List> unpausedEvents = new ArrayList<>(tracked.size()); - for (Map.Entry, Boolean> entry : tracked.entrySet()) { - boolean isPaused = entry.getValue(); + for (Map.Entry, State> entry : tracked.entrySet()) { + State state = entry.getValue(); - if (isPaused) { + if (state == State.PAUSED) { // Don't reap "paused" events continue; } @@ -203,7 +226,7 @@ public List> uncompletedEvents() { // path of the ConsumerNetworkThread loop. List> events = new ArrayList<>(); - for (Map.Entry, Boolean> entry : tracked.entrySet()) { + for (Map.Entry, State> entry : tracked.entrySet()) { CompletableEvent event = entry.getKey(); if (!event.future().isDone()) From 3e0b9203991bcdad2816835885daa97ec0e6aacd Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 17 Sep 2025 20:49:53 -0700 Subject: [PATCH 012/123] Refactor CompositePollEvent to use Blocker for state management Replaces CompletableFuture-based state handling in CompositePollEvent with a new Blocker class for improved synchronization and exception handling. Updates AsyncKafkaConsumer, WakeupTrigger, ApplicationEventProcessor, and related tests to use Blocker, simplifying event completion and error propagation. --- .../internals/AsyncKafkaConsumer.java | 27 ++- .../clients/consumer/internals/Blocker.java | 157 ++++++++++++++++++ .../consumer/internals/WakeupTrigger.java | 39 ++++- .../events/ApplicationEventProcessor.java | 20 ++- .../internals/events/CompositePollEvent.java | 20 ++- .../internals/AsyncKafkaConsumerTest.java | 22 ++- 6 files changed, 259 insertions(+), 26 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index bd4a62c85d518..cf8dbdaab152a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -882,7 +882,7 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); prepareFetch(timer); - final Fetch fetch = collectFetch(); + final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches // and avoid block waiting for their responses to enable pipelining while the user @@ -914,31 +914,42 @@ private void prepareFetch(Timer timer) { long deadlineMs = calculateDeadlineMs(timer); ApplicationEvent.Type nextEventType = ApplicationEvent.Type.POLL; + log.debug("prepareFetch - timer: {}", timer.remainingMs()); + + Timer blockerTimer = time.timer(defaultApiTimeoutMs.toMillis()); + while (true) { CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); + applicationEventHandler.add(event); + CompositePollEvent.State state; + wakeupTrigger.setFetchAction(event); try { - state = applicationEventHandler.addAndGet(event); + state = event.blocker().await(blockerTimer); } catch (TimeoutException e) { // Timeouts are OK, there's just no data to return on this pass. - break; + return; + } catch (InterruptException e) { + log.trace("Interrupt during composite poll", e); + throw e; + } finally { + timer.update(blockerTimer.currentTimeMs()); + wakeupTrigger.clearTask(); } - if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { + if (state == null || state == CompositePollEvent.State.COMPLETE) { + break; + } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { offsetCommitCallbackInvoker.executeCallbacks(); nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { processBackgroundEvents(); nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - } else if (state == CompositePollEvent.State.COMPLETE) { - break; } else { throw new IllegalStateException("Unexpected state: " + state); } } - - timer.update(); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java new file mode 100644 index 0000000000000..b1f61990b428e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java @@ -0,0 +1,157 @@ +/* + * 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; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class Blocker { + + private final Logger log; + private final Lock lock; + private final Condition condition; + private final AtomicBoolean wokenup = new AtomicBoolean(false); + private T value; + private KafkaException error; + + public Blocker() { + this.log = LoggerFactory.getLogger(getClass()); + this.lock = new ReentrantLock(); + this.condition = lock.newCondition(); + } + + public boolean complete(T value) { + Objects.requireNonNull(value); + + try { + lock.lock(); + + if (isSet()) + return false; + + log.debug("Setting value to {}", value); + this.value = value; + wokenup.set(true); + condition.signalAll(); + return true; + } finally { + lock.unlock(); + } + } + + public boolean completeExceptionally(KafkaException error) { + Objects.requireNonNull(error); + + try { + lock.lock(); + + if (isSet()) + return false; + + log.debug("Setting exception to {}", String.valueOf(error)); + this.error = error; + wokenup.set(true); + condition.signalAll(); + return true; + } finally { + lock.unlock(); + } + } + + private boolean isSet() { + return error != null || value != null; + } + + /** + * Allows the caller to await a response from the broker for requested data. The method will block, returning only + * under one of the following conditions: + * + *
    + *
  1. The buffer was already woken
  2. + *
  3. The buffer was woken during the wait
  4. + *
  5. The remaining time on the {@link Timer timer} elapsed
  6. + *
  7. The thread was interrupted
  8. + *
+ * + * @param timer Timer that provides time to wait + */ + public T await(Timer timer) { + try { + lock.lock(); + + log.debug("At start of method, error: {}, value: {}", error, value); + + if (error != null) + throw error; + else if (value != null) + return value; + + while (!wokenup.compareAndSet(true, false)) { + // Update the timer before we head into the loop in case it took a while to get the lock. + timer.update(); + + if (timer.isExpired()) { + // If the thread was interrupted before we start waiting, it still counts as + // interrupted from the point of view of the KafkaConsumer.poll(Duration) contract. + // We only need to check this when we are not going to wait because waiting + // already checks whether the thread is interrupted. + if (Thread.interrupted()) + throw error = new InterruptException("Interrupted waiting for completion"); + + break; + } + + if (!condition.await(timer.remainingMs(), TimeUnit.MILLISECONDS)) { + break; + } + } + + log.debug("At end of method, error: {}, value: {}", error, value); + + if (error != null) + throw error; + else if (value != null) + return value; + + throw error = new TimeoutException("Timed out waiting for completion"); + } catch (InterruptedException e) { + throw new InterruptException("Interrupted waiting for completion", e); + } finally { + lock.unlock(); + timer.update(); + } + } + + @Override + public String toString() { + return "Blocker{" + + "value=" + value + + ", error=" + error + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java index 7893cf29f23bb..b7873f56ebf76 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.WakeupException; @@ -49,6 +50,10 @@ public void wakeup() { // will be ignored. If it was already completed, we then need to return a new WakeupFuture so that the // next call to setActiveTask will throw the WakeupException. return wasTriggered ? null : new WakeupFuture(); + } else if (task instanceof CompositePollEventAction) { + CompositePollEventAction compositePollEventAction = (CompositePollEventAction) task; + compositePollEventAction.event().blocker().completeExceptionally(new WakeupException()); + return new WakeupFuture(); } else if (task instanceof FetchAction) { FetchAction fetchAction = (FetchAction) task; fetchAction.fetchBuffer().wakeup(); @@ -89,6 +94,25 @@ public CompletableFuture setActiveTask(final CompletableFuture current return currentTask; } + public void setFetchAction(final CompositePollEvent event) { + final AtomicBoolean throwWakeupException = new AtomicBoolean(false); + pendingTask.getAndUpdate(task -> { + if (task == null) { + return new CompositePollEventAction(event); + } else if (task instanceof WakeupFuture) { + throwWakeupException.set(true); + return null; + } else if (task instanceof DisabledWakeups) { + return task; + } + // last active state is still active + throw new IllegalStateException("Last active task is still active"); + }); + if (throwWakeupException.get()) { + throw new WakeupException(); + } + } + public void setFetchAction(final FetchBuffer fetchBuffer) { final AtomicBoolean throwWakeupException = new AtomicBoolean(false); pendingTask.getAndUpdate(task -> { @@ -135,7 +159,7 @@ public void clearTask() { pendingTask.getAndUpdate(task -> { if (task == null) { return null; - } else if (task instanceof ActiveFuture || task instanceof FetchAction || task instanceof ShareFetchAction) { + } else if (task instanceof ActiveFuture || task instanceof CompositePollEventAction || task instanceof FetchAction || task instanceof ShareFetchAction) { return null; } return task; @@ -182,6 +206,19 @@ public CompletableFuture future() { static class WakeupFuture implements Wakeupable { } + static class CompositePollEventAction implements Wakeupable { + + private final CompositePollEvent event; + + public CompositePollEventAction(CompositePollEvent event) { + this.event = event; + } + + public CompositePollEvent event() { + return event; + } + } + static class FetchAction implements Wakeupable { private final FetchBuffer fetchBuffer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 358071d8dcde1..b69c19a87ae6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; @@ -48,6 +49,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -292,14 +294,12 @@ else if (maybePauseCompositePoll(event, backgroundEventProcessingRequiredTest)) if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { // This is a bit tricky... The CompositePollEvent should be "paused" from being reaped while the code // for new CheckAndUpdatePositionsEvent is in flight. - applicationEventReaper.pause(event); CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { // Make sure to resume the CompositePollEvent *before* checking for failure so that it is assured // to be resumed. - applicationEventReaper.resume(event); if (maybeFailCompositePoll(event, updatePositionsError)) return; @@ -309,15 +309,15 @@ else if (maybePauseCompositePoll(event, backgroundEventProcessingRequiredTest)) if (maybeFailCompositePoll(event, fetchError)) return; - log.trace("Completing CompositePollEvent {}", event); - event.future().complete(CompositePollEvent.State.COMPLETE); + event.blocker().complete(CompositePollEvent.State.COMPLETE); + log.trace("Completed CompositePollEvent {}", event); }); }); return; } - event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextEventType)); + event.blocker().completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); } private boolean maybePauseCompositePoll(CompositePollEvent event, RequiresApplicationThreadExecution test) { @@ -325,8 +325,8 @@ private boolean maybePauseCompositePoll(CompositePollEvent event, RequiresApplic return false; CompositePollEvent.State targetState = test.targetState(); + event.blocker().complete(targetState); log.trace("Pausing CompositePollEvent {} to process logic for target state {}", event, targetState); - event.future().complete(targetState); return true; } @@ -342,8 +342,12 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { return false; } + if (t instanceof CompletionException) { + t = t.getCause(); + } + + event.blocker().completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(t)); log.trace("Failing CompositePollEvent {}", event, t); - event.future().completeExceptionally(t); return true; } @@ -353,7 +357,7 @@ private boolean maybeFailCompositePoll(CompositePollEvent event) { if (exception.isPresent()) { Exception e = exception.get(); log.trace("Failing CompositePollEvent {} with error from NetworkClient", event, e); - event.future().completeExceptionally(e); + event.blocker().completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(e)); return true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index a54e70bb78bfa..004607f129e3a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class CompositePollEvent extends CompletableApplicationEvent { +import org.apache.kafka.clients.consumer.internals.Blocker; + +public class CompositePollEvent extends ApplicationEvent { public enum State { @@ -25,13 +27,21 @@ public enum State { COMPLETE } + private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; + private final Blocker blocker; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { - super(Type.COMPOSITE_POLL, deadlineMs); + super(Type.COMPOSITE_POLL); + this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; + this.blocker = new Blocker<>(); + } + + public long deadlineMs() { + return deadlineMs; } public long pollTimeMs() { @@ -42,8 +52,12 @@ public Type nextEventType() { return nextEventType; } + public Blocker blocker() { + return blocker; + } + @Override protected String toStringBase() { - return super.toStringBase() + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", blocker=" + blocker; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index b27eb18f59736..3734e2c2e5f92 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -102,6 +102,8 @@ import org.mockito.ArgumentMatchers; import org.mockito.MockedStatic; import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Duration; import java.util.Arrays; @@ -169,6 +171,7 @@ @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { + private static final Logger log = LoggerFactory.getLogger(AsyncKafkaConsumerTest.class); private AsyncKafkaConsumer consumer = null; private Time time = new MockTime(0); private final Metrics metrics = new Metrics(); @@ -1677,7 +1680,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { markReconcileAndAutoCommitCompleteForPollEvent(); markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).addAndGet(any(CompositePollEvent.class)); + verify(applicationEventHandler).add(any(CompositePollEvent.class)); } private Properties requiredConsumerConfigAndGroupId(final String groupId) { @@ -2255,26 +2258,33 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { private void markResultForCompositePollEvent(CompositePollEvent.State state) { doAnswer(invocation -> { + CompositePollEvent event = invocation.getArgument(0); + log.error("Am I invoked: {}", event); + if (Thread.currentThread().isInterrupted()) - throw new InterruptException("Test interrupt"); + event.blocker().completeExceptionally(new InterruptException("Test interrupt")); + event.blocker().complete(state); return state; - }).when(applicationEventHandler).addAndGet(ArgumentMatchers.isA(CompositePollEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } private void markResultForCompositePollEvent(Collection states) { LinkedList statesQueue = new LinkedList<>(states); doAnswer(invocation -> { + CompositePollEvent event = invocation.getArgument(0); + log.error("Am I invoked: {}", event); CompositePollEvent.State state = statesQueue.poll(); if (state == null) - throw new IllegalStateException("The array of " + CompositePollEvent.State.class.getSimpleName() + " did not provide enough values"); + event.blocker().completeExceptionally(new KafkaException("The array of " + CompositePollEvent.State.class.getSimpleName() + " did not provide enough values")); if (Thread.currentThread().isInterrupted()) - throw new InterruptException("Test interrupt"); + event.blocker().completeExceptionally(new InterruptException("Test interrupt")); + event.blocker().complete(state); return state; - }).when(applicationEventHandler).addAndGet(ArgumentMatchers.isA(CompositePollEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } } From 529aab33166ca37c4046e32a20239dbb198b1632 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 18 Sep 2025 08:51:26 -0700 Subject: [PATCH 013/123] Update AsyncKafkaConsumer.java --- .../consumer/internals/AsyncKafkaConsumer.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cf8dbdaab152a..d6bc3f217fb52 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -882,7 +882,7 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); prepareFetch(timer); - final Fetch fetch = pollForFetches(timer); + final Fetch fetch = collectFetch(); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches // and avoid block waiting for their responses to enable pipelining while the user @@ -916,8 +916,6 @@ private void prepareFetch(Timer timer) { log.debug("prepareFetch - timer: {}", timer.remainingMs()); - Timer blockerTimer = time.timer(defaultApiTimeoutMs.toMillis()); - while (true) { CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); applicationEventHandler.add(event); @@ -926,6 +924,7 @@ private void prepareFetch(Timer timer) { wakeupTrigger.setFetchAction(event); try { + Timer blockerTimer = time.timer(defaultApiTimeoutMs.toMillis()); state = event.blocker().await(blockerTimer); } catch (TimeoutException e) { // Timeouts are OK, there's just no data to return on this pass. @@ -934,7 +933,7 @@ private void prepareFetch(Timer timer) { log.trace("Interrupt during composite poll", e); throw e; } finally { - timer.update(blockerTimer.currentTimeMs()); + timer.update(); wakeupTrigger.clearTask(); } @@ -1879,12 +1878,7 @@ private Fetch pollForFetches(Timer timer) { * done as an optimization so that the next round of data can be pre-fetched. */ private Fetch collectFetch() { - final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); - - // Notify the network thread to wake up and start the next round of fetching. - applicationEventHandler.wakeupNetworkThread(); - - return fetch; + return fetchCollector.collectFetch(fetchBuffer); } /** From 784aad2d4cde19f2041f96070ec197153974a6b2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 19 Sep 2025 11:22:42 -0700 Subject: [PATCH 014/123] Moving toward a non-blocking poll() implementation --- .../internals/AsyncKafkaConsumer.java | 99 ++--------- .../clients/consumer/internals/Blocker.java | 157 ------------------ .../internals/ConsumerNetworkThread.java | 24 --- .../internals/NetworkClientDelegate.java | 22 +-- .../internals/OffsetsRequestManager.java | 53 +++--- .../consumer/internals/RequestManagers.java | 1 + .../consumer/internals/ShareConsumerImpl.java | 3 +- .../consumer/internals/WakeupTrigger.java | 39 +---- .../events/ApplicationEventProcessor.java | 82 ++------- .../events/CompletableEventReaper.java | 102 ++---------- .../internals/events/CompositePollEvent.java | 10 +- .../internals/AsyncKafkaConsumerTest.java | 90 ++-------- .../internals/FetchRequestManagerTest.java | 7 +- .../internals/NetworkClientDelegateTest.java | 45 ++--- .../internals/OffsetsRequestManagerTest.java | 5 + .../ShareConsumeRequestManagerTest.java | 7 +- 16 files changed, 133 insertions(+), 613 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index d6bc3f217fb52..c6bdffed9a6f8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -362,8 +362,6 @@ private StreamsRebalanceListener streamsRebalanceListener() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates - private boolean cachedSubscriptionHasAllFetchPositions; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -475,7 +473,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, fetchMetricsManager.throttleTimeSensor(), clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, - false, asyncConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); @@ -669,7 +666,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, client, metadata, backgroundEventHandler, - false, asyncConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); @@ -882,7 +878,7 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); prepareFetch(timer); - final Fetch fetch = collectFetch(); + final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches // and avoid block waiting for their responses to enable pipelining while the user @@ -916,39 +912,11 @@ private void prepareFetch(Timer timer) { log.debug("prepareFetch - timer: {}", timer.remainingMs()); - while (true) { - CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); - applicationEventHandler.add(event); - - CompositePollEvent.State state; - wakeupTrigger.setFetchAction(event); - - try { - Timer blockerTimer = time.timer(defaultApiTimeoutMs.toMillis()); - state = event.blocker().await(blockerTimer); - } catch (TimeoutException e) { - // Timeouts are OK, there's just no data to return on this pass. - return; - } catch (InterruptException e) { - log.trace("Interrupt during composite poll", e); - throw e; - } finally { - timer.update(); - wakeupTrigger.clearTask(); - } + processBackgroundEvents(); + offsetCommitCallbackInvoker.executeCallbacks(); - if (state == null || state == CompositePollEvent.State.COMPLETE) { - break; - } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { - offsetCommitCallbackInvoker.executeCallbacks(); - nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { - processBackgroundEvents(); - nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - } else { - throw new IllegalStateException("Unexpected state: " + state); - } - } + CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); + applicationEventHandler.add(event); } /** @@ -1191,6 +1159,7 @@ public List partitionsFor(String topic, Duration timeout) { Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); + processBackgroundEvents(); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1216,7 +1185,9 @@ public Map> listTopics(Duration timeout) { final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataEvent); + Map> map = applicationEventHandler.addAndGet(topicMetadataEvent); + processBackgroundEvents(); + return map; } finally { wakeupTrigger.clearTask(); } @@ -1298,6 +1269,7 @@ public Map offsetsForTimes(Map offsets = applicationEventHandler.addAndGet(listOffsetsEvent); + processBackgroundEvents(); Map results = new HashMap<>(offsets.size()); offsets.forEach((k, v) -> results.put(k, v != null ? v.buildOffsetAndTimestamp() : null)); return results; @@ -1363,6 +1335,7 @@ private Map beginningOrEndOffset(Collection offsetAndTimestampMap; try { offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); + processBackgroundEvents(); return offsetAndTimestampMap.entrySet() .stream() .collect(Collectors.toMap( @@ -1833,18 +1806,6 @@ private Fetch pollForFetches(Timer timer) { return fetch; } - // send any new fetches (won't resend pending fetches) - sendFetches(timer); - - // 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 - - // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call - // updateAssignmentMetadataIfNeeded before this method. - if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > retryBackoffMs) { - pollTimeout = retryBackoffMs; - } - log.trace("Polling for fetches with timeout {}", pollTimeout); Timer pollTimer = time.timer(pollTimeout); @@ -1891,11 +1852,10 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { - cachedSubscriptionHasAllFetchPositions = false; try { CheckAndUpdatePositionsEvent checkAndUpdatePositionsEvent = new CheckAndUpdatePositionsEvent(calculateDeadlineMs(timer)); wakeupTrigger.setActiveTask(checkAndUpdatePositionsEvent.future()); - cachedSubscriptionHasAllFetchPositions = applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent); + applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent); } catch (TimeoutException e) { return false; } finally { @@ -1913,41 +1873,6 @@ private boolean isCommittedOffsetsManagementEnabled() { return groupMetadata.get().isPresent(); } - /** - * This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests}. - * - *

- * - * This method takes the following steps to maintain compatibility with the {@link ClassicKafkaConsumer} method - * of the same name: - * - *

    - *
  • - * The method will wait for confirmation of the request creation before continuing. - *
  • - *
  • - * The method will throw exceptions encountered during request creation to the user immediately. - *
  • - *
  • - * The method will suppress {@link TimeoutException}s that occur while waiting for the confirmation. - * Timeouts during request creation are a byproduct of this consumer's thread communication mechanisms. - * That exception type isn't thrown in the request creation step of the {@link ClassicKafkaConsumer}. - * Additionally, timeouts will not impact the logic of {@link #pollForFetches(Timer) blocking requests} - * as it can handle requests that are created after the timeout. - *
  • - *
- * - * @param timer Timer used to bound how long the consumer waits for the requests to be created, which in practice - * is used to avoid using {@link Long#MAX_VALUE} to wait "forever" - */ - private void sendFetches(Timer timer) { - try { - applicationEventHandler.addAndGet(new CreateFetchRequestsEvent(calculateDeadlineMs(timer))); - } catch (TimeoutException swallow) { - // Can be ignored, per above comments. - } - } - /** * This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests} for the * pre-fetch case, i.e. right before {@link #poll(Duration)} exits. In the pre-fetch case, the application thread diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java index b1f61990b428e..e69de29bb2d1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java @@ -1,157 +0,0 @@ -/* - * 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; - -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.errors.InterruptException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.utils.Timer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Objects; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -public class Blocker { - - private final Logger log; - private final Lock lock; - private final Condition condition; - private final AtomicBoolean wokenup = new AtomicBoolean(false); - private T value; - private KafkaException error; - - public Blocker() { - this.log = LoggerFactory.getLogger(getClass()); - this.lock = new ReentrantLock(); - this.condition = lock.newCondition(); - } - - public boolean complete(T value) { - Objects.requireNonNull(value); - - try { - lock.lock(); - - if (isSet()) - return false; - - log.debug("Setting value to {}", value); - this.value = value; - wokenup.set(true); - condition.signalAll(); - return true; - } finally { - lock.unlock(); - } - } - - public boolean completeExceptionally(KafkaException error) { - Objects.requireNonNull(error); - - try { - lock.lock(); - - if (isSet()) - return false; - - log.debug("Setting exception to {}", String.valueOf(error)); - this.error = error; - wokenup.set(true); - condition.signalAll(); - return true; - } finally { - lock.unlock(); - } - } - - private boolean isSet() { - return error != null || value != null; - } - - /** - * Allows the caller to await a response from the broker for requested data. The method will block, returning only - * under one of the following conditions: - * - *
    - *
  1. The buffer was already woken
  2. - *
  3. The buffer was woken during the wait
  4. - *
  5. The remaining time on the {@link Timer timer} elapsed
  6. - *
  7. The thread was interrupted
  8. - *
- * - * @param timer Timer that provides time to wait - */ - public T await(Timer timer) { - try { - lock.lock(); - - log.debug("At start of method, error: {}, value: {}", error, value); - - if (error != null) - throw error; - else if (value != null) - return value; - - while (!wokenup.compareAndSet(true, false)) { - // Update the timer before we head into the loop in case it took a while to get the lock. - timer.update(); - - if (timer.isExpired()) { - // If the thread was interrupted before we start waiting, it still counts as - // interrupted from the point of view of the KafkaConsumer.poll(Duration) contract. - // We only need to check this when we are not going to wait because waiting - // already checks whether the thread is interrupted. - if (Thread.interrupted()) - throw error = new InterruptException("Interrupted waiting for completion"); - - break; - } - - if (!condition.await(timer.remainingMs(), TimeUnit.MILLISECONDS)) { - break; - } - } - - log.debug("At end of method, error: {}, value: {}", error, value); - - if (error != null) - throw error; - else if (value != null) - return value; - - throw error = new TimeoutException("Timed out waiting for completion"); - } catch (InterruptedException e) { - throw new InterruptException("Interrupted waiting for completion", e); - } finally { - lock.unlock(); - timer.update(); - } - } - - @Override - public String toString() { - return "Blocker{" + - "value=" + value + - ", error=" + error + - '}'; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index d2d178a88c38b..e4f6958a0cb40 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -173,8 +173,6 @@ void runOnce() { cachedMaximumTimeToWait = maxTimeToWaitMs; reapExpiredApplicationEvents(currentTimeMs); - List> uncompletedEvents = applicationEventReaper.uncompletedEvents(); - maybeFailOnMetadataError(uncompletedEvents); } /** @@ -193,10 +191,6 @@ private void processApplicationEvents() { try { if (event instanceof CompletableEvent) { applicationEventReaper.add((CompletableEvent) event); - // Check if there are any metadata errors and fail the CompletableEvent if an error is present. - // This call is meant to handle "immediately completed events" which may not enter the awaiting state, - // so metadata errors need to be checked and handled right away. - maybeFailOnMetadataError(List.of((CompletableEvent) event)); } applicationEventProcessor.process(event); } catch (Throwable t) { @@ -364,22 +358,4 @@ void cleanup() { log.debug("Closed the consumer network thread"); } } - - /** - * If there is a metadata error, complete all uncompleted events that require subscription metadata. - */ - private void maybeFailOnMetadataError(List> events) { - List> subscriptionMetadataEvent = new ArrayList<>(); - - for (CompletableEvent ce : events) { - if (ce instanceof CompletableApplicationEvent && ((CompletableApplicationEvent) ce).requireSubscriptionMetadata()) - subscriptionMetadataEvent.add((CompletableApplicationEvent) ce); - } - - if (subscriptionMetadataEvent.isEmpty()) - return; - networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError -> - subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) - ); - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 374ff96fd5e8b..1cb25bb46e069 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -70,8 +70,6 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private Optional metadataError; - private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; public NetworkClientDelegate( @@ -81,7 +79,6 @@ public NetworkClientDelegate( final KafkaClient client, final Metadata metadata, final BackgroundEventHandler backgroundEventHandler, - final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { this.time = time; this.client = client; @@ -91,8 +88,6 @@ public NetworkClientDelegate( this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - this.metadataError = Optional.empty(); - this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -160,11 +155,7 @@ private void maybePropagateMetadataError() { try { metadata.maybeThrowAnyException(); } catch (Exception e) { - if (notifyMetadataErrorsViaErrorQueue) { - backgroundEventHandler.add(new ErrorEvent(e)); - } else { - metadataError = Optional.of(e); - } + backgroundEventHandler.add(new ErrorEvent(e)); } } @@ -248,12 +239,6 @@ private ClientRequest makeClientRequest( ); } - public Optional getAndClearMetadataError() { - Optional metadataError = this.metadataError; - this.metadataError = Optional.empty(); - return metadataError; - } - public Node leastLoadedNode() { return this.client.leastLoadedNode(time.milliseconds()).node(); } @@ -452,7 +437,6 @@ public static Supplier supplier(final Time time, final Sensor throttleTimeSensor, final ClientTelemetrySender clientTelemetrySender, final BackgroundEventHandler backgroundEventHandler, - final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { return new CachedSupplier<>() { @Override @@ -467,7 +451,7 @@ protected NetworkClientDelegate create() { metadata, throttleTimeSensor, clientTelemetrySender); - return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics); + return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, asyncConsumerMetrics); } }; } @@ -482,7 +466,6 @@ public static Supplier supplier(final Time time, final KafkaClient client, final Metadata metadata, final BackgroundEventHandler backgroundEventHandler, - final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { return new CachedSupplier<>() { @Override @@ -494,7 +477,6 @@ protected NetworkClientDelegate create() { client, metadata, backgroundEventHandler, - notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics ); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 4c8d10ad323ac..7fd588a7648ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -26,6 +26,8 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -53,8 +55,8 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -84,6 +86,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final Logger log; private final OffsetFetcherUtils offsetFetcherUtils; private final SubscriptionState subscriptionState; + private final BackgroundEventHandler backgroundEventHandler; private final Set requestsToRetry; private final List requestsToSend; @@ -94,12 +97,6 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; - /** - * Exception that occurred while updating positions after the triggering event had already - * expired. It will be propagated and cleared on the next call to update fetch positions. - */ - private final AtomicReference cachedUpdatePositionsException = new AtomicReference<>(); - /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update * fetch positions that hasn't completed yet. When a response is received, it's used to @@ -111,6 +108,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou public OffsetsRequestManager(final SubscriptionState subscriptionState, final ConsumerMetadata metadata, + final BackgroundEventHandler backgroundEventHandler, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, @@ -122,6 +120,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); + requireNonNull(backgroundEventHandler); requireNonNull(isolationLevel); requireNonNull(time); requireNonNull(apiVersions); @@ -134,6 +133,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToRetry = new HashSet<>(); this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; + this.backgroundEventHandler = backgroundEventHandler; this.time = time; this.requestTimeoutMs = requestTimeoutMs; this.defaultApiTimeoutMs = defaultApiTimeoutMs; @@ -235,10 +235,6 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { CompletableFuture result = new CompletableFuture<>(); try { - if (maybeCompleteWithPreviousException(result)) { - return result; - } - validatePositionsIfNeeded(); if (subscriptionState.hasAllFetchPositions()) { @@ -262,15 +258,6 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { return result; } - private boolean maybeCompleteWithPreviousException(CompletableFuture result) { - Throwable cachedException = cachedUpdatePositionsException.getAndSet(null); - if (cachedException != null) { - result.completeExceptionally(cachedException); - return true; - } - return false; - } - /** * Generate requests to fetch offsets and update positions once a response is received. This will first attempt * to use the committed offsets if available. If no committed offsets available, it will use the partition @@ -318,7 +305,12 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - cachedUpdatePositionsException.set(error); + log.debug("Adding error: {}", error.getClass()); + + if (error instanceof CompletionException) + error = error.getCause(); + + backgroundEventHandler.add(new ErrorEvent(error)); } }); } @@ -334,12 +326,19 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d * @throws NoOffsetForPartitionException If no reset strategy is configured. */ private CompletableFuture initWithPartitionOffsetsIfNeeded(Set initializingPartitions) { + log.debug("initWithPartitionOffsetsIfNeeded - initializingPartitions: {}", initializingPartitions); CompletableFuture result = new CompletableFuture<>(); try { // Mark partitions that need reset, using the configured reset strategy. If no // strategy is defined, this will raise a NoOffsetForPartitionException exception. subscriptionState.resetInitializingPositions(initializingPartitions::contains); } catch (Exception e) { + Throwable t = e; + + if (t instanceof CompletionException) + t = t.getCause(); + + backgroundEventHandler.add(new ErrorEvent(t)); result.completeExceptionally(e); return result; } @@ -366,7 +365,7 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set result = new CompletableFuture<>(); // The shorter the timeout provided to poll(), the more likely the offsets fetch will time out. To handle @@ -374,6 +373,8 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set> fetchOffsets = @@ -409,11 +410,14 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set offsets, final Throwable error, final CompletableFuture result) { + log.debug("refreshOffsets - offsets: {}, partitions: {}", offsets, String.valueOf(error)); + if (error == null) { // Ensure we only set positions for the partitions that still require one (ex. some partitions may have // been assigned a position manually) Map offsetsToApply = offsetsForInitializingPartitions(offsets); + log.debug("refreshOffsets - offsetsToApply: {}", offsetsToApply); refreshCommittedOffsets(offsetsToApply, metadata, subscriptionState); @@ -458,6 +462,7 @@ private boolean canReusePendingOffsetFetchEvent(Set partitions) return false; } + log.debug("canReusePendingOffsetFetchEvent - pendingOffsetFetchEvent.requestedPartitions: {}, partitions: {}", pendingOffsetFetchEvent.requestedPartitions, partitions); return pendingOffsetFetchEvent.requestedPartitions.equals(partitions); } @@ -472,11 +477,15 @@ private boolean canReusePendingOffsetFetchEvent(Set partitions) * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ CompletableFuture resetPositionsIfNeeded() { + log.debug("resetPositionsIfNeeded"); Map partitionAutoOffsetResetStrategyMap; try { partitionAutoOffsetResetStrategyMap = offsetFetcherUtils.getOffsetResetStrategyForPartitions(); } catch (Exception e) { + log.debug("resetPositionsIfNeeded - e: {}", e.toString()); + + backgroundEventHandler.add(new ErrorEvent(e.getCause())); CompletableFuture result = new CompletableFuture<>(); result.completeExceptionally(e); return result; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index ae39753f3d8e8..3571726216e8e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -284,6 +284,7 @@ protected RequestManagers create() { final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions, metadata, + backgroundEventHandler, fetchConfig.isolationLevel, time, retryBackoffMs, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 08767c397e41d..a5f2066afc16c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -279,7 +279,6 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { shareFetchMetricsManager.throttleTimeSensor(), clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, - true, asyncConsumerMetrics ); this.completedAcknowledgements = new LinkedList<>(); @@ -389,7 +388,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { backgroundEventQueue, time, asyncConsumerMetrics); final Supplier networkClientDelegateSupplier = - NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics); + NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, asyncConsumerMetrics); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java index b7873f56ebf76..7893cf29f23bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.WakeupException; @@ -50,10 +49,6 @@ public void wakeup() { // will be ignored. If it was already completed, we then need to return a new WakeupFuture so that the // next call to setActiveTask will throw the WakeupException. return wasTriggered ? null : new WakeupFuture(); - } else if (task instanceof CompositePollEventAction) { - CompositePollEventAction compositePollEventAction = (CompositePollEventAction) task; - compositePollEventAction.event().blocker().completeExceptionally(new WakeupException()); - return new WakeupFuture(); } else if (task instanceof FetchAction) { FetchAction fetchAction = (FetchAction) task; fetchAction.fetchBuffer().wakeup(); @@ -94,25 +89,6 @@ public CompletableFuture setActiveTask(final CompletableFuture current return currentTask; } - public void setFetchAction(final CompositePollEvent event) { - final AtomicBoolean throwWakeupException = new AtomicBoolean(false); - pendingTask.getAndUpdate(task -> { - if (task == null) { - return new CompositePollEventAction(event); - } else if (task instanceof WakeupFuture) { - throwWakeupException.set(true); - return null; - } else if (task instanceof DisabledWakeups) { - return task; - } - // last active state is still active - throw new IllegalStateException("Last active task is still active"); - }); - if (throwWakeupException.get()) { - throw new WakeupException(); - } - } - public void setFetchAction(final FetchBuffer fetchBuffer) { final AtomicBoolean throwWakeupException = new AtomicBoolean(false); pendingTask.getAndUpdate(task -> { @@ -159,7 +135,7 @@ public void clearTask() { pendingTask.getAndUpdate(task -> { if (task == null) { return null; - } else if (task instanceof ActiveFuture || task instanceof CompositePollEventAction || task instanceof FetchAction || task instanceof ShareFetchAction) { + } else if (task instanceof ActiveFuture || task instanceof FetchAction || task instanceof ShareFetchAction) { return null; } return task; @@ -206,19 +182,6 @@ public CompletableFuture future() { static class WakeupFuture implements Wakeupable { } - static class CompositePollEventAction implements Wakeupable { - - private final CompositePollEvent event; - - public CompositePollEventAction(CompositePollEvent event) { - this.event = event; - } - - public CompositePollEvent event() { - return event; - } - } - static class FetchAction implements Wakeupable { private final FetchBuffer fetchBuffer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index b69c19a87ae6c..3c92d938142f8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -69,6 +69,7 @@ public class ApplicationEventProcessor implements EventProcessor 0; - } - - @Override - public CompositePollEvent.State targetState() { - return CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED; - } - }; + this.backgroundEventProcessingRequiredTest = () -> backgroundEventHandler.size() > 0; // If there are enqueued callbacks to invoke, exit to the application thread. - this.offsetCommitCallbackInvocationRequiredTest = new RequiresApplicationThreadExecution() { - @Override - public boolean requiresApplicationThread() { - return offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; - } - - @Override - public CompositePollEvent.State targetState() { - return CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED; - } - }; + this.offsetCommitCallbackInvocationRequiredTest = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; } @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @@ -264,7 +246,7 @@ public void process(ApplicationEvent event) { } private void process(final CompositePollEvent event) { - if (maybeFailCompositePoll(event)) + if (maybePauseCompositePoll()) return; ApplicationEvent.Type nextEventType = event.nextEventType(); @@ -272,9 +254,7 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.POLL) { processPollEvent(event.pollTimeMs()); - if (maybeFailCompositePoll(event)) - return; - else if (maybePauseCompositePoll(event, offsetCommitCallbackInvocationRequiredTest)) + if (maybePauseCompositePoll()) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; @@ -283,33 +263,25 @@ else if (maybePauseCompositePoll(event, offsetCommitCallbackInvocationRequiredTe if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { processUpdatePatternSubscriptionEvent(); - if (maybeFailCompositePoll(event)) - return; - else if (maybePauseCompositePoll(event, backgroundEventProcessingRequiredTest)) + if (maybePauseCompositePoll()) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; } if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - // This is a bit tricky... The CompositePollEvent should be "paused" from being reaped while the code - // for new CheckAndUpdatePositionsEvent is in flight. CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - // Make sure to resume the CompositePollEvent *before* checking for failure so that it is assured - // to be resumed. - - if (maybeFailCompositePoll(event, updatePositionsError)) + if (maybeFailCompositePoll(event, updatePositionsError) || maybePauseCompositePoll()) return; // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event, fetchError)) + if (maybeFailCompositePoll(event, fetchError) || maybePauseCompositePoll()) return; - event.blocker().complete(CompositePollEvent.State.COMPLETE); log.trace("Completed CompositePollEvent {}", event); }); }); @@ -317,23 +289,20 @@ else if (maybePauseCompositePoll(event, backgroundEventProcessingRequiredTest)) return; } - event.blocker().completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); + log.warn("Unknown next step for composite poll: {}", nextEventType); } - private boolean maybePauseCompositePoll(CompositePollEvent event, RequiresApplicationThreadExecution test) { - if (!test.requiresApplicationThread()) - return false; + private boolean maybePauseCompositePoll() { + if (backgroundEventProcessingRequiredTest.requiresApplicationThread()) + return true; - CompositePollEvent.State targetState = test.targetState(); - event.blocker().complete(targetState); - log.trace("Pausing CompositePollEvent {} to process logic for target state {}", event, targetState); - return true; + if (offsetCommitCallbackInvocationRequiredTest.requiresApplicationThread()) + return true; + + return false; } private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { - if (maybeFailCompositePoll(event)) - return true; - if (t == null) return false; @@ -346,24 +315,11 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { t = t.getCause(); } - event.blocker().completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(t)); + backgroundEventHandler.add(new ErrorEvent(t)); log.trace("Failing CompositePollEvent {}", event, t); return true; } - private boolean maybeFailCompositePoll(CompositePollEvent event) { - Optional exception = networkClientDelegate.getAndClearMetadataError(); - - if (exception.isPresent()) { - Exception e = exception.get(); - log.trace("Failing CompositePollEvent {} with error from NetworkClient", event, e); - event.blocker().completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(e)); - return true; - } - - return false; - } - private void process(final PollEvent event) { processPollEvent(event.pollTimeMs()); event.markReconcileAndAutoCommitComplete(); @@ -962,8 +918,6 @@ private CompletableFuture processCheckAndUpdatePositionsEvent(final lon private interface RequiresApplicationThreadExecution { boolean requiresApplicationThread(); - - CompositePollEvent.State targetState(); } private static class CompositePollPsuedoEvent implements CompletableEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 3a676b81490d0..a1c7900032e53 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -25,9 +25,8 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; @@ -39,36 +38,16 @@ */ public class CompletableEventReaper { - private enum State { - ACTIVE, - PAUSED - } - - private enum Modification { - - ADD(State.ACTIVE, "added"), - PAUSE(State.PAUSED, "paused"), - RESUME(State.ACTIVE, "resumed"); - - private final State state; - private final String verb; - - Modification(State state, String verb) { - this.state = state; - this.verb = verb; - } - } - private final Logger log; /** - * Tracked events that are candidates for expiration. + * List of tracked events that are candidates for expiration. */ - private final Map, State> tracked; + private final List> tracked; public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); - this.tracked = new HashMap<>(); + this.tracked = new ArrayList<>(); } /** @@ -77,39 +56,7 @@ public CompletableEventReaper(LogContext logContext) { * @param event Event to track */ public void add(CompletableEvent event) { - put(event, Modification.ADD); - } - - /** - * Pauses an event to exclude it as a candidate for reaps. - * - * @param event Event to pause - */ - public void pause(CompletableEvent event) { - put(event, Modification.PAUSE); - } - - /** - * Resumes tracking an event for reaping. - * - * @param event Event to resume - */ - public void resume(CompletableEvent event) { - put(event, Modification.RESUME); - } - - private void put(CompletableEvent event, Modification modification) { - Objects.requireNonNull(event, "Event must be non-null"); - - if (modification == Modification.ADD) { - if (tracked.containsKey(event)) - throw new IllegalArgumentException("The event " + event + " was previously added, so it cannot be " + modification.verb + " again"); - } else if (!tracked.containsKey(event)) { - throw new IllegalArgumentException("The event " + event + " was not previously added, so it cannot be " + modification.verb); - } - - tracked.put(event, modification.state); - log.trace("Event {} was {} and is now in state {}", event, modification.verb, modification.state); + tracked.add(Objects.requireNonNull(event, "Event to track must be non-null")); } /** @@ -138,23 +85,15 @@ private void put(CompletableEvent event, Modification modification) { */ public long reap(long currentTimeMs) { int count = 0; - List> unpausedEvents = new ArrayList<>(tracked.size()); - for (Map.Entry, State> entry : tracked.entrySet()) { - State state = entry.getValue(); + Iterator> iterator = tracked.iterator(); - if (state == State.PAUSED) { - // Don't reap "paused" events - continue; - } - - unpausedEvents.add(entry.getKey()); - } + while (iterator.hasNext()) { + CompletableEvent event = iterator.next(); - for (CompletableEvent event : unpausedEvents) { if (event.future().isDone()) { // Remove any events that are already complete. - tracked.remove(event); + iterator.remove(); continue; } @@ -176,7 +115,7 @@ public long reap(long currentTimeMs) { count++; // Remove the events so that we don't hold a reference to it. - tracked.remove(event); + iterator.remove(); } return count; @@ -204,7 +143,7 @@ public long reap(long currentTimeMs) { public long reap(Collection events) { Objects.requireNonNull(events, "Event queue to reap must be non-null"); - long trackedExpiredCount = completeEventsExceptionallyOnClose(tracked.keySet()); + long trackedExpiredCount = completeEventsExceptionallyOnClose(tracked); tracked.clear(); long eventExpiredCount = completeEventsExceptionallyOnClose(events); @@ -217,25 +156,6 @@ public int size() { return tracked.size(); } - public boolean contains(CompletableEvent event) { - return event != null && tracked.containsKey(event); - } - - public List> uncompletedEvents() { - // The following code does not use the Java Collections Streams API to reduce overhead in the critical - // path of the ConsumerNetworkThread loop. - List> events = new ArrayList<>(); - - for (Map.Entry, State> entry : tracked.entrySet()) { - CompletableEvent event = entry.getKey(); - - if (!event.future().isDone()) - events.add(event); - } - - return events; - } - /** * For all the {@link CompletableEvent}s in the collection, if they're not already complete, invoke * {@link CompletableFuture#completeExceptionally(Throwable)}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 004607f129e3a..4ff0c65ccbe52 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.Blocker; - public class CompositePollEvent extends ApplicationEvent { public enum State { @@ -30,14 +28,12 @@ public enum State { private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; - private final Blocker blocker; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { super(Type.COMPOSITE_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; - this.blocker = new Blocker<>(); } public long deadlineMs() { @@ -52,12 +48,8 @@ public Type nextEventType() { return nextEventType; } - public Blocker blocker() { - return blocker; - } - @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", blocker=" + blocker; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 3734e2c2e5f92..b7eb08c99166d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -102,8 +102,6 @@ import org.mockito.ArgumentMatchers; import org.mockito.MockedStatic; import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.time.Duration; import java.util.Arrays; @@ -111,7 +109,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Map; @@ -171,7 +168,6 @@ @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { - private static final Logger log = LoggerFactory.getLogger(AsyncKafkaConsumerTest.class); private AsyncKafkaConsumer consumer = null; private Time time = new MockTime(0); private final Metrics metrics = new Metrics(); @@ -429,7 +425,7 @@ public void testWakeupBeforeCallingPoll() { consumer.wakeup(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -450,7 +446,7 @@ public void testWakeupAfterEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -475,7 +471,7 @@ public void testWakeupAfterNonEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); // the previously ignored wake-up should not be ignored in the next call @@ -513,13 +509,7 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent( - List.of( - CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, - CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, - CompositePollEvent.State.COMPLETE - ) - ); + markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); assertTrue(callbackExecuted.get()); } @@ -542,10 +532,9 @@ public void testClearWakeupTriggerAfterPoll() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -690,13 +679,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent( - List.of( - CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, - CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, - CompositePollEvent.State.COMPLETE - ) - ); + markResultForCompositePollEvent(); assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @@ -1497,12 +1480,7 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); assertEquals(expectedException.getMessage(), exception.getMessage()); @@ -1594,7 +1572,7 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + markResultForCompositePollEvent(); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); assertEquals(expectedException1.getMessage(), exception.getMessage()); @@ -1678,7 +1656,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); consumer.poll(Duration.ofMillis(100)); verify(applicationEventHandler).add(any(CompositePollEvent.class)); } @@ -1697,7 +1675,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); } @@ -1732,7 +1710,7 @@ public void testLongPollWaitIsLimited() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); assertEquals(2, returnedRecords.count()); @@ -1837,7 +1815,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { try { Thread.currentThread().interrupt(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); assertThrows(InterruptException.class, () -> consumer.poll(Duration.ZERO)); } finally { // clear interrupted state again since this thread may be reused by JUnit @@ -1870,12 +1848,7 @@ void testReaperInvokedInPoll() { consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent( - List.of( - CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, - CompositePollEvent.State.COMPLETE - ) - ); + markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); } @@ -1939,7 +1912,7 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer.assign(singleton(new TopicPartition("topic1", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.COMPLETE); + markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); @@ -2256,35 +2229,8 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); } - private void markResultForCompositePollEvent(CompositePollEvent.State state) { - doAnswer(invocation -> { - CompositePollEvent event = invocation.getArgument(0); - log.error("Am I invoked: {}", event); - - if (Thread.currentThread().isInterrupted()) - event.blocker().completeExceptionally(new InterruptException("Test interrupt")); - - event.blocker().complete(state); - return state; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); - } - - private void markResultForCompositePollEvent(Collection states) { - LinkedList statesQueue = new LinkedList<>(states); - - doAnswer(invocation -> { - CompositePollEvent event = invocation.getArgument(0); - log.error("Am I invoked: {}", event); - CompositePollEvent.State state = statesQueue.poll(); - - if (state == null) - event.blocker().completeExceptionally(new KafkaException("The array of " + CompositePollEvent.State.class.getSimpleName() + " did not provide enough values")); - - if (Thread.currentThread().isInterrupted()) - event.blocker().completeExceptionally(new InterruptException("Test interrupt")); - - event.blocker().complete(state); - return state; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); + private void markResultForCompositePollEvent() { + doAnswer(invocation -> null) + .when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index f806ab65b6b65..6c05bb3c12f02 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -4127,7 +4127,7 @@ private void buildDependencies(MetricConfig metricConfig, properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); - networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true)); + networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler)); } private List collectRecordOffsets(List> records) { @@ -4212,9 +4212,8 @@ public TestableNetworkClientDelegate(Time time, LogContext logContext, KafkaClient client, Metadata metadata, - BackgroundEventHandler backgroundEventHandler, - boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); + BackgroundEventHandler backgroundEventHandler) { + super(time, config, logContext, client, metadata, backgroundEventHandler, mock(AsyncConsumerMetrics.class)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 0347423137b57..2abe2584e2222 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -45,6 +45,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -73,19 +74,22 @@ public class NetworkClientDelegateTest { private MockTime time; private MockClient client; private Metadata metadata; + private AsyncConsumerMetrics asyncConsumerMetrics; private BackgroundEventHandler backgroundEventHandler; @BeforeEach public void setup() { this.time = new MockTime(0); this.metadata = mock(Metadata.class); - this.backgroundEventHandler = mock(BackgroundEventHandler.class); + this.asyncConsumerMetrics = mock(AsyncConsumerMetrics.class); + BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + this.backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, asyncConsumerMetrics); this.client = new MockClient(time, Collections.singletonList(mockNode())); } @Test void testPollResultTimer() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() @@ -109,7 +113,7 @@ void testPollResultTimer() throws Exception { @Test public void testSuccessfulResponse() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); prepareFindCoordinatorResponse(Errors.NONE); @@ -123,7 +127,7 @@ public void testSuccessfulResponse() throws Exception { @Test public void testTimeoutBeforeSend() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { client.setUnreachable(mockNode(), REQUEST_TIMEOUT_MS); NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); @@ -137,7 +141,7 @@ public void testTimeoutBeforeSend() throws Exception { @Test public void testTimeoutAfterSend() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); @@ -171,7 +175,7 @@ public void testEnsureCorrectCompletionTimeOnComplete() { @Test public void testEnsureTimerSetOnAdd() { - NetworkClientDelegate ncd = newNetworkClientDelegate(false); + NetworkClientDelegate ncd = newNetworkClientDelegate(); NetworkClientDelegate.UnsentRequest findCoordRequest = newUnsentFindCoordinatorRequest(); assertNull(findCoordRequest.timer()); @@ -188,7 +192,7 @@ public void testEnsureTimerSetOnAdd() { @Test public void testHasAnyPendingRequests() throws Exception { - try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false)) { + try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate()) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); networkClientDelegate.add(unsentRequest); @@ -219,14 +223,18 @@ public void testPropagateMetadataError() { AuthenticationException authException = new AuthenticationException("Test Auth Exception"); doThrow(authException).when(metadata).maybeThrowAnyException(); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false); - assertTrue(networkClientDelegate.getAndClearMetadataError().isEmpty()); + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(); + List backgroundEvents = backgroundEventHandler.drainEvents(); + assertTrue(backgroundEvents.isEmpty()); networkClientDelegate.poll(0, time.milliseconds()); - Optional metadataError = networkClientDelegate.getAndClearMetadataError(); - assertTrue(metadataError.isPresent()); - assertInstanceOf(AuthenticationException.class, metadataError.get()); - assertEquals(authException.getMessage(), metadataError.get().getMessage()); + backgroundEvents = backgroundEventHandler.drainEvents(); + assertEquals(1, backgroundEvents.size()); + BackgroundEvent event = backgroundEvents.get(0); + assertInstanceOf(ErrorEvent.class, event); + ErrorEvent errorEvent = (ErrorEvent) event; + assertInstanceOf(AuthenticationException.class, errorEvent.error()); + assertEquals(authException.getMessage(), errorEvent.error().getMessage()); } @Test @@ -236,7 +244,7 @@ public void testPropagateMetadataErrorWithErrorEvent() { BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class)); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(true); + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(); assertEquals(0, backgroundEventQueue.size()); networkClientDelegate.poll(0, time.milliseconds()); @@ -253,7 +261,7 @@ public void testPropagateMetadataErrorWithErrorEvent() { public void testRecordUnsentRequestsQueueTime(String groupName) throws Exception { try (Metrics metrics = new Metrics(); AsyncConsumerMetrics asyncConsumerMetrics = new AsyncConsumerMetrics(metrics, groupName); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false, asyncConsumerMetrics)) { + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(asyncConsumerMetrics)) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); networkClientDelegate.add(unsentRequest); asyncConsumerMetrics.recordUnsentRequestsQueueSize(1, time.milliseconds()); @@ -282,11 +290,11 @@ public void testRecordUnsentRequestsQueueTime(String groupName) throws Exception } } - public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue) { - return newNetworkClientDelegate(notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); + public NetworkClientDelegate newNetworkClientDelegate() { + return newNetworkClientDelegate(asyncConsumerMetrics); } - public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue, AsyncConsumerMetrics asyncConsumerMetrics) { + public NetworkClientDelegate newNetworkClientDelegate(AsyncConsumerMetrics asyncConsumerMetrics) { LogContext logContext = new LogContext(); Properties properties = new Properties(); properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); @@ -300,7 +308,6 @@ public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErro this.client, this.metadata, this.backgroundEventHandler, - notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java index ed96b81790002..7f42d3b190d92 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.IsolationLevel; @@ -88,6 +89,7 @@ public class OffsetsRequestManagerTest { private OffsetsRequestManager requestManager; private ConsumerMetadata metadata; private SubscriptionState subscriptionState; + private BackgroundEventHandler backgroundEventHandler; private final Time time = mock(Time.class); private ApiVersions apiVersions; private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); @@ -106,10 +108,12 @@ public void setup() { LogContext logContext = new LogContext(); metadata = mock(ConsumerMetadata.class); subscriptionState = mock(SubscriptionState.class); + backgroundEventHandler = mock(BackgroundEventHandler.class); apiVersions = mock(ApiVersions.class); requestManager = new OffsetsRequestManager( subscriptionState, metadata, + backgroundEventHandler, DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, @@ -799,6 +803,7 @@ public void testRemoteListOffsetsRequestTimeoutMs() { requestManager = new OffsetsRequestManager( subscriptionState, metadata, + backgroundEventHandler, DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index a4268b7eca0a7..f2b3d7210ece9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -2687,7 +2687,7 @@ private void buildDependencies(MetricConfig metricConfig, ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate( time, config, logContext, client, metadata, - new BackgroundEventHandler(new LinkedBlockingQueue<>(), time, mock(AsyncConsumerMetrics.class)), false)); + new BackgroundEventHandler(new LinkedBlockingQueue<>(), time, mock(AsyncConsumerMetrics.class)))); } private class TestableShareConsumeRequestManager extends ShareConsumeRequestManager { @@ -2751,9 +2751,8 @@ public TestableNetworkClientDelegate(Time time, LogContext logContext, KafkaClient client, Metadata metadata, - BackgroundEventHandler backgroundEventHandler, - boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); + BackgroundEventHandler backgroundEventHandler) { + super(time, config, logContext, client, metadata, backgroundEventHandler, mock(AsyncConsumerMetrics.class)); } @Override From 0ac19f96b906f957f3ec2ee37dfe15a749c05e66 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 19 Sep 2025 14:44:37 -0700 Subject: [PATCH 015/123] Clean up --- .../internals/AsyncKafkaConsumer.java | 16 +++++------ .../clients/consumer/internals/Blocker.java | 0 .../internals/ConsumerNetworkThread.java | 2 -- .../internals/OffsetsRequestManager.java | 27 ++++++------------- .../consumer/internals/ShareConsumerImpl.java | 2 -- .../events/ApplicationEventProcessor.java | 8 ------ .../internals/events/CompositePollEvent.java | 7 ----- .../events/ApplicationEventProcessorTest.java | 3 --- 8 files changed, 15 insertions(+), 50 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 9f8ee0e0550d5..80a0619d0672a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -463,7 +463,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker), applicationEventReaper @@ -660,7 +659,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker), applicationEventReaper @@ -874,16 +872,16 @@ public ConsumerRecords poll(final Duration timeout) { } private void prepareFetch(Timer timer) { - long pollTimeMs = timer.currentTimeMs(); - long deadlineMs = calculateDeadlineMs(timer); - ApplicationEvent.Type nextEventType = ApplicationEvent.Type.POLL; - - log.debug("prepareFetch - timer: {}", timer.remainingMs()); - processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); - CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextEventType); + long pollTimeMs = timer.currentTimeMs(); + + CompositePollEvent event = new CompositePollEvent( + calculateDeadlineMs(timer), + pollTimeMs, + ApplicationEvent.Type.POLL + ); applicationEventHandler.add(event); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Blocker.java deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index e4f6958a0cb40..d3ac47903b27c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; @@ -35,7 +34,6 @@ import java.io.Closeable; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; import java.util.LinkedList; import java.util.List; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 7fd588a7648ea..0e41177637424 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -305,8 +305,6 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - log.debug("Adding error: {}", error.getClass()); - if (error instanceof CompletionException) error = error.getCause(); @@ -326,20 +324,17 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d * @throws NoOffsetForPartitionException If no reset strategy is configured. */ private CompletableFuture initWithPartitionOffsetsIfNeeded(Set initializingPartitions) { - log.debug("initWithPartitionOffsetsIfNeeded - initializingPartitions: {}", initializingPartitions); CompletableFuture result = new CompletableFuture<>(); try { // Mark partitions that need reset, using the configured reset strategy. If no // strategy is defined, this will raise a NoOffsetForPartitionException exception. subscriptionState.resetInitializingPositions(initializingPartitions::contains); - } catch (Exception e) { - Throwable t = e; - + } catch (Throwable t) { if (t instanceof CompletionException) t = t.getCause(); backgroundEventHandler.add(new ErrorEvent(t)); - result.completeExceptionally(e); + result.completeExceptionally(t); return result; } @@ -365,7 +360,7 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set result = new CompletableFuture<>(); // The shorter the timeout provided to poll(), the more likely the offsets fetch will time out. To handle @@ -373,8 +368,6 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set> fetchOffsets = @@ -410,14 +403,11 @@ private CompletableFuture initWithCommittedOffsetsIfNeeded(Set offsets, final Throwable error, final CompletableFuture result) { - log.debug("refreshOffsets - offsets: {}, partitions: {}", offsets, String.valueOf(error)); - if (error == null) { // Ensure we only set positions for the partitions that still require one (ex. some partitions may have // been assigned a position manually) Map offsetsToApply = offsetsForInitializingPartitions(offsets); - log.debug("refreshOffsets - offsetsToApply: {}", offsetsToApply); refreshCommittedOffsets(offsetsToApply, metadata, subscriptionState); @@ -462,7 +452,6 @@ private boolean canReusePendingOffsetFetchEvent(Set partitions) return false; } - log.debug("canReusePendingOffsetFetchEvent - pendingOffsetFetchEvent.requestedPartitions: {}, partitions: {}", pendingOffsetFetchEvent.requestedPartitions, partitions); return pendingOffsetFetchEvent.requestedPartitions.equals(partitions); } @@ -477,17 +466,17 @@ private boolean canReusePendingOffsetFetchEvent(Set partitions) * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ CompletableFuture resetPositionsIfNeeded() { - log.debug("resetPositionsIfNeeded"); Map partitionAutoOffsetResetStrategyMap; try { partitionAutoOffsetResetStrategyMap = offsetFetcherUtils.getOffsetResetStrategyForPartitions(); - } catch (Exception e) { - log.debug("resetPositionsIfNeeded - e: {}", e.toString()); + } catch (Throwable t) { + if (t instanceof CompletionException) + t = t.getCause(); - backgroundEventHandler.add(new ErrorEvent(e.getCause())); + backgroundEventHandler.add(new ErrorEvent(t)); CompletableFuture result = new CompletableFuture<>(); - result.completeExceptionally(e); + result.completeExceptionally(t); return result; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index a5f2066afc16c..dd6d35427afd4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -302,7 +302,6 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, backgroundEventHandler, Optional.empty(), applicationEventReaper @@ -413,7 +412,6 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, backgroundEventHandler, Optional.empty(), applicationEventReaper diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 3c92d938142f8..e498fc3797afd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,8 +22,6 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; @@ -65,7 +63,6 @@ public class ApplicationEventProcessor implements EventProcessor supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, - final Supplier networkClientDelegateSupplier, final BackgroundEventHandler backgroundEventHandler, final Optional offsetCommitCallbackInvoker, final CompletableEventReaper applicationEventReaper) { @@ -822,12 +816,10 @@ public static Supplier supplier(final LogContext logC @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); - NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); return new ApplicationEventProcessor( logContext, requestManagers, - networkClientDelegate, metadata, subscriptions, backgroundEventHandler, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 4ff0c65ccbe52..ae69745dbf992 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -18,13 +18,6 @@ public class CompositePollEvent extends ApplicationEvent { - public enum State { - - OFFSET_COMMIT_CALLBACKS_REQUIRED, - BACKGROUND_EVENT_PROCESSING_REQUIRED, - COMPLETE - } - private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 8c5623a7f2637..8c14a87e3190f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -95,7 +95,6 @@ public class ApplicationEventProcessorTest { private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); private final BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); - private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private ApplicationEventProcessor processor; @@ -115,7 +114,6 @@ private void setupProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, @@ -140,7 +138,6 @@ private void setupStreamProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, From ae0ddcc4c0dd538b616350b10bcadd2911627e73 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 13:41:07 -0700 Subject: [PATCH 016/123] Add completion tracking to CompositePollEvent Introduces an AtomicBoolean to track completion state in CompositePollEvent and updates ApplicationEventProcessor to mark events as complete when appropriate. Refactors AsyncKafkaConsumer to use a new CompositePollEventInvoker for polling, replacing prepareFetch, and implements exponential backoff for incomplete events. --- .../internals/AsyncKafkaConsumer.java | 49 +++++++++++++------ .../events/ApplicationEventProcessor.java | 23 ++++++--- .../internals/events/CompositePollEvent.java | 13 ++++- 3 files changed, 61 insertions(+), 24 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 80a0619d0672a..4ac326f8bb5bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -174,6 +174,35 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private static final long NO_CURRENT_THREAD = -1L; + private class CompositePollEventInvoker { + + private final Timer timer; + private final long pollTimeMs; + private CompositePollEvent latest; + private int backoff = -1; + + public CompositePollEventInvoker(Timer timer, long pollTimeMs) { + this.timer = timer; + this.pollTimeMs = pollTimeMs; + } + + private void poll() { + if (latest == null || latest.isComplete()) { + long deadlineMs = calculateDeadlineMs(timer); + latest = new CompositePollEvent(deadlineMs, pollTimeMs, ApplicationEvent.Type.POLL); + applicationEventHandler.add(latest); + } else { + if (backoff == -1) + backoff = 1; + else + backoff *= 2; + + long sleep = Math.min(Math.min(backoff, retryBackoffMs), timer.remainingMs()); + timer.sleep(sleep); + } + } + } + /** * An {@link org.apache.kafka.clients.consumer.internals.events.EventProcessor} that is created and executes in the * application thread for the purpose of processing {@link BackgroundEvent background events} generated by the @@ -837,13 +866,17 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } + CompositePollEventInvoker pollEventInvoker = new CompositePollEventInvoker(timer, time.milliseconds()); + do { // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling // of the fetches. A wakeup between returned fetches and returning records would lead to never // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - prepareFetch(timer); + processBackgroundEvents(); + offsetCommitCallbackInvoker.executeCallbacks(); + pollEventInvoker.poll(); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -871,20 +904,6 @@ public ConsumerRecords poll(final Duration timeout) { } } - private void prepareFetch(Timer timer) { - processBackgroundEvents(); - offsetCommitCallbackInvoker.executeCallbacks(); - - long pollTimeMs = timer.currentTimeMs(); - - CompositePollEvent event = new CompositePollEvent( - calculateDeadlineMs(timer), - pollTimeMs, - ApplicationEvent.Type.POLL - ); - applicationEventHandler.add(event); - } - /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index e498fc3797afd..e510b21f80233 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -241,7 +241,7 @@ public void process(ApplicationEvent event) { } private void process(final CompositePollEvent event) { - if (maybePauseCompositePoll()) + if (maybePauseCompositePoll(event)) return; ApplicationEvent.Type nextEventType = event.nextEventType(); @@ -249,7 +249,7 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.POLL) { processPollEvent(event.pollTimeMs()); - if (maybePauseCompositePoll()) + if (maybePauseCompositePoll(event)) return; nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; @@ -258,7 +258,7 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { processUpdatePatternSubscriptionEvent(); - if (maybePauseCompositePoll()) + if (maybePauseCompositePoll(event)) return; nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; @@ -269,14 +269,15 @@ private void process(final CompositePollEvent event) { applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeFailCompositePoll(event, updatePositionsError) || maybePauseCompositePoll()) + if (maybeFailCompositePoll(event, updatePositionsError) || maybePauseCompositePoll(event)) return; // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event, fetchError) || maybePauseCompositePoll()) + if (maybeFailCompositePoll(event, fetchError) || maybePauseCompositePoll(event)) return; + event.complete(); log.trace("Completed CompositePollEvent {}", event); }); }); @@ -285,14 +286,19 @@ private void process(final CompositePollEvent event) { } log.warn("Unknown next step for composite poll: {}", nextEventType); + event.complete(); } - private boolean maybePauseCompositePoll() { - if (backgroundEventProcessingRequiredTest.requiresApplicationThread()) + private boolean maybePauseCompositePoll(CompositePollEvent event) { + if (backgroundEventProcessingRequiredTest.requiresApplicationThread()) { + event.complete(); return true; + } - if (offsetCommitCallbackInvocationRequiredTest.requiresApplicationThread()) + if (offsetCommitCallbackInvocationRequiredTest.requiresApplicationThread()) { + event.complete(); return true; + } return false; } @@ -311,6 +317,7 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { } backgroundEventHandler.add(new ErrorEvent(t)); + event.complete(); log.trace("Failing CompositePollEvent {}", event, t); return true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index ae69745dbf992..7106f03f04c44 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,11 +16,14 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import java.util.concurrent.atomic.AtomicBoolean; + public class CompositePollEvent extends ApplicationEvent { private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; + private final AtomicBoolean complete = new AtomicBoolean(); public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { super(Type.COMPOSITE_POLL); @@ -41,8 +44,16 @@ public Type nextEventType() { return nextEventType; } + public boolean isComplete() { + return complete.get(); + } + + public void complete() { + complete.set(true); + } + @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", complete=" + complete; } } From 6775aacc2c7f299ce85e5a1b3db4164a1e2b133a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 15:40:48 -0700 Subject: [PATCH 017/123] Refactor poll event handling and metadata error propagation Refactored AsyncKafkaConsumer and related classes to improve composite poll event handling, including explicit state management and pausing for background event processing or offset commit callbacks. Metadata errors are now optionally propagated via a dedicated error field in NetworkClientDelegate, allowing for more flexible error handling. Updated tests and logging to reflect these changes. --- .../internals/AsyncKafkaConsumer.java | 58 +++++++++----- .../internals/ConsumerNetworkThread.java | 23 +++++- .../internals/NetworkClientDelegate.java | 24 +++++- .../internals/OffsetsRequestManager.java | 45 ++++++----- .../consumer/internals/ShareConsumerImpl.java | 3 +- .../events/ApplicationEventProcessor.java | 46 ++++++----- .../events/CompletableEventReaper.java | 13 +++ .../internals/events/CompositePollEvent.java | 80 +++++++++++++++++-- .../clients/consumer/KafkaConsumerTest.java | 4 +- .../internals/FetchRequestManagerTest.java | 7 +- .../internals/NetworkClientDelegateTest.java | 40 +++++----- .../ShareConsumeRequestManagerTest.java | 7 +- 12 files changed, 245 insertions(+), 105 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 4ac326f8bb5bc..5c43d26b28a67 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -109,6 +109,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -176,22 +177,26 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private class CompositePollEventInvoker { - private final Timer timer; - private final long pollTimeMs; private CompositePollEvent latest; private int backoff = -1; - public CompositePollEventInvoker(Timer timer, long pollTimeMs) { - this.timer = timer; - this.pollTimeMs = pollTimeMs; - } + private void poll(Timer timer) { + if (latest == null) { + submitEvent(ApplicationEvent.Type.POLL, timer); + } - private void poll() { - if (latest == null || latest.isComplete()) { - long deadlineMs = calculateDeadlineMs(timer); - latest = new CompositePollEvent(deadlineMs, pollTimeMs, ApplicationEvent.Type.POLL); - applicationEventHandler.add(latest); - } else { + log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); + + CompositePollEvent.Result result = latest.resultOrError(); + CompositePollEvent.State state = result.state(); + + if (state == CompositePollEvent.State.COMPLETE) { + if (fetchBuffer.isEmpty()) + submitEvent(ApplicationEvent.Type.POLL, timer); + } else if (state == CompositePollEvent.State.UNKNOWN) { + latest = null; + throw new KafkaException("Unexpected poll result received"); + } else if (state == CompositePollEvent.State.INCOMPLETE) { if (backoff == -1) backoff = 1; else @@ -199,10 +204,25 @@ private void poll() { long sleep = Math.min(Math.min(backoff, retryBackoffMs), timer.remainingMs()); timer.sleep(sleep); + } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { + processBackgroundEvents(); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { + offsetCommitCallbackInvoker.executeCallbacks(); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); } } + + private void submitEvent(ApplicationEvent.Type type, Timer timer) { + long deadlineMs = calculateDeadlineMs(timer); + latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); + applicationEventHandler.add(latest); + log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); + } } + private final CompositePollEventInvoker pollInvoker = new CompositePollEventInvoker(); + /** * An {@link org.apache.kafka.clients.consumer.internals.events.EventProcessor} that is created and executes in the * application thread for the purpose of processing {@link BackgroundEvent background events} generated by the @@ -466,6 +486,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, fetchMetricsManager.throttleTimeSensor(), clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, + false, asyncConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); @@ -661,6 +682,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, client, metadata, backgroundEventHandler, + false, asyncConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); @@ -866,8 +888,6 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - CompositePollEventInvoker pollEventInvoker = new CompositePollEventInvoker(timer, time.milliseconds()); - do { // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling @@ -876,7 +896,7 @@ public ConsumerRecords poll(final Duration timeout) { wakeupTrigger.maybeTriggerWakeup(); processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); - pollEventInvoker.poll(); + pollInvoker.poll(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -1143,8 +1163,6 @@ public List partitionsFor(String topic, Duration timeout) { try { Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); - - processBackgroundEvents(); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1170,9 +1188,7 @@ public Map> listTopics(Duration timeout) { final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - Map> map = applicationEventHandler.addAndGet(topicMetadataEvent); - processBackgroundEvents(); - return map; + return applicationEventHandler.addAndGet(topicMetadataEvent); } finally { wakeupTrigger.clearTask(); } @@ -1254,7 +1270,6 @@ public Map offsetsForTimes(Map offsets = applicationEventHandler.addAndGet(listOffsetsEvent); - processBackgroundEvents(); Map results = new HashMap<>(offsets.size()); offsets.forEach((k, v) -> results.put(k, v != null ? v.buildOffsetAndTimestamp() : null)); return results; @@ -1320,7 +1335,6 @@ private Map beginningOrEndOffset(Collection offsetAndTimestampMap; try { offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); - processBackgroundEvents(); return offsetAndTimestampMap.entrySet() .stream() .collect(Collectors.toMap( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index d3ac47903b27c..d5b2dc02b74bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; @@ -34,6 +35,7 @@ import java.io.Closeable; import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; import java.util.LinkedList; import java.util.List; @@ -171,7 +173,8 @@ void runOnce() { cachedMaximumTimeToWait = maxTimeToWaitMs; reapExpiredApplicationEvents(currentTimeMs); - } + List> uncompletedEvents = applicationEventReaper.uncompletedEvents(); + maybeFailOnMetadataError(uncompletedEvents); } /** * Process the events—if any—that were produced by the application thread. @@ -356,4 +359,22 @@ void cleanup() { log.debug("Closed the consumer network thread"); } } + + /** + * If there is a metadata error, complete all uncompleted events that require subscription metadata. + */ + private void maybeFailOnMetadataError(List> events) { + List> subscriptionMetadataEvent = new ArrayList<>(); + + for (CompletableEvent ce : events) { + if (ce instanceof CompletableApplicationEvent && ((CompletableApplicationEvent) ce).requireSubscriptionMetadata()) + subscriptionMetadataEvent.add((CompletableApplicationEvent) ce); + } + + if (subscriptionMetadataEvent.isEmpty()) + return; + networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError -> + subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) + ); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 1cb25bb46e069..0b827f9e1c09d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -70,6 +70,8 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; + private Optional metadataError; + private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; public NetworkClientDelegate( @@ -79,6 +81,7 @@ public NetworkClientDelegate( final KafkaClient client, final Metadata metadata, final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { this.time = time; this.client = client; @@ -88,6 +91,8 @@ public NetworkClientDelegate( this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + this.metadataError = Optional.empty(); + this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -155,7 +160,11 @@ private void maybePropagateMetadataError() { try { metadata.maybeThrowAnyException(); } catch (Exception e) { - backgroundEventHandler.add(new ErrorEvent(e)); + if (notifyMetadataErrorsViaErrorQueue) { + backgroundEventHandler.add(new ErrorEvent(e)); + } else { + metadataError = Optional.of(e); + } } } @@ -238,7 +247,13 @@ private ClientRequest makeClientRequest( unsent.handler ); } - + + public Optional getAndClearMetadataError() { + Optional metadataError = this.metadataError; + this.metadataError = Optional.empty(); + return metadataError; + } + public Node leastLoadedNode() { return this.client.leastLoadedNode(time.milliseconds()).node(); } @@ -437,6 +452,7 @@ public static Supplier supplier(final Time time, final Sensor throttleTimeSensor, final ClientTelemetrySender clientTelemetrySender, final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { return new CachedSupplier<>() { @Override @@ -451,7 +467,7 @@ protected NetworkClientDelegate create() { metadata, throttleTimeSensor, clientTelemetrySender); - return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, asyncConsumerMetrics); + return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics); } }; } @@ -466,6 +482,7 @@ public static Supplier supplier(final Time time, final KafkaClient client, final Metadata metadata, final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics) { return new CachedSupplier<>() { @Override @@ -477,6 +494,7 @@ protected NetworkClientDelegate create() { client, metadata, backgroundEventHandler, + notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics ); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 0e41177637424..112fa3c37ddec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -27,7 +27,6 @@ import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -55,8 +54,8 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -86,7 +85,6 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final Logger log; private final OffsetFetcherUtils offsetFetcherUtils; private final SubscriptionState subscriptionState; - private final BackgroundEventHandler backgroundEventHandler; private final Set requestsToRetry; private final List requestsToSend; @@ -97,6 +95,12 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; + /** + * Exception that occurred while updating positions after the triggering event had already + * expired. It will be propagated and cleared on the next call to update fetch positions. + */ + private final AtomicReference cachedUpdatePositionsException = new AtomicReference<>(); + /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update * fetch positions that hasn't completed yet. When a response is received, it's used to @@ -133,7 +137,6 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToRetry = new HashSet<>(); this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; - this.backgroundEventHandler = backgroundEventHandler; this.time = time; this.requestTimeoutMs = requestTimeoutMs; this.defaultApiTimeoutMs = defaultApiTimeoutMs; @@ -235,6 +238,10 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { CompletableFuture result = new CompletableFuture<>(); try { + if (maybeCompleteWithPreviousException(result)) { + return result; + } + validatePositionsIfNeeded(); if (subscriptionState.hasAllFetchPositions()) { @@ -258,6 +265,15 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { return result; } + private boolean maybeCompleteWithPreviousException(CompletableFuture result) { + Throwable cachedException = cachedUpdatePositionsException.getAndSet(null); + if (cachedException != null) { + result.completeExceptionally(cachedException); + return true; + } + return false; + } + /** * Generate requests to fetch offsets and update positions once a response is received. This will first attempt * to use the committed offsets if available. If no committed offsets available, it will use the partition @@ -305,10 +321,7 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - if (error instanceof CompletionException) - error = error.getCause(); - - backgroundEventHandler.add(new ErrorEvent(error)); + cachedUpdatePositionsException.set(error); } }); } @@ -329,12 +342,8 @@ private CompletableFuture initWithPartitionOffsetsIfNeeded(Set resetPositionsIfNeeded() { try { partitionAutoOffsetResetStrategyMap = offsetFetcherUtils.getOffsetResetStrategyForPartitions(); - } catch (Throwable t) { - if (t instanceof CompletionException) - t = t.getCause(); - - backgroundEventHandler.add(new ErrorEvent(t)); + } catch (Exception e) { CompletableFuture result = new CompletableFuture<>(); - result.completeExceptionally(t); + result.completeExceptionally(e); return result; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index dd6d35427afd4..c9db04ceb9ed6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -279,6 +279,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { shareFetchMetricsManager.throttleTimeSensor(), clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, + true, asyncConsumerMetrics ); this.completedAcknowledgements = new LinkedList<>(); @@ -387,7 +388,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { backgroundEventQueue, time, asyncConsumerMetrics); final Supplier networkClientDelegateSupplier = - NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, asyncConsumerMetrics); + NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index e510b21f80233..e7593eee9e319 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; @@ -66,7 +67,6 @@ public class ApplicationEventProcessor implements EventProcessor updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeFailCompositePoll(event, updatePositionsError) || maybePauseCompositePoll(event)) + if (maybeFailCompositePoll(event, updatePositionsError)) return; + log.debug("Processing {} logic for {}", ApplicationEvent.Type.POLL, event); + // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event, fetchError) || maybePauseCompositePoll(event)) + if (maybeFailCompositePoll(event, fetchError)) return; - event.complete(); - log.trace("Completed CompositePollEvent {}", event); + event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); + log.debug("Completed CompositePollEvent {}", event); }); }); @@ -286,17 +288,19 @@ private void process(final CompositePollEvent event) { } log.warn("Unknown next step for composite poll: {}", nextEventType); - event.complete(); + event.complete(CompositePollEvent.State.UNKNOWN, Optional.empty()); } - private boolean maybePauseCompositePoll(CompositePollEvent event) { + private boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { if (backgroundEventProcessingRequiredTest.requiresApplicationThread()) { - event.complete(); + log.debug("Pausing event processing for {} with {} as next step", event, nextEventType); + event.complete(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, Optional.of(nextEventType)); return true; } if (offsetCommitCallbackInvocationRequiredTest.requiresApplicationThread()) { - event.complete(); + log.debug("Pausing event processing for {} with {} as next step", event, nextEventType); + event.complete(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, Optional.of(nextEventType)); return true; } @@ -308,7 +312,7 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { return false; if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { - log.trace("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); + log.debug("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); return false; } @@ -316,9 +320,9 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { t = t.getCause(); } - backgroundEventHandler.add(new ErrorEvent(t)); - event.complete(); - log.trace("Failing CompositePollEvent {}", event, t); + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + event.completeExceptionally(e); + log.debug("Failing event processing for {}", event, e); return true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index a1c7900032e53..ba9d740df9fe8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -156,6 +156,19 @@ public int size() { return tracked.size(); } + public List> uncompletedEvents() { + // The following code does not use the Java Collections Streams API to reduce overhead in the critical + // path of the ConsumerNetworkThread loop. + List> events = new ArrayList<>(); + + for (CompletableEvent event : tracked) { + if (!event.future().isDone()) + events.add(event); + } + + return events; + } + /** * For all the {@link CompletableEvent}s in the collection, if they're not already complete, invoke * {@link CompletableFuture#completeExceptionally(Throwable)}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 7106f03f04c44..4796a78bb7923 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,20 +16,72 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.kafka.common.KafkaException; + +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; public class CompositePollEvent extends ApplicationEvent { + public enum State { + + OFFSET_COMMIT_CALLBACKS_REQUIRED, + BACKGROUND_EVENT_PROCESSING_REQUIRED, + INCOMPLETE, + COMPLETE, + UNKNOWN + } + + public static class Result { + + private static final Result INCOMPLETE = new Result(State.INCOMPLETE, Optional.empty()); + private final State state; + + private final Optional nextEventType; + + public Result(State state, Optional nextEventType) { + this.state = state; + this.nextEventType = nextEventType; + } + + public State state() { + return state; + } + + public Optional nextEventType() { + return nextEventType; + } + + @Override + public String toString() { + return "Result{" + "state=" + state + ", nextEventType=" + nextEventType + '}'; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) return false; + Result result = (Result) o; + return state == result.state && Objects.equals(nextEventType, result.nextEventType); + } + + @Override + public int hashCode() { + return Objects.hash(state, nextEventType); + } + } + private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; - private final AtomicBoolean complete = new AtomicBoolean(); + private final AtomicReference resultOrError; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { super(Type.COMPOSITE_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; + this.resultOrError = new AtomicReference<>(Result.INCOMPLETE); } public long deadlineMs() { @@ -44,16 +96,30 @@ public Type nextEventType() { return nextEventType; } - public boolean isComplete() { - return complete.get(); + public Result resultOrError() { + Object o = resultOrError.get(); + + if (o instanceof KafkaException) + throw (KafkaException) o; + else + return (Result) o; + } + + public void complete(State state, Optional nextEventType) { + Result result = new Result( + Objects.requireNonNull(state), + Objects.requireNonNull(nextEventType) + ); + + resultOrError.compareAndSet(Result.INCOMPLETE, result); } - public void complete() { - complete.set(true); + public void completeExceptionally(KafkaException e) { + resultOrError.compareAndSet(Result.INCOMPLETE, Objects.requireNonNull(e)); } @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", complete=" + complete; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", resultOrError=" + resultOrError; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 78ff15cee5f8e..5bdd329661933 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1046,7 +1046,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1826,7 +1826,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(fetchResponse(tp0, 10L, 1)); @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 6c05bb3c12f02..f806ab65b6b65 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -4127,7 +4127,7 @@ private void buildDependencies(MetricConfig metricConfig, properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); - networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler)); + networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true)); } private List collectRecordOffsets(List> records) { @@ -4212,8 +4212,9 @@ public TestableNetworkClientDelegate(Time time, LogContext logContext, KafkaClient client, Metadata metadata, - BackgroundEventHandler backgroundEventHandler) { - super(time, config, logContext, client, metadata, backgroundEventHandler, mock(AsyncConsumerMetrics.class)); + BackgroundEventHandler backgroundEventHandler, + boolean notifyMetadataErrorsViaErrorQueue) { + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 2abe2584e2222..da68a2626a70f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -45,7 +45,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -89,7 +88,7 @@ public void setup() { @Test void testPollResultTimer() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() @@ -113,7 +112,7 @@ void testPollResultTimer() throws Exception { @Test public void testSuccessfulResponse() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); prepareFindCoordinatorResponse(Errors.NONE); @@ -127,7 +126,7 @@ public void testSuccessfulResponse() throws Exception { @Test public void testTimeoutBeforeSend() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { client.setUnreachable(mockNode(), REQUEST_TIMEOUT_MS); NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); @@ -141,7 +140,7 @@ public void testTimeoutBeforeSend() throws Exception { @Test public void testTimeoutAfterSend() throws Exception { - try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { + try (NetworkClientDelegate ncd = newNetworkClientDelegate(false)) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); @@ -175,7 +174,7 @@ public void testEnsureCorrectCompletionTimeOnComplete() { @Test public void testEnsureTimerSetOnAdd() { - NetworkClientDelegate ncd = newNetworkClientDelegate(); + NetworkClientDelegate ncd = newNetworkClientDelegate(false); NetworkClientDelegate.UnsentRequest findCoordRequest = newUnsentFindCoordinatorRequest(); assertNull(findCoordRequest.timer()); @@ -192,7 +191,7 @@ public void testEnsureTimerSetOnAdd() { @Test public void testHasAnyPendingRequests() throws Exception { - try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate()) { + try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false)) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); networkClientDelegate.add(unsentRequest); @@ -223,18 +222,14 @@ public void testPropagateMetadataError() { AuthenticationException authException = new AuthenticationException("Test Auth Exception"); doThrow(authException).when(metadata).maybeThrowAnyException(); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(); - List backgroundEvents = backgroundEventHandler.drainEvents(); - assertTrue(backgroundEvents.isEmpty()); + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false); + assertTrue(networkClientDelegate.getAndClearMetadataError().isEmpty()); networkClientDelegate.poll(0, time.milliseconds()); - backgroundEvents = backgroundEventHandler.drainEvents(); - assertEquals(1, backgroundEvents.size()); - BackgroundEvent event = backgroundEvents.get(0); - assertInstanceOf(ErrorEvent.class, event); - ErrorEvent errorEvent = (ErrorEvent) event; - assertInstanceOf(AuthenticationException.class, errorEvent.error()); - assertEquals(authException.getMessage(), errorEvent.error().getMessage()); + Optional metadataError = networkClientDelegate.getAndClearMetadataError(); + assertTrue(metadataError.isPresent()); + assertInstanceOf(AuthenticationException.class, metadataError.get()); + assertEquals(authException.getMessage(), metadataError.get().getMessage()); } @Test @@ -244,7 +239,7 @@ public void testPropagateMetadataErrorWithErrorEvent() { BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class)); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(); + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(true); assertEquals(0, backgroundEventQueue.size()); networkClientDelegate.poll(0, time.milliseconds()); @@ -261,7 +256,7 @@ public void testPropagateMetadataErrorWithErrorEvent() { public void testRecordUnsentRequestsQueueTime(String groupName) throws Exception { try (Metrics metrics = new Metrics(); AsyncConsumerMetrics asyncConsumerMetrics = new AsyncConsumerMetrics(metrics, groupName); - NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(asyncConsumerMetrics)) { + NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false, asyncConsumerMetrics)) { NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); networkClientDelegate.add(unsentRequest); asyncConsumerMetrics.recordUnsentRequestsQueueSize(1, time.milliseconds()); @@ -290,11 +285,11 @@ public void testRecordUnsentRequestsQueueTime(String groupName) throws Exception } } - public NetworkClientDelegate newNetworkClientDelegate() { - return newNetworkClientDelegate(asyncConsumerMetrics); + public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue) { + return newNetworkClientDelegate(notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics); } - public NetworkClientDelegate newNetworkClientDelegate(AsyncConsumerMetrics asyncConsumerMetrics) { + public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue, AsyncConsumerMetrics asyncConsumerMetrics) { LogContext logContext = new LogContext(); Properties properties = new Properties(); properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); @@ -308,6 +303,7 @@ public NetworkClientDelegate newNetworkClientDelegate(AsyncConsumerMetrics async this.client, this.metadata, this.backgroundEventHandler, + notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index f2b3d7210ece9..a4268b7eca0a7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -2687,7 +2687,7 @@ private void buildDependencies(MetricConfig metricConfig, ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate( time, config, logContext, client, metadata, - new BackgroundEventHandler(new LinkedBlockingQueue<>(), time, mock(AsyncConsumerMetrics.class)))); + new BackgroundEventHandler(new LinkedBlockingQueue<>(), time, mock(AsyncConsumerMetrics.class)), false)); } private class TestableShareConsumeRequestManager extends ShareConsumeRequestManager { @@ -2751,8 +2751,9 @@ public TestableNetworkClientDelegate(Time time, LogContext logContext, KafkaClient client, Metadata metadata, - BackgroundEventHandler backgroundEventHandler) { - super(time, config, logContext, client, metadata, backgroundEventHandler, mock(AsyncConsumerMetrics.class)); + BackgroundEventHandler backgroundEventHandler, + boolean notifyMetadataErrorsViaErrorQueue) { + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); } @Override From 2c3547e06aaaefd53e059c54e9c271abd6de6baf Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:05:44 -0700 Subject: [PATCH 018/123] Inject NetworkClientDelegate into ApplicationEventProcessor Adds NetworkClientDelegate as a dependency to ApplicationEventProcessor and updates AsyncKafkaConsumer and ShareConsumerImpl to supply it. Introduces error handling in composite poll processing using metadata errors from NetworkClientDelegate. Updates related tests to mock the new dependency. --- .../internals/AsyncKafkaConsumer.java | 3 +- .../consumer/internals/ShareConsumerImpl.java | 2 ++ .../events/ApplicationEventProcessor.java | 32 +++++++++++++++---- .../events/ApplicationEventProcessorTest.java | 2 ++ 4 files changed, 32 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 5c43d26b28a67..91a328b1837a8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -109,7 +109,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -513,6 +512,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker), applicationEventReaper @@ -710,6 +710,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.of(offsetCommitCallbackInvoker), applicationEventReaper diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index c9db04ceb9ed6..08767c397e41d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -303,6 +303,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.empty(), applicationEventReaper @@ -413,6 +414,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metadata, subscriptions, requestManagersSupplier, + networkClientDelegateSupplier, backgroundEventHandler, Optional.empty(), applicationEventReaper diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index e7593eee9e319..dffc81feacded 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; @@ -64,6 +65,7 @@ public class ApplicationEventProcessor implements EventProcessor(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeFailCompositePoll(event, updatePositionsError)) + if (maybeFailCompositePoll(event) || maybeFailCompositePoll(event, updatePositionsError)) return; - log.debug("Processing {} logic for {}", ApplicationEvent.Type.POLL, event); + log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event, fetchError)) + if (maybeFailCompositePoll(event) || maybeFailCompositePoll(event, fetchError)) return; event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); @@ -326,6 +330,19 @@ private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { return true; } + private boolean maybeFailCompositePoll(CompositePollEvent event) { + Optional exception = networkClientDelegate.getAndClearMetadataError(); + + if (exception.isPresent()) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(exception.get()); + event.completeExceptionally(e); + log.debug("Failing event processing for {}", event, e); + return true; + } + + return false; + } + private void process(final PollEvent event) { processPollEvent(event.pollTimeMs()); event.markReconcileAndAutoCommitComplete(); @@ -820,6 +837,7 @@ public static Supplier supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, + final Supplier networkClientDelegateSupplier, final BackgroundEventHandler backgroundEventHandler, final Optional offsetCommitCallbackInvoker, final CompletableEventReaper applicationEventReaper) { @@ -827,10 +845,12 @@ public static Supplier supplier(final LogContext logC @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); + NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); return new ApplicationEventProcessor( logContext, requestManagers, + networkClientDelegate, metadata, subscriptions, backgroundEventHandler, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 8c14a87e3190f..ed37fecf28d0e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -114,6 +114,7 @@ private void setupProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, + mock(NetworkClientDelegate.class), metadata, subscriptionState, backgroundEventHandler, @@ -138,6 +139,7 @@ private void setupStreamProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, + mock(NetworkClientDelegate.class), metadata, subscriptionState, backgroundEventHandler, From 18f4fa11f3e73b6c0eb908086e7abbdc3df8c7fa Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:08:33 -0700 Subject: [PATCH 019/123] Remove BackgroundEventHandler from OffsetsRequestManager Eliminated the BackgroundEventHandler parameter from OffsetsRequestManager and its usages in RequestManagers and related tests. This simplifies the constructor and removes unnecessary dependencies. --- .../clients/consumer/internals/OffsetsRequestManager.java | 3 --- .../kafka/clients/consumer/internals/RequestManagers.java | 1 - .../consumer/internals/OffsetsRequestManagerTest.java | 5 ----- 3 files changed, 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 112fa3c37ddec..4c8d10ad323ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -26,7 +26,6 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -112,7 +111,6 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou public OffsetsRequestManager(final SubscriptionState subscriptionState, final ConsumerMetadata metadata, - final BackgroundEventHandler backgroundEventHandler, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, @@ -124,7 +122,6 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); - requireNonNull(backgroundEventHandler); requireNonNull(isolationLevel); requireNonNull(time); requireNonNull(apiVersions); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index 3571726216e8e..ae39753f3d8e8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -284,7 +284,6 @@ protected RequestManagers create() { final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions, metadata, - backgroundEventHandler, fetchConfig.isolationLevel, time, retryBackoffMs, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java index 7f42d3b190d92..ed96b81790002 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.IsolationLevel; @@ -89,7 +88,6 @@ public class OffsetsRequestManagerTest { private OffsetsRequestManager requestManager; private ConsumerMetadata metadata; private SubscriptionState subscriptionState; - private BackgroundEventHandler backgroundEventHandler; private final Time time = mock(Time.class); private ApiVersions apiVersions; private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); @@ -108,12 +106,10 @@ public void setup() { LogContext logContext = new LogContext(); metadata = mock(ConsumerMetadata.class); subscriptionState = mock(SubscriptionState.class); - backgroundEventHandler = mock(BackgroundEventHandler.class); apiVersions = mock(ApiVersions.class); requestManager = new OffsetsRequestManager( subscriptionState, metadata, - backgroundEventHandler, DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, @@ -803,7 +799,6 @@ public void testRemoteListOffsetsRequestTimeoutMs() { requestManager = new OffsetsRequestManager( subscriptionState, metadata, - backgroundEventHandler, DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, From ea99a13021347b801b4c44a7fc600885e1752c7d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:12:38 -0700 Subject: [PATCH 020/123] Handle immediate metadata errors for CompletableEvents Added logic to check and fail CompletableEvents for metadata errors immediately upon processing, ensuring events that do not enter the awaiting state are handled correctly. Updated related tests to use consistent mocks and reduced poll durations for faster execution. --- .../consumer/internals/ConsumerNetworkThread.java | 9 +++++++-- .../apache/kafka/clients/consumer/KafkaConsumerTest.java | 4 ++-- .../consumer/internals/NetworkClientDelegateTest.java | 7 ++----- .../internals/events/ApplicationEventProcessorTest.java | 5 +++-- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index d5b2dc02b74bf..d2d178a88c38b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -174,7 +174,8 @@ void runOnce() { reapExpiredApplicationEvents(currentTimeMs); List> uncompletedEvents = applicationEventReaper.uncompletedEvents(); - maybeFailOnMetadataError(uncompletedEvents); } + maybeFailOnMetadataError(uncompletedEvents); + } /** * Process the events—if any—that were produced by the application thread. @@ -192,6 +193,10 @@ private void processApplicationEvents() { try { if (event instanceof CompletableEvent) { applicationEventReaper.add((CompletableEvent) event); + // Check if there are any metadata errors and fail the CompletableEvent if an error is present. + // This call is meant to handle "immediately completed events" which may not enter the awaiting state, + // so metadata errors need to be checked and handled right away. + maybeFailOnMetadataError(List.of((CompletableEvent) event)); } applicationEventProcessor.process(event); } catch (Throwable t) { @@ -374,7 +379,7 @@ private void maybeFailOnMetadataError(List> events) { if (subscriptionMetadataEvent.isEmpty()) return; networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError -> - subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) + subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) ); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 5bdd329661933..78ff15cee5f8e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1046,7 +1046,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1826,7 +1826,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(fetchResponse(tp0, 10L, 1)); @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index da68a2626a70f..0347423137b57 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -73,16 +73,13 @@ public class NetworkClientDelegateTest { private MockTime time; private MockClient client; private Metadata metadata; - private AsyncConsumerMetrics asyncConsumerMetrics; private BackgroundEventHandler backgroundEventHandler; @BeforeEach public void setup() { this.time = new MockTime(0); this.metadata = mock(Metadata.class); - this.asyncConsumerMetrics = mock(AsyncConsumerMetrics.class); - BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - this.backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, asyncConsumerMetrics); + this.backgroundEventHandler = mock(BackgroundEventHandler.class); this.client = new MockClient(time, Collections.singletonList(mockNode())); } @@ -286,7 +283,7 @@ public void testRecordUnsentRequestsQueueTime(String groupName) throws Exception } public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue) { - return newNetworkClientDelegate(notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics); + return newNetworkClientDelegate(notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); } public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErrorsViaErrorQueue, AsyncConsumerMetrics asyncConsumerMetrics) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index ed37fecf28d0e..8c5623a7f2637 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -95,6 +95,7 @@ public class ApplicationEventProcessorTest { private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); private final BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); + private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private ApplicationEventProcessor processor; @@ -114,7 +115,7 @@ private void setupProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - mock(NetworkClientDelegate.class), + networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, @@ -139,7 +140,7 @@ private void setupStreamProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - mock(NetworkClientDelegate.class), + networkClientDelegate, metadata, subscriptionState, backgroundEventHandler, From 56062f5b01e5f0a78bd93af2949488d9959d6f65 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:13:40 -0700 Subject: [PATCH 021/123] Update NetworkClientDelegate.java --- .../kafka/clients/consumer/internals/NetworkClientDelegate.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 0b827f9e1c09d..374ff96fd5e8b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -247,7 +247,7 @@ private ClientRequest makeClientRequest( unsent.handler ); } - + public Optional getAndClearMetadataError() { Optional metadataError = this.metadataError; this.metadataError = Optional.empty(); From 99304db9e85daa1854bcfab671773c10a1d54840 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:18:19 -0700 Subject: [PATCH 022/123] Remove extra whitespace in NetworkClientDelegate Cleaned up unnecessary whitespace and blank lines in NetworkClientDelegate.java to improve code readability. --- .../clients/consumer/internals/NetworkClientDelegate.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 374ff96fd5e8b..0149ffa72c178 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -247,7 +247,7 @@ private ClientRequest makeClientRequest( unsent.handler ); } - + public Optional getAndClearMetadataError() { Optional metadataError = this.metadataError; this.metadataError = Optional.empty(); @@ -500,5 +500,4 @@ protected NetworkClientDelegate create() { } }; } - } From 702b25753b0c0b28cac43805d4bb5b8d66a06e64 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:18:38 -0700 Subject: [PATCH 023/123] Revert removal of contains() from CompletableEventReaper --- .../consumer/internals/events/CompletableEventReaper.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index ba9d740df9fe8..b4440de06264b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -156,6 +156,10 @@ public int size() { return tracked.size(); } + public boolean contains(CompletableEvent event) { + return event != null && tracked.contains(event); + } + public List> uncompletedEvents() { // The following code does not use the Java Collections Streams API to reduce overhead in the critical // path of the ConsumerNetworkThread loop. From 81b707e7458f595528d490776a8578831b06b431 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:19:30 -0700 Subject: [PATCH 024/123] Update NetworkClientDelegate.java --- .../kafka/clients/consumer/internals/NetworkClientDelegate.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 0149ffa72c178..31c402df2a6db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -247,7 +247,7 @@ private ClientRequest makeClientRequest( unsent.handler ); } - + public Optional getAndClearMetadataError() { Optional metadataError = this.metadataError; this.metadataError = Optional.empty(); From 81598844bde86cc64c72260349abee4c12e79caf Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:44:49 -0700 Subject: [PATCH 025/123] Refactor application thread requirement handling Introduces AsyncConsumerApplicationThreadRequirement to encapsulate logic for determining when to pause event processing for application thread execution. Updates ApplicationEventProcessor and related classes to use a unified CompositePollApplicationThreadRequirement interface, simplifying constructor signatures and improving code clarity. --- .../internals/AsyncKafkaConsumer.java | 45 +++++++++++++++++-- .../consumer/internals/ShareConsumerImpl.java | 6 +-- .../events/ApplicationEventProcessor.java | 45 +++++++------------ .../events/ApplicationEventProcessorTest.java | 13 ++---- 4 files changed, 63 insertions(+), 46 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 91a328b1837a8..124500432beec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -174,6 +174,31 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private static final long NO_CURRENT_THREAD = -1L; + private static class AsyncConsumerApplicationThreadRequirement implements ApplicationEventProcessor.CompositePollApplicationThreadRequirement { + + private final BackgroundEventHandler backgroundEventHandler; + private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; + + public AsyncConsumerApplicationThreadRequirement(BackgroundEventHandler backgroundEventHandler, + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker) { + this.backgroundEventHandler = backgroundEventHandler; + this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; + } + + @Override + public Optional requirement() { + // If there are background events to process, exit to the application thread. + if (backgroundEventHandler.size() > 0) + return Optional.of(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + + // If there are enqueued callbacks to invoke, exit to the application thread. + if (offsetCommitCallbackInvoker.size() > 0) + return Optional.of(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); + + return Optional.empty(); + } + } + private class CompositePollEventInvoker { private CompositePollEvent latest; @@ -373,6 +398,8 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); + private final AsyncConsumerApplicationThreadRequirement asyncApplicationThreadRequirement; + private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -508,13 +535,16 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); + this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( + backgroundEventHandler, + offsetCommitCallbackInvoker + ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, - backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker), + asyncApplicationThreadRequirement, applicationEventReaper ); this.applicationEventHandler = applicationEventHandlerFactory.build( @@ -616,6 +646,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, asyncConsumerMetrics ); + this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( + backgroundEventHandler, + offsetCommitCallbackInvoker + ); } AsyncKafkaConsumer(LogContext logContext, @@ -705,14 +739,17 @@ public AsyncKafkaConsumer(final ConsumerConfig config, Optional.empty() ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); + this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( + backgroundEventHandler, + offsetCommitCallbackInvoker + ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, subscriptions, requestManagersSupplier, networkClientDelegateSupplier, - backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker), + asyncApplicationThreadRequirement, applicationEventReaper ); this.applicationEventHandler = new ApplicationEventHandler(logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 08767c397e41d..5b37407a178c6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -304,8 +304,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { subscriptions, requestManagersSupplier, networkClientDelegateSupplier, - backgroundEventHandler, - Optional.empty(), + Optional::empty, applicationEventReaper ); @@ -415,8 +414,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { subscriptions, requestManagersSupplier, networkClientDelegateSupplier, - backgroundEventHandler, - Optional.empty(), + Optional::empty, applicationEventReaper ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index dffc81feacded..20af1f7054245 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -25,7 +25,6 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; -import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; @@ -66,32 +65,25 @@ public class ApplicationEventProcessor implements EventProcessor offsetCommitCallbackInvoker, + final NetworkClientDelegate networkClientDelegate, + final CompositePollApplicationThreadRequirement compositePollApplicationThreadRequirement, final CompletableEventReaper applicationEventReaper) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; - this.networkClientDelegate = networkClientDelegate; this.metadata = metadata; this.subscriptions = subscriptions; + this.networkClientDelegate = networkClientDelegate; + this.compositePollApplicationThreadRequirement = compositePollApplicationThreadRequirement; this.applicationEventReaper = applicationEventReaper; this.metadataVersionSnapshot = metadata.updateVersion(); - - // If there are background events to process, exit to the application thread. - this.backgroundEventProcessingRequiredTest = () -> backgroundEventHandler.size() > 0; - - // If there are enqueued callbacks to invoke, exit to the application thread. - this.offsetCommitCallbackInvocationRequiredTest = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0; } @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @@ -296,15 +288,12 @@ private void process(final CompositePollEvent event) { } private boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { - if (backgroundEventProcessingRequiredTest.requiresApplicationThread()) { - log.debug("Pausing event processing for {} with {} as next step", event, nextEventType); - event.complete(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED, Optional.of(nextEventType)); - return true; - } + Optional stateOpt = compositePollApplicationThreadRequirement.requirement(); - if (offsetCommitCallbackInvocationRequiredTest.requiresApplicationThread()) { - log.debug("Pausing event processing for {} with {} as next step", event, nextEventType); - event.complete(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED, Optional.of(nextEventType)); + if (stateOpt.isPresent()) { + CompositePollEvent.State state = stateOpt.get(); + log.debug("Pausing event processing for {} with {} as next step", state, nextEventType); + event.complete(state, Optional.of(nextEventType)); return true; } @@ -838,8 +827,7 @@ public static Supplier supplier(final LogContext logC final SubscriptionState subscriptions, final Supplier requestManagersSupplier, final Supplier networkClientDelegateSupplier, - final BackgroundEventHandler backgroundEventHandler, - final Optional offsetCommitCallbackInvoker, + final CompositePollApplicationThreadRequirement applicationThreadRequirement, final CompletableEventReaper applicationEventReaper) { return new CachedSupplier<>() { @Override @@ -850,11 +838,10 @@ protected ApplicationEventProcessor create() { return new ApplicationEventProcessor( logContext, requestManagers, - networkClientDelegate, metadata, subscriptions, - backgroundEventHandler, - offsetCommitCallbackInvoker, + networkClientDelegate, + applicationThreadRequirement, applicationEventReaper ); } @@ -934,13 +921,13 @@ private CompletableFuture processCheckAndUpdatePositionsEvent(final lon } /** - * This interface exists mostly to make the code more intuitive. When {@link #requiresApplicationThread()} + * This interface exists mostly to make the code more intuitive. When {@link #requirement()} * returns true, the {@link CompositePollEvent} processing needs to be interrupted so that processing * can return to the application thread. */ - private interface RequiresApplicationThreadExecution { + public interface CompositePollApplicationThreadRequirement { - boolean requiresApplicationThread(); + Optional requirement(); } private static class CompositePollPsuedoEvent implements CompletableEvent { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 8c5623a7f2637..dcf7ea03bc9e6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.clients.consumer.internals.FetchRequestManager; import org.apache.kafka.clients.consumer.internals.MockRebalanceListener; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; -import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.clients.consumer.internals.OffsetsRequestManager; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.StreamsGroupHeartbeatRequestManager; @@ -93,8 +92,6 @@ public class ApplicationEventProcessorTest { private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock(StreamsGroupHeartbeatRequestManager.class); private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); - private final BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); - private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private ApplicationEventProcessor processor; @@ -115,11 +112,10 @@ private void setupProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - networkClientDelegate, metadata, subscriptionState, - backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker), + networkClientDelegate, + Optional::empty, applicationEventReaper ); } @@ -140,11 +136,10 @@ private void setupStreamProcessor(boolean withGroupId) { processor = new ApplicationEventProcessor( new LogContext(), requestManagers, - networkClientDelegate, metadata, subscriptionState, - backgroundEventHandler, - Optional.of(offsetCommitCallbackInvoker), + networkClientDelegate, + Optional::empty, applicationEventReaper ); } From 71120224f4d5d2528f02d5c58fe183c0bb9a4848 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 20 Sep 2025 16:51:30 -0700 Subject: [PATCH 026/123] Update AsyncKafkaConsumer.java --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 124500432beec..7402fbf305010 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -399,7 +399,6 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { private Optional clientTelemetryReporter = Optional.empty(); private final AsyncConsumerApplicationThreadRequirement asyncApplicationThreadRequirement; - private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -932,6 +931,7 @@ public ConsumerRecords poll(final Duration timeout) { // of the fetches. A wakeup between returned fetches and returning records would lead to never // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); + processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); pollInvoker.poll(timer); @@ -1201,6 +1201,7 @@ public List partitionsFor(String topic, Duration timeout) { try { Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); + return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1889,7 +1890,12 @@ private Fetch pollForFetches(Timer timer) { * done as an optimization so that the next round of data can be pre-fetched. */ private Fetch collectFetch() { - return fetchCollector.collectFetch(fetchBuffer); + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + // Notify the network thread to wake up and start the next round of fetching. + applicationEventHandler.wakeupNetworkThread(); + + return fetch; } /** From f40a4ac27ff4585ddae03bd0894ef3c419778918 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 22 Sep 2025 18:48:37 -0700 Subject: [PATCH 027/123] Refactor consumer record polling in tests Replaces direct calls to consumer.poll with a new pollForRecords() helper method in multiple test cases. This improves code reuse and reliability by waiting for records to be available, and removes unnecessary suppress warnings and unchecked casts. --- .../clients/consumer/KafkaConsumerTest.java | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 78ff15cee5f8e..08d3ce12e81dd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -102,6 +102,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.MockConsumerInterceptor; import org.apache.kafka.test.MockDeserializer; @@ -935,7 +936,6 @@ public void verifyPollTimesOutDuringMetadataUpdate(GroupProtocol groupProtocol) @ParameterizedTest @EnumSource(GroupProtocol.class) - @SuppressWarnings("unchecked") public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -951,7 +951,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol g client.prepareResponse(listOffsetsResponse(Map.of(tp0, 50L))); client.prepareResponse(fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + ConsumerRecords records = pollForRecords(); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -1045,8 +1045,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); - @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + ConsumerRecords records = pollForRecords(); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1731,7 +1730,6 @@ private void initializeSubscriptionWithSingleTopic(KafkaConsumer consumer, @ParameterizedTest @EnumSource(GroupProtocol.class) - @SuppressWarnings("unchecked") public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1766,7 +1764,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(100)); + ConsumerRecords records = pollForRecords(); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); @@ -1825,8 +1823,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + ConsumerRecords records = pollForRecords(); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -2655,7 +2652,6 @@ public void testInvalidGroupMetadata(GroupProtocol groupProtocol) throws Interru @ParameterizedTest @EnumSource(GroupProtocol.class) - @SuppressWarnings("unchecked") public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedException { final ConsumerMetadata metadata = createMetadata(subscription); final MockClient client = new MockClient(time, metadata); @@ -2715,7 +2711,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept final FetchInfo fetchInfo = new FetchInfo(1L, 99L, 50L, 5); client.respondToRequest(fetchRequest, fetchResponse(Map.of(tp0, fetchInfo))); - final ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); + final ConsumerRecords records = pollForRecords(); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -2725,6 +2721,22 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept assertEquals(OptionalLong.of(45L), consumer.currentLag(tp0)); } + @SuppressWarnings("unchecked") + private ConsumerRecords pollForRecords() { + Timer timer = time.timer(15000); + + while (timer.notExpired()) { + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); + + if (!records.isEmpty()) + return records; + } + + throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); + } + + + @ParameterizedTest @EnumSource(GroupProtocol.class) public void testListOffsetShouldUpdateSubscriptions(GroupProtocol groupProtocol) { From abaa4dc6392ce4d6bf789decaeb09a22e059fbb0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 22 Sep 2025 18:48:46 -0700 Subject: [PATCH 028/123] Reset backoff on event submission in AsyncKafkaConsumer Sets backoff to -1 when submitting a new application event in AsyncKafkaConsumer. This ensures backoff state is reset for each event submission. --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 7402fbf305010..fb03c112c0bfc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -240,6 +240,7 @@ private void poll(Timer timer) { private void submitEvent(ApplicationEvent.Type type, Timer timer) { long deadlineMs = calculateDeadlineMs(timer); latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); + backoff = -1; applicationEventHandler.add(latest); log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); } From 1570f6997f851a1fb5e240f7bb736e9837df5391 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 22 Sep 2025 21:12:06 -0700 Subject: [PATCH 029/123] Handle exceptions in AsyncKafkaConsumer poll event Wrap and propagate exceptions from resultOrError in AsyncKafkaConsumer, ensuring latest request is cleared when an error occurs. Also clear latest when poll event completes to properly track request lifecycle. --- .../consumer/internals/AsyncKafkaConsumer.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index fb03c112c0bfc..e7560d92c1490 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -211,10 +211,23 @@ private void poll(Timer timer) { log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); - CompositePollEvent.Result result = latest.resultOrError(); + CompositePollEvent.Result result; + + try { + result = latest.resultOrError(); + } catch (Throwable t) { + // If the background thread hit an exception, bubble it up to the user but make sure to clear + // out the latest request to signify this one is complete. + latest = null; + throw ConsumerUtils.maybeWrapAsKafkaException(t); + } + CompositePollEvent.State state = result.state(); if (state == CompositePollEvent.State.COMPLETE) { + // Make sure to clear out the latest request since it's complete. + latest = null; + if (fetchBuffer.isEmpty()) submitEvent(ApplicationEvent.Type.POLL, timer); } else if (state == CompositePollEvent.State.UNKNOWN) { From 2d21fa0fdfd4e4ebba59408b5a67f11adbbbbd5b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 13:49:37 -0700 Subject: [PATCH 030/123] Refactor poll event handling and metadata error management Simplifies AsyncKafkaConsumer's CompositePollEventInvoker by removing backoff logic and streamlining state handling. NetworkClientDelegate now uses AtomicReference for metadataError to improve thread safety. ApplicationEventProcessor refines error handling in composite poll events. Updates tests to reflect API changes and exception types. --- .../internals/AsyncKafkaConsumer.java | 55 +++++++------------ .../internals/NetworkClientDelegate.java | 12 ++-- .../events/ApplicationEventProcessor.java | 7 ++- .../clients/consumer/KafkaConsumerTest.java | 4 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- 5 files changed, 33 insertions(+), 47 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e7560d92c1490..c1aacca4ce410 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -202,58 +202,41 @@ public Optional requirement() { private class CompositePollEventInvoker { private CompositePollEvent latest; - private int backoff = -1; private void poll(Timer timer) { if (latest == null) { submitEvent(ApplicationEvent.Type.POLL, timer); } - log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); - - CompositePollEvent.Result result; - try { - result = latest.resultOrError(); + log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); + + CompositePollEvent.Result result = latest.resultOrError(); + CompositePollEvent.State state = result.state(); + + if (state == CompositePollEvent.State.COMPLETE) { + // Make sure to clear out the latest request since it's complete. + latest = null; + } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { + processBackgroundEvents(); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { + offsetCommitCallbackInvoker.executeCallbacks(); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + } else if (state == CompositePollEvent.State.UNKNOWN) { + throw new KafkaException("Unexpected poll result received"); + } } catch (Throwable t) { - // If the background thread hit an exception, bubble it up to the user but make sure to clear - // out the latest request to signify this one is complete. + // If an exception is hit, bubble it up to the user but make sure to clear out the latest request + // to signify this one is complete. latest = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); } - - CompositePollEvent.State state = result.state(); - - if (state == CompositePollEvent.State.COMPLETE) { - // Make sure to clear out the latest request since it's complete. - latest = null; - - if (fetchBuffer.isEmpty()) - submitEvent(ApplicationEvent.Type.POLL, timer); - } else if (state == CompositePollEvent.State.UNKNOWN) { - latest = null; - throw new KafkaException("Unexpected poll result received"); - } else if (state == CompositePollEvent.State.INCOMPLETE) { - if (backoff == -1) - backoff = 1; - else - backoff *= 2; - - long sleep = Math.min(Math.min(backoff, retryBackoffMs), timer.remainingMs()); - timer.sleep(sleep); - } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { - processBackgroundEvents(); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { - offsetCommitCallbackInvoker.executeCallbacks(); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } } private void submitEvent(ApplicationEvent.Type type, Timer timer) { long deadlineMs = calculateDeadlineMs(timer); latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); - backoff = -1; applicationEventHandler.add(latest); log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 31c402df2a6db..5f71cd3fbc74f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -51,6 +51,7 @@ import java.util.Optional; import java.util.Queue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Supplier; @@ -70,7 +71,7 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private Optional metadataError; + private final AtomicReference metadataError; private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; @@ -91,7 +92,7 @@ public NetworkClientDelegate( this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - this.metadataError = Optional.empty(); + this.metadataError = new AtomicReference<>(); this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -163,7 +164,7 @@ private void maybePropagateMetadataError() { if (notifyMetadataErrorsViaErrorQueue) { backgroundEventHandler.add(new ErrorEvent(e)); } else { - metadataError = Optional.of(e); + metadataError.compareAndSet(null, e); } } } @@ -249,9 +250,8 @@ private ClientRequest makeClientRequest( } public Optional getAndClearMetadataError() { - Optional metadataError = this.metadataError; - this.metadataError = Optional.empty(); - return metadataError; + Exception exception = metadataError.getAndSet(null); + return Optional.ofNullable(exception); } public Node leastLoadedNode() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 20af1f7054245..035d24c41a274 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -265,14 +265,14 @@ private void process(final CompositePollEvent event) { applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeFailCompositePoll(event) || maybeFailCompositePoll(event, updatePositionsError)) + if (maybeFailCompositePoll(event, updatePositionsError)) return; log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); // If needed, create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event) || maybeFailCompositePoll(event, fetchError)) + if (maybeFailCompositePoll(event, fetchError)) return; event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); @@ -301,6 +301,9 @@ private boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEve } private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { + if (maybeFailCompositePoll(event)) + return true; + if (t == null) return false; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 08d3ce12e81dd..6ff8d98d1f2a9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2118,7 +2118,7 @@ public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol gro time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); - final ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ZERO); + final ConsumerRecords records = pollForRecords(); assertFalse(records.isEmpty()); assertFalse(records.nextOffsets().isEmpty()); } @@ -3666,7 +3666,7 @@ public void testPreventMultiThread(GroupProtocol groupProtocol) throws Interrupt service.execute(() -> consumer.poll(Duration.ofSeconds(5))); try { TimeUnit.SECONDS.sleep(1); - assertThrows(ConcurrentModificationException.class, () -> consumer.poll(Duration.ZERO)); + assertThrows(ConcurrentModificationException.class, () -> consumer.poll(Duration.ofSeconds(5))); client.wakeup(); consumer.wakeup(); } finally { diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index f950362354c8c..4f09abc89dd65 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -1358,7 +1358,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val consumer = createConsumer() consumer.assign(java.util.List.of(tp)) - assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) + assertThrows(classOf[AuthorizationException], () => consumeRecords(consumer)) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) From b1937702d2047f32c7d0a400bf716d5fa0db735b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 16:42:53 -0700 Subject: [PATCH 031/123] Refactor CompositePollEventInvoker to standalone class Moved CompositePollEventInvoker from AsyncKafkaConsumer to its own file for better separation of concerns and testability. Updated AsyncKafkaConsumer to use the new class and refactored constructors accordingly. Enhanced related tests to use new helper methods for polling and exception handling, improving test clarity and reliability. --- .../internals/AsyncKafkaConsumer.java | 71 +++++------- .../internals/CompositePollEventInvoker.java | 96 +++++++++++++++++ .../clients/consumer/KafkaConsumerTest.java | 21 +++- .../internals/AsyncKafkaConsumerTest.java | 102 ++++++++++++++---- 4 files changed, 219 insertions(+), 71 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index c1aacca4ce410..8cce81dcc8c55 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -199,51 +199,6 @@ public Optional requirement() { } } - private class CompositePollEventInvoker { - - private CompositePollEvent latest; - - private void poll(Timer timer) { - if (latest == null) { - submitEvent(ApplicationEvent.Type.POLL, timer); - } - - try { - log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); - - CompositePollEvent.Result result = latest.resultOrError(); - CompositePollEvent.State state = result.state(); - - if (state == CompositePollEvent.State.COMPLETE) { - // Make sure to clear out the latest request since it's complete. - latest = null; - } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { - processBackgroundEvents(); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { - offsetCommitCallbackInvoker.executeCallbacks(); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } else if (state == CompositePollEvent.State.UNKNOWN) { - throw new KafkaException("Unexpected poll result received"); - } - } catch (Throwable t) { - // If an exception is hit, bubble it up to the user but make sure to clear out the latest request - // to signify this one is complete. - latest = null; - throw ConsumerUtils.maybeWrapAsKafkaException(t); - } - } - - private void submitEvent(ApplicationEvent.Type type, Timer timer) { - long deadlineMs = calculateDeadlineMs(timer); - latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); - applicationEventHandler.add(latest); - log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); - } - } - - private final CompositePollEventInvoker pollInvoker = new CompositePollEventInvoker(); - /** * An {@link org.apache.kafka.clients.consumer.internals.events.EventProcessor} that is created and executes in the * application thread for the purpose of processing {@link BackgroundEvent background events} generated by the @@ -396,6 +351,7 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { private Optional clientTelemetryReporter = Optional.empty(); private final AsyncConsumerApplicationThreadRequirement asyncApplicationThreadRequirement; + private final CompositePollEventInvoker pollInvoker; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -563,6 +519,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config, new StreamsRebalanceListenerInvoker(logContext, s)); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); + this.pollInvoker = new CompositePollEventInvoker( + logContext, + time, + applicationEventHandler, + this::processBackgroundEvents, + offsetCommitCallbackInvoker::executeCallbacks + ); // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, @@ -637,6 +600,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.clientTelemetryReporter = Optional.empty(); this.autoCommitEnabled = autoCommitEnabled; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.pollInvoker = new CompositePollEventInvoker( + logContext, + time, + applicationEventHandler, + this::processBackgroundEvents, + offsetCommitCallbackInvoker::executeCallbacks + ); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, time, @@ -759,6 +729,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.streamsRebalanceListenerInvoker = Optional.empty(); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = new CompletableEventReaper(logContext); + this.pollInvoker = new CompositePollEventInvoker( + logContext, + time, + applicationEventHandler, + this::processBackgroundEvents, + offsetCommitCallbackInvoker::executeCallbacks + ); } // auxiliary interface for testing @@ -929,8 +906,8 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - processBackgroundEvents(); - offsetCommitCallbackInvoker.executeCallbacks(); +// processBackgroundEvents(); +// offsetCommitCallbackInvoker.executeCallbacks(); pollInvoker.poll(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java new file mode 100644 index 0000000000000..473378a82b44c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java @@ -0,0 +1,96 @@ +/* + * 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; + +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; + +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; + +public class CompositePollEventInvoker { + + private final Logger log; + private final Time time; + private final ApplicationEventHandler applicationEventHandler; + private final Runnable backgroundEventProcessor; + private final Runnable offsetCommitProcessor; + private CompositePollEvent latest; + + public CompositePollEventInvoker(LogContext logContext, + Time time, + ApplicationEventHandler applicationEventHandler, + Runnable backgroundEventProcessor, + Runnable offsetCommitProcessor) { + this.log = logContext.logger(getClass()); + this.time = time; + this.applicationEventHandler = applicationEventHandler; + this.backgroundEventProcessor = backgroundEventProcessor; + this.offsetCommitProcessor = offsetCommitProcessor; + } + + public void poll(Timer timer) { + if (latest == null) { + log.debug("latest was null, so submitting new event..."); + submitEvent(ApplicationEvent.Type.POLL, timer); + } + + try { + log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); + + CompositePollEvent.Result result = latest.resultOrError(); + CompositePollEvent.State state = result.state(); + log.debug("Retrieved result: {}, with state: {}", result, state); + + if (state == CompositePollEvent.State.COMPLETE) { + // Make sure to clear out the latest request since it's complete. + log.debug("We're supposedly complete with event {}, so clearing...", latest); + latest = null; + } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { + log.debug("About to process background events"); + backgroundEventProcessor.run(); + log.debug("Done processing background events"); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { + log.debug("About to process offset commits"); + offsetCommitProcessor.run(); + log.debug("Done processing offset commits"); + result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + } else if (state == CompositePollEvent.State.UNKNOWN) { + throw new KafkaException("Unexpected poll result received"); + } + } catch (Throwable t) { + log.debug("Caught error, rethrowing...", t); + // If an exception is hit, bubble it up to the user but make sure to clear out the latest request + // to signify this one is complete. + latest = null; + throw ConsumerUtils.maybeWrapAsKafkaException(t); + } + } + + private void submitEvent(ApplicationEvent.Type type, Timer timer) { + long deadlineMs = calculateDeadlineMs(timer); + latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); + applicationEventHandler.add(latest); + log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 6ff8d98d1f2a9..50d0b96daea46 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -147,6 +147,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -2666,8 +2667,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept consumer.assign(Set.of(tp0)); // poll once to update with the current metadata - consumer.poll(Duration.ofMillis(0)); - TestUtils.waitForCondition(() -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), + waitForConsumerPoll(() -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), "No metadata requests sent"); client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, metadata.fetch().nodes().get(0))); @@ -2681,9 +2681,8 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept } // poll once again, which should send the list-offset request consumer.seek(tp0, 50L); - consumer.poll(Duration.ofMillis(0)); // requests: list-offset, fetch - TestUtils.waitForCondition(() -> { + waitForConsumerPoll(() -> { boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); return hasListOffsetRequest && hasFetchRequest; @@ -3817,6 +3816,20 @@ private MetricName expectedMetricName(String clientId, String config, Class c return new MetricName(NAME, "plugins", DESCRIPTION, expectedTags); } + private void waitForConsumerPoll(Supplier testCondition, String conditionDetails) { + try { + TestUtils.waitForCondition( + () -> { + consumer.poll(Duration.ZERO); + return testCondition.get(); + }, + conditionDetails + ); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + private static final String NAME = "name"; private static final String DESCRIPTION = "description"; private static final LinkedHashMap TAGS = new LinkedHashMap<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index c93fdbfd56ee0..fa00b9ad974b7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -112,6 +112,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Properties; import java.util.Set; @@ -125,7 +126,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -509,9 +512,11 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); - consumer.poll(Duration.ZERO); - assertTrue(callbackExecuted.get()); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + waitForConsumerPoll( + callbackExecuted::get, + "Consumer.poll() did not execute callback within timeout" + ); } @Test @@ -679,8 +684,11 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); - assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); + markResultForCompositePollEvent(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); + waitForConsumerPoll( + () -> callback.invoked == 1 && callback.exception == null, + "Consumer.poll() did not execute the callback once (without error) in allottec timeout" + ); } @Test @@ -1461,7 +1469,7 @@ public void testListenerCallbacksInvoke(List expectedException + Optional expectedExceptionOpt ) { consumer = newConsumer(); CounterConsumerRebalanceListener consumerRebalanceListener = new CounterConsumerRebalanceListener( @@ -1480,14 +1488,21 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); - assertEquals(expectedException.get().getMessage(), exception.getMessage()); - assertEquals(expectedException.get().getCause(), exception.getCause()); + if (expectedExceptionOpt.isPresent()) { + Exception expectedException = expectedExceptionOpt.get(); + + waitForConsumerPollException( + e -> + Objects.equals(e.getClass(), expectedException.getClass()) && + Objects.equals(e.getMessage(), expectedException.getMessage()) && + Objects.equals(e.getCause(), expectedException.getCause()) + , + "Consumer.poll() did not throw the expected exception " + expectedException + ); } else { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); @@ -1552,10 +1567,11 @@ public void testBackgroundError() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException.getMessage(), exception.getMessage()); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + waitForConsumerPollException( + e -> e.getMessage().equals(expectedException.getMessage()), + "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" + ); } @Test @@ -1572,10 +1588,11 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException1.getMessage(), exception.getMessage()); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + waitForConsumerPollException( + e -> e.getMessage().equals(expectedException1.getMessage()), + "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" + ); assertTrue(backgroundEventQueue.isEmpty()); } @@ -1849,7 +1866,12 @@ void testReaperInvokedInPoll() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); markResultForCompositePollEvent(); - consumer.poll(Duration.ZERO); + markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + + waitForConsumerPoll( + () -> backgroundEventReaper.size() == 0, + "Consumer.poll() did not reap background events within timeout" + ); verify(backgroundEventReaper).reap(time.milliseconds()); } @@ -2300,4 +2322,44 @@ private void markResultForCompositePollEvent() { doAnswer(invocation -> null) .when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } + + private void markResultForCompositePollEvent(CompositePollEvent.State state) { + doAnswer(invocation -> { + CompositePollEvent event = invocation.getArgument(0); + event.complete(state, Optional.empty()); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); + } + + private void waitForConsumerPoll(Supplier testCondition, String conditionDetails) { + try { + TestUtils.waitForCondition( + () -> { + consumer.poll(Duration.ZERO); + return testCondition.get(); + }, + conditionDetails + ); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + + private void waitForConsumerPollException(Function testCondition, String conditionDetails) { + try { + TestUtils.waitForCondition( + () -> { + try { + consumer.poll(Duration.ZERO); + return false; + } catch (KafkaException e) { + return testCondition.apply(e); + } + }, + conditionDetails + ); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } } From 23024271163855bd679fba646633a96f70a3173a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 17:26:37 -0700 Subject: [PATCH 032/123] Remove debug logging from CompositePollEventInvoker Eliminated a debug log statement that printed the result and state in CompositePollEventInvoker. This helps reduce unnecessary log output during normal operation. --- .../clients/consumer/internals/CompositePollEventInvoker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java index 473378a82b44c..2085f33153416 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; + import org.slf4j.Logger; import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; @@ -59,7 +60,6 @@ public void poll(Timer timer) { CompositePollEvent.Result result = latest.resultOrError(); CompositePollEvent.State state = result.state(); - log.debug("Retrieved result: {}, with state: {}", result, state); if (state == CompositePollEvent.State.COMPLETE) { // Make sure to clear out the latest request since it's complete. From 91e881f2b4f661be579fd83963a8eff2f665a727 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 17:26:48 -0700 Subject: [PATCH 033/123] Fix typo in CompositePollPseudoEvent class name Renamed CompositePollPsuedoEvent to CompositePollPseudoEvent in ApplicationEventProcessor to correct a spelling error and ensure consistency in class naming. --- .../internals/events/ApplicationEventProcessor.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 035d24c41a274..183d79ef90a64 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -262,7 +262,7 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { log.debug("Processing {} logic for {}", nextEventType, event); CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); - applicationEventReaper.add(new CompositePollPsuedoEvent<>(updatePositionsFuture, event.deadlineMs())); + applicationEventReaper.add(new CompositePollPseudoEvent<>(updatePositionsFuture, event.deadlineMs())); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { if (maybeFailCompositePoll(event, updatePositionsError)) @@ -933,12 +933,12 @@ public interface CompositePollApplicationThreadRequirement { Optional requirement(); } - private static class CompositePollPsuedoEvent implements CompletableEvent { + private static class CompositePollPseudoEvent implements CompletableEvent { private final CompletableFuture future; private final long deadlineMs; - public CompositePollPsuedoEvent(CompletableFuture future, long deadlineMs) { + public CompositePollPseudoEvent(CompletableFuture future, long deadlineMs) { this.future = future; this.deadlineMs = deadlineMs; } From 8b33f081ce3044776fd435c34ee591bc214ecb97 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 17:26:58 -0700 Subject: [PATCH 034/123] Refactor lambda in waitForConsumerPollException call Replaces single-line lambda with block lambda for clarity in the waitForConsumerPollException call within AsyncKafkaConsumerTest. No functional changes; improves readability. --- .../consumer/internals/AsyncKafkaConsumerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index fa00b9ad974b7..de2597cec0df1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1496,11 +1496,11 @@ public void testListenerCallbacksInvoke(List - Objects.equals(e.getClass(), expectedException.getClass()) && + e -> { + return Objects.equals(e.getClass(), expectedException.getClass()) && Objects.equals(e.getMessage(), expectedException.getMessage()) && - Objects.equals(e.getCause(), expectedException.getCause()) - , + Objects.equals(e.getCause(), expectedException.getCause()); + }, "Consumer.poll() did not throw the expected exception " + expectedException ); } else { From 2aaca8db9d0060a05a71ba65cdccacede1fcc276 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 17:27:09 -0700 Subject: [PATCH 035/123] Move pollForRecords helper method in KafkaConsumerTest Relocated the pollForRecords() helper method from its previous position to after the testPollIdleRatio method for improved code organization in KafkaConsumerTest. --- .../clients/consumer/KafkaConsumerTest.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 50d0b96daea46..16faefe0539b4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2720,22 +2720,6 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept assertEquals(OptionalLong.of(45L), consumer.currentLag(tp0)); } - @SuppressWarnings("unchecked") - private ConsumerRecords pollForRecords() { - Timer timer = time.timer(15000); - - while (timer.notExpired()) { - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); - - if (!records.isEmpty()) - return records; - } - - throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); - } - - - @ParameterizedTest @EnumSource(GroupProtocol.class) public void testListOffsetShouldUpdateSubscriptions(GroupProtocol groupProtocol) { @@ -3816,6 +3800,20 @@ private MetricName expectedMetricName(String clientId, String config, Class c return new MetricName(NAME, "plugins", DESCRIPTION, expectedTags); } + @SuppressWarnings("unchecked") + private ConsumerRecords pollForRecords() { + Timer timer = time.timer(15000); + + while (timer.notExpired()) { + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); + + if (!records.isEmpty()) + return records; + } + + throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); + } + private void waitForConsumerPoll(Supplier testCondition, String conditionDetails) { try { TestUtils.waitForCondition( From 1f1ae2453830f0c9e51eb32a7d409e7745e60b36 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 17:27:38 -0700 Subject: [PATCH 036/123] Remove commented-out event processing code Deleted commented lines for processBackgroundEvents and offsetCommitCallbackInvoker.executeCallbacks in AsyncKafkaConsumer, cleaning up unused code. --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 8cce81dcc8c55..e6f29323fef50 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -906,8 +906,6 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); -// processBackgroundEvents(); -// offsetCommitCallbackInvoker.executeCallbacks(); pollInvoker.poll(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { From 985bbd7582d7a7e61a096284aaa7982826d10552 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Sep 2025 18:31:57 -0700 Subject: [PATCH 037/123] Improve consumer poll reliability in integration tests Replaced direct assertions on consumer.poll() with TestUtils.waitUntilTrue in several integration tests to ensure expected results or exceptions are observed within a timeout. Also refactored CompositePollEventInvoker to rename 'latest' to 'inflight' for clarity and improved logging. These changes enhance test robustness and code readability. --- .../internals/CompositePollEventInvoker.java | 36 +++++++++++-------- .../api/PlaintextAdminIntegrationTest.scala | 7 ++-- ...aslClientsWithInvalidCredentialsTest.scala | 20 +++++++++-- .../server/GssapiAuthenticationTest.scala | 14 ++++++-- 4 files changed, 56 insertions(+), 21 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java index 2085f33153416..8e45cc9c240d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java @@ -35,7 +35,7 @@ public class CompositePollEventInvoker { private final ApplicationEventHandler applicationEventHandler; private final Runnable backgroundEventProcessor; private final Runnable offsetCommitProcessor; - private CompositePollEvent latest; + private CompositePollEvent inflight; public CompositePollEventInvoker(LogContext logContext, Time time, @@ -50,21 +50,27 @@ public CompositePollEventInvoker(LogContext logContext, } public void poll(Timer timer) { - if (latest == null) { - log.debug("latest was null, so submitting new event..."); + if (inflight == null) { + log.debug("No existing inflight event, submitting"); submitEvent(ApplicationEvent.Type.POLL, timer); } try { - log.debug("Attempting to retrieve result from previously submitted {} with {} remaining on timer", latest, timer.remainingMs()); + if (log.isTraceEnabled()) { + log.trace( + "Attempting to retrieve result from previously submitted {} with {} remaining on timer", + inflight, + timer.remainingMs() + ); + } - CompositePollEvent.Result result = latest.resultOrError(); + CompositePollEvent.Result result = inflight.resultOrError(); CompositePollEvent.State state = result.state(); if (state == CompositePollEvent.State.COMPLETE) { - // Make sure to clear out the latest request since it's complete. - log.debug("We're supposedly complete with event {}, so clearing...", latest); - latest = null; + // Make sure to clear out the inflight request since it's complete. + log.debug("Event {} completed, clearing inflight", inflight); + inflight = null; } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { log.debug("About to process background events"); backgroundEventProcessor.run(); @@ -79,18 +85,18 @@ public void poll(Timer timer) { throw new KafkaException("Unexpected poll result received"); } } catch (Throwable t) { - log.debug("Caught error, rethrowing...", t); - // If an exception is hit, bubble it up to the user but make sure to clear out the latest request - // to signify this one is complete. - latest = null; + // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request + // because the error effectively renders it complete. + log.debug("Event {} \"completed\" via error ({}), clearing inflight", inflight, String.valueOf(t)); + inflight = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); } } private void submitEvent(ApplicationEvent.Type type, Timer timer) { long deadlineMs = calculateDeadlineMs(timer); - latest = new CompositePollEvent(deadlineMs, time.milliseconds(), type); - applicationEventHandler.add(latest); - log.debug("Submitted new {} submitted with {} remaining on timer", latest, timer.remainingMs()); + inflight = new CompositePollEvent(deadlineMs, time.milliseconds(), type); + applicationEventHandler.add(inflight); + log.debug("Submitted new {} with {} remaining on timer", inflight, timer.remainingMs()); } } \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 1c3a7ed42e37c..631e9f8be4691 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -568,8 +568,11 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { consumer.assign(util.Set.of(tp)) consumer.seekToBeginning(util.Set.of(tp)) - val records = consumer.poll(time.Duration.ofSeconds(3)) - assertEquals(expectedNumber, records.count()) + + TestUtils.waitUntilTrue(() => { + val records = consumer.poll(time.Duration.ofSeconds(3)) + expectedNumber == records.count() + }, s"Consumer.poll() did not return the expected number of records ($expectedNumber) within the timeout") } finally consumer.close() } diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index c08c43081e6a2..d984deccec30c 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -145,13 +145,29 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { } private def verifyConsumerWithAuthenticationFailure(consumer: Consumer[Array[Byte], Array[Byte]]): Unit = { - verifyAuthenticationException(consumer.poll(Duration.ofMillis(1000))) + TestUtils.waitUntilTrue(() => { + try { + consumer.poll(Duration.ofMillis(1000)) + false + } catch { + case _: SaslAuthenticationException => true + } + }, s"Consumer.poll() did not throw a ${classOf[SaslAuthenticationException]} exception within the timeout") + verifyAuthenticationException(consumer.partitionsFor(topic)) createClientCredential() val producer = createProducer() verifyWithRetry(sendOneRecord(producer))() - verifyWithRetry(consumer.poll(Duration.ofMillis(1000)))(_.count == 1) + + val expectedNumber = 1 + TestUtils.waitUntilTrue(() => { + try { + consumer.poll(Duration.ofMillis(1000)).count == expectedNumber + } catch { + case _: SaslAuthenticationException => false + } + }, s"Consumer.poll() did not read the expected number of records ($expectedNumber) within the timeout") } @Test diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 575c612bf26a1..c4ea3df48510d 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -26,6 +26,7 @@ import javax.security.auth.login.LoginContext import kafka.api.{IntegrationTestHarness, SaslSetup} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.consumer.CloseOptions import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs @@ -185,10 +186,19 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { consumer.assign(java.util.List.of(tp)) val startMs = System.currentTimeMillis() - assertThrows(classOf[SaslAuthenticationException], () => consumer.poll(Duration.ofMillis(50))) + + TestUtils.waitUntilTrue(() => { + try { + consumer.poll(Duration.ofMillis(50)) + false + } catch { + case _: SaslAuthenticationException => true + } + }, "Client not ready or disconnected within timeout") + val endMs = System.currentTimeMillis() require(endMs - startMs < failedAuthenticationDelayMs, "Failed authentication must not be delayed on the client") - consumer.close() + consumer.close(CloseOptions.timeout(Duration.ZERO)) } /** From 5af8fc435756d3041baf4981f1fdbc3dfc72a4a3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 09:21:09 -0700 Subject: [PATCH 038/123] Updates and fixes for a couple of integration tests --- .../consumer/PlaintextConsumerCommitTest.java | 13 ++++--- .../consumer/PlaintextConsumerTest.java | 23 +++++++------ .../internals/AsyncKafkaConsumer.java | 34 ++++++++----------- .../internals/CompositePollEventInvoker.java | 25 ++++---------- .../events/ApplicationEventProcessor.java | 3 +- .../internals/events/CompositePollEvent.java | 16 ++++----- .../internals/AsyncKafkaConsumerTest.java | 12 +++---- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- 8 files changed, 57 insertions(+), 71 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java index b5bd27cf41bb3..2f305aad54cb3 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.test.api.Type; import org.apache.kafka.test.MockConsumerInterceptor; +import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; import java.time.Duration; @@ -282,11 +283,13 @@ public void onPartitionsRevoked(Collection partitions) { // In both CLASSIC and CONSUMER protocols, interceptors are executed in poll and close. // However, in the CONSUMER protocol, the assignment may be changed outside a poll, so // we need to poll once to ensure the interceptor is called. - if (groupProtocol == GroupProtocol.CONSUMER) { - consumer.poll(Duration.ZERO); - } - - assertTrue(MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance); + TestUtils.waitForCondition( + () -> { + consumer.poll(Duration.ZERO); + return MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance; + }, + "Consumer.poll() did not invoke onCommit() before timeout elapse" + ); // verify commits are intercepted on close var commitCountBeforeClose = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue(); diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index bd92f0c56851e..1782dd13c8dd8 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -1203,12 +1203,21 @@ private void testConsumingWithNullGroupId(Map consumerConfig) th consumer3.assign(List.of(TP)); consumer3.seek(TP, 1); - var numRecords1 = consumer1.poll(Duration.ofMillis(5000)).count(); + TestUtils.waitForCondition( + () -> consumer1.poll(Duration.ofMillis(5000)).count() == 3, + "consumer1 did not consume from earliest offset" + ); assertThrows(InvalidGroupIdException.class, consumer1::commitSync); assertThrows(InvalidGroupIdException.class, () -> consumer2.committed(Set.of(TP))); - var numRecords2 = consumer2.poll(Duration.ofMillis(5000)).count(); - var numRecords3 = consumer3.poll(Duration.ofMillis(5000)).count(); + TestUtils.waitForCondition( + () -> consumer2.poll(Duration.ofMillis(5000)).count() == 0, + "Expected consumer2 to consume from latest offset" + ); + TestUtils.waitForCondition( + () -> consumer3.poll(Duration.ofMillis(5000)).count() == 2, + "Expected consumer3 to consume from offset 1" + ); consumer1.unsubscribe(); consumer2.unsubscribe(); @@ -1217,14 +1226,6 @@ private void testConsumingWithNullGroupId(Map consumerConfig) th assertTrue(consumer1.assignment().isEmpty()); assertTrue(consumer2.assignment().isEmpty()); assertTrue(consumer3.assignment().isEmpty()); - - consumer1.close(); - consumer2.close(); - consumer3.close(); - - assertEquals(3, numRecords1, "Expected consumer1 to consume from earliest offset"); - assertEquals(0, numRecords2, "Expected consumer2 to consume from latest offset"); - assertEquals(2, numRecords3, "Expected consumer3 to consume from offset 1"); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e6f29323fef50..64d1429d155fb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -189,11 +189,11 @@ public AsyncConsumerApplicationThreadRequirement(BackgroundEventHandler backgrou public Optional requirement() { // If there are background events to process, exit to the application thread. if (backgroundEventHandler.size() > 0) - return Optional.of(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + return Optional.of(CompositePollEvent.State.CALLBACKS_REQUIRED); // If there are enqueued callbacks to invoke, exit to the application thread. if (offsetCommitCallbackInvoker.size() > 0) - return Optional.of(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); + return Optional.of(CompositePollEvent.State.CALLBACKS_REQUIRED); return Optional.empty(); } @@ -523,8 +523,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, logContext, time, applicationEventHandler, - this::processBackgroundEvents, - offsetCommitCallbackInvoker::executeCallbacks + () -> { + processBackgroundEvents(); + offsetCommitCallbackInvoker.executeCallbacks(); + } ); // The FetchCollector is only used on the application thread. @@ -604,8 +606,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, logContext, time, applicationEventHandler, - this::processBackgroundEvents, - offsetCommitCallbackInvoker::executeCallbacks + () -> { + processBackgroundEvents(); + offsetCommitCallbackInvoker.executeCallbacks(); + } ); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, @@ -733,8 +737,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, logContext, time, applicationEventHandler, - this::processBackgroundEvents, - offsetCommitCallbackInvoker::executeCallbacks + () -> { + processBackgroundEvents(); + offsetCommitCallbackInvoker.executeCallbacks(); + } ); } @@ -1855,19 +1861,9 @@ private Fetch pollForFetches(Timer timer) { * of the {@link #fetchBuffer}, converting it to a well-formed {@link CompletedFetch}, validating that it and * the internal {@link SubscriptionState state} are correct, and then converting it all into a {@link Fetch} * for returning. - * - *

- * - * This method will {@link ConsumerNetworkThread#wakeup() wake up the network thread} before returning. This is - * done as an optimization so that the next round of data can be pre-fetched. */ private Fetch collectFetch() { - final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); - - // Notify the network thread to wake up and start the next round of fetching. - applicationEventHandler.wakeupNetworkThread(); - - return fetch; + return fetchCollector.collectFetch(fetchBuffer); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java index 8e45cc9c240d5..3643b75045898 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -33,20 +32,17 @@ public class CompositePollEventInvoker { private final Logger log; private final Time time; private final ApplicationEventHandler applicationEventHandler; - private final Runnable backgroundEventProcessor; - private final Runnable offsetCommitProcessor; + private final Runnable applicationThreadCallbacks; private CompositePollEvent inflight; public CompositePollEventInvoker(LogContext logContext, Time time, ApplicationEventHandler applicationEventHandler, - Runnable backgroundEventProcessor, - Runnable offsetCommitProcessor) { + Runnable applicationThreadCallbacks) { this.log = logContext.logger(getClass()); this.time = time; this.applicationEventHandler = applicationEventHandler; - this.backgroundEventProcessor = backgroundEventProcessor; - this.offsetCommitProcessor = offsetCommitProcessor; + this.applicationThreadCallbacks = applicationThreadCallbacks; } public void poll(Timer timer) { @@ -71,18 +67,11 @@ public void poll(Timer timer) { // Make sure to clear out the inflight request since it's complete. log.debug("Event {} completed, clearing inflight", inflight); inflight = null; - } else if (state == CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED) { - log.debug("About to process background events"); - backgroundEventProcessor.run(); - log.debug("Done processing background events"); + } else if (state == CompositePollEvent.State.CALLBACKS_REQUIRED) { + log.debug("About to process callbacks"); + applicationThreadCallbacks.run(); + log.debug("Done processing callbacks"); result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } else if (state == CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED) { - log.debug("About to process offset commits"); - offsetCommitProcessor.run(); - log.debug("Done processing offset commits"); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); - } else if (state == CompositePollEvent.State.UNKNOWN) { - throw new KafkaException("Unexpected poll result received"); } } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 183d79ef90a64..c82b8f98bc6f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -283,8 +283,7 @@ private void process(final CompositePollEvent event) { return; } - log.warn("Unknown next step for composite poll: {}", nextEventType); - event.complete(CompositePollEvent.State.UNKNOWN, Optional.empty()); + event.completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); } private boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 4796a78bb7923..e3304f2106dd9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -26,16 +26,14 @@ public class CompositePollEvent extends ApplicationEvent { public enum State { - OFFSET_COMMIT_CALLBACKS_REQUIRED, - BACKGROUND_EVENT_PROCESSING_REQUIRED, - INCOMPLETE, - COMPLETE, - UNKNOWN + CALLBACKS_REQUIRED, + IN_PROGRESS, + COMPLETE } public static class Result { - private static final Result INCOMPLETE = new Result(State.INCOMPLETE, Optional.empty()); + private static final Result IN_PROGRESS = new Result(State.IN_PROGRESS, Optional.empty()); private final State state; private final Optional nextEventType; @@ -81,7 +79,7 @@ public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; - this.resultOrError = new AtomicReference<>(Result.INCOMPLETE); + this.resultOrError = new AtomicReference<>(Result.IN_PROGRESS); } public long deadlineMs() { @@ -111,11 +109,11 @@ public void complete(State state, Optional nextEventType) { Objects.requireNonNull(nextEventType) ); - resultOrError.compareAndSet(Result.INCOMPLETE, result); + resultOrError.compareAndSet(Result.IN_PROGRESS, result); } public void completeExceptionally(KafkaException e) { - resultOrError.compareAndSet(Result.INCOMPLETE, Objects.requireNonNull(e)); + resultOrError.compareAndSet(Result.IN_PROGRESS, Objects.requireNonNull(e)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index de2597cec0df1..0e7df79022351 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -512,7 +512,7 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( callbackExecuted::get, "Consumer.poll() did not execute callback within timeout" @@ -684,7 +684,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.OFFSET_COMMIT_CALLBACKS_REQUIRED); + markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( () -> callback.invoked == 1 && callback.exception == null, "Consumer.poll() did not execute the callback once (without error) in allottec timeout" @@ -1488,7 +1488,7 @@ public void testListenerCallbacksInvoke(List e.getMessage().equals(expectedException.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" @@ -1588,7 +1588,7 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPollException( e -> e.getMessage().equals(expectedException1.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" @@ -1866,7 +1866,7 @@ void testReaperInvokedInPoll() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); markResultForCompositePollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.BACKGROUND_EVENT_PROCESSING_REQUIRED); + markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( () -> backgroundEventReaper.size() == 0, diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 4f09abc89dd65..f950362354c8c 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -1358,7 +1358,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val consumer = createConsumer() consumer.assign(java.util.List.of(tp)) - assertThrows(classOf[AuthorizationException], () => consumeRecords(consumer)) + assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) From 52c08455c1a66c8a441c5894946a852a8555ae19 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 11:56:40 -0700 Subject: [PATCH 039/123] Refactoring and clean up --- .../consumer/PlaintextConsumerCommitTest.java | 2 +- .../internals/AsyncKafkaConsumer.java | 57 ++--- .../internals/FetchRequestManager.java | 3 - .../consumer/internals/ShareConsumerImpl.java | 10 +- .../events/ApplicationEventProcessor.java | 230 ++++++------------ .../CompositePollEventInvoker.java | 6 +- .../CompositePollEventProcessorContext.java | 142 +++++++++++ .../events/ApplicationEventProcessorTest.java | 10 +- 8 files changed, 240 insertions(+), 220 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{ => events}/CompositePollEventInvoker.java (92%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java index 2f305aad54cb3..329bba87c6b73 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java @@ -28,8 +28,8 @@ import org.apache.kafka.common.test.api.ClusterTestDefaults; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.test.MockConsumerInterceptor; - import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.BeforeEach; import java.time.Duration; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 64d1429d155fb..e98e631b2d396 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -49,7 +49,8 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEventInvoker; +import org.apache.kafka.clients.consumer.internals.events.CompositePollEventProcessorContext; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent; @@ -174,31 +175,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private static final long NO_CURRENT_THREAD = -1L; - private static class AsyncConsumerApplicationThreadRequirement implements ApplicationEventProcessor.CompositePollApplicationThreadRequirement { - - private final BackgroundEventHandler backgroundEventHandler; - private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; - - public AsyncConsumerApplicationThreadRequirement(BackgroundEventHandler backgroundEventHandler, - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker) { - this.backgroundEventHandler = backgroundEventHandler; - this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; - } - - @Override - public Optional requirement() { - // If there are background events to process, exit to the application thread. - if (backgroundEventHandler.size() > 0) - return Optional.of(CompositePollEvent.State.CALLBACKS_REQUIRED); - - // If there are enqueued callbacks to invoke, exit to the application thread. - if (offsetCommitCallbackInvoker.size() > 0) - return Optional.of(CompositePollEvent.State.CALLBACKS_REQUIRED); - - return Optional.empty(); - } - } - /** * An {@link org.apache.kafka.clients.consumer.internals.events.EventProcessor} that is created and executes in the * application thread for the purpose of processing {@link BackgroundEvent background events} generated by the @@ -350,7 +326,6 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - private final AsyncConsumerApplicationThreadRequirement asyncApplicationThreadRequirement; private final CompositePollEventInvoker pollInvoker; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; @@ -487,17 +462,18 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( - backgroundEventHandler, - offsetCommitCallbackInvoker + Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + logContext, + networkClientDelegateSupplier, + backgroundEventHandler, + offsetCommitCallbackInvoker, + applicationEventReaper ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, - asyncApplicationThreadRequirement, - applicationEventReaper + Optional.of(compositePollContextSupplier) ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, @@ -616,10 +592,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, asyncConsumerMetrics ); - this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( - backgroundEventHandler, - offsetCommitCallbackInvoker - ); } AsyncKafkaConsumer(LogContext logContext, @@ -709,18 +681,19 @@ public AsyncKafkaConsumer(final ConsumerConfig config, Optional.empty() ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - this.asyncApplicationThreadRequirement = new AsyncConsumerApplicationThreadRequirement( + Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + logContext, + networkClientDelegateSupplier, backgroundEventHandler, - offsetCommitCallbackInvoker + offsetCommitCallbackInvoker, + applicationEventReaper ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, subscriptions, requestManagersSupplier, - networkClientDelegateSupplier, - asyncApplicationThreadRequirement, - applicationEventReaper + Optional.of(compositePollContextSupplier) ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index 5eaef82388b34..c52b5453e21d9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -77,9 +77,6 @@ protected void maybeThrowAuthFailure(Node node) { * @return Future on which the caller can wait to ensure that the requests have been created */ public CompletableFuture createFetchRequests() { - if (!fetchBuffer.isEmpty()) - return CompletableFuture.completedFuture(null); - CompletableFuture future = new CompletableFuture<>(); if (pendingFetchRequestFuture != null) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 5b37407a178c6..cc6efc67e98e6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -302,10 +302,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, metadata, subscriptions, - requestManagersSupplier, - networkClientDelegateSupplier, - Optional::empty, - applicationEventReaper + requestManagersSupplier ); this.applicationEventHandler = applicationEventHandlerFactory.build( @@ -412,10 +409,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, metadata, subscriptions, - requestManagersSupplier, - networkClientDelegateSupplier, - Optional::empty, - applicationEventReaper + requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index c82b8f98bc6f3..db93b82c7676d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,8 +22,6 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; @@ -48,7 +46,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -64,25 +61,19 @@ public class ApplicationEventProcessor implements EventProcessor compositePollContext; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final NetworkClientDelegate networkClientDelegate, - final CompositePollApplicationThreadRequirement compositePollApplicationThreadRequirement, - final CompletableEventReaper applicationEventReaper) { + final Optional compositePollContext) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; - this.networkClientDelegate = networkClientDelegate; - this.compositePollApplicationThreadRequirement = compositePollApplicationThreadRequirement; - this.applicationEventReaper = applicationEventReaper; + this.compositePollContext = compositePollContext; this.metadataVersionSnapshot = metadata.updateVersion(); } @@ -235,104 +226,6 @@ public void process(ApplicationEvent event) { } } - private void process(final CompositePollEvent event) { - if (maybeFailCompositePoll(event) || maybePauseCompositePoll(event, ApplicationEvent.Type.POLL)) - return; - - ApplicationEvent.Type nextEventType = event.nextEventType(); - - if (nextEventType == ApplicationEvent.Type.POLL) { - log.debug("Processing {} logic for {}", nextEventType, event); - processPollEvent(event.pollTimeMs()); - nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - - if (maybeFailCompositePoll(event) || maybePauseCompositePoll(event, nextEventType)) - return; - } - - if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("Processing {} logic for {}", nextEventType, event); - processUpdatePatternSubscriptionEvent(); - nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - - if (maybeFailCompositePoll(event) || maybePauseCompositePoll(event, nextEventType)) - return; - } - - if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.debug("Processing {} logic for {}", nextEventType, event); - CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); - applicationEventReaper.add(new CompositePollPseudoEvent<>(updatePositionsFuture, event.deadlineMs())); - - updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeFailCompositePoll(event, updatePositionsError)) - return; - - log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); - - // If needed, create a fetch request if there's no data in the FetchBuffer. - requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeFailCompositePoll(event, fetchError)) - return; - - event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); - log.debug("Completed CompositePollEvent {}", event); - }); - }); - - return; - } - - event.completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); - } - - private boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { - Optional stateOpt = compositePollApplicationThreadRequirement.requirement(); - - if (stateOpt.isPresent()) { - CompositePollEvent.State state = stateOpt.get(); - log.debug("Pausing event processing for {} with {} as next step", state, nextEventType); - event.complete(state, Optional.of(nextEventType)); - return true; - } - - return false; - } - - private boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { - if (maybeFailCompositePoll(event)) - return true; - - if (t == null) - return false; - - if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { - log.debug("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); - return false; - } - - if (t instanceof CompletionException) { - t = t.getCause(); - } - - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - event.completeExceptionally(e); - log.debug("Failing event processing for {}", event, e); - return true; - } - - private boolean maybeFailCompositePoll(CompositePollEvent event) { - Optional exception = networkClientDelegate.getAndClearMetadataError(); - - if (exception.isPresent()) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(exception.get()); - event.completeExceptionally(e); - log.debug("Failing event processing for {}", event, e); - return true; - } - - return false; - } private void process(final PollEvent event) { processPollEvent(event.pollTimeMs()); @@ -811,6 +704,59 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { requestManagers.streamsMembershipManager.get().onAllTasksLostCallbackCompleted(event); } + private void process(final CompositePollEvent event) { + CompositePollEventProcessorContext context = compositePollContext.orElseThrow(IllegalArgumentException::new); + + if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, ApplicationEvent.Type.POLL)) + return; + + ApplicationEvent.Type nextEventType = event.nextEventType(); + + if (nextEventType == ApplicationEvent.Type.POLL) { + log.debug("Processing {} logic for {}", nextEventType, event); + processPollEvent(event.pollTimeMs()); + nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + + if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, nextEventType)) + return; + } + + if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { + log.debug("Processing {} logic for {}", nextEventType, event); + processUpdatePatternSubscriptionEvent(); + nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + + if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, nextEventType)) + return; + } + + if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { + log.debug("Processing {} logic for {}", nextEventType, event); + CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); + context.trackExpirableEvent(updatePositionsFuture, event.deadlineMs()); + + updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + if (context.maybeFailCompositePoll(event, updatePositionsError)) + return; + + log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); + + // Create a fetch request if there's no data in the FetchBuffer. + requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { + if (context.maybeFailCompositePoll(event, fetchError)) + return; + + event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); + log.debug("Completed CompositePollEvent {}", event); + }); + }); + + return; + } + + event.completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); + } + private BiConsumer complete(final CompletableFuture b) { return (value, exception) -> { if (exception != null) @@ -828,28 +774,40 @@ public static Supplier supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, - final Supplier networkClientDelegateSupplier, - final CompositePollApplicationThreadRequirement applicationThreadRequirement, - final CompletableEventReaper applicationEventReaper) { + final Optional> compositePollEventProcessorContextSupplier) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); - NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - + Optional compositePollContext = compositePollEventProcessorContextSupplier.map(Supplier::get); return new ApplicationEventProcessor( logContext, requestManagers, metadata, subscriptions, - networkClientDelegate, - applicationThreadRequirement, - applicationEventReaper + compositePollContext ); } }; } + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ + public static Supplier supplier(final LogContext logContext, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final Supplier requestManagersSupplier) { + return supplier( + logContext, + metadata, + subscriptions, + requestManagersSupplier, + Optional.empty() + ); + } + /** * This function evaluates the regex that the consumer subscribed to * against the list of topic names from metadata, and updates @@ -921,40 +879,4 @@ private void processUpdatePatternSubscriptionEvent() { private CompletableFuture processCheckAndUpdatePositionsEvent(final long deadlineMs) { return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs); } - - /** - * This interface exists mostly to make the code more intuitive. When {@link #requirement()} - * returns true, the {@link CompositePollEvent} processing needs to be interrupted so that processing - * can return to the application thread. - */ - public interface CompositePollApplicationThreadRequirement { - - Optional requirement(); - } - - private static class CompositePollPseudoEvent implements CompletableEvent { - - private final CompletableFuture future; - private final long deadlineMs; - - public CompositePollPseudoEvent(CompletableFuture future, long deadlineMs) { - this.future = future; - this.deadlineMs = deadlineMs; - } - - @Override - public CompletableFuture future() { - return future; - } - - @Override - public long deadlineMs() { - return deadlineMs; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{future=" + future + ", deadlineMs=" + deadlineMs + '}'; - } - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java similarity index 92% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java index 3643b75045898..8b927acc22ac2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java @@ -14,11 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.clients.consumer.internals; +package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java new file mode 100644 index 0000000000000..7ce050a7bbc13 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java @@ -0,0 +1,142 @@ +/* + * 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.CachedSupplier; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; +import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.LogContext; + +import org.slf4j.Logger; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.Supplier; + +public class CompositePollEventProcessorContext { + + private final Logger log; + private final NetworkClientDelegate networkClientDelegate; + private final BackgroundEventHandler backgroundEventHandler; + private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; + private final CompletableEventReaper applicationEventReaper; + + private CompositePollEventProcessorContext(LogContext logContext, + NetworkClientDelegate networkClientDelegate, + BackgroundEventHandler backgroundEventHandler, + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + CompletableEventReaper applicationEventReaper) { + this.log = logContext.logger(getClass()); + this.networkClientDelegate = networkClientDelegate; + this.backgroundEventHandler = backgroundEventHandler; + this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; + this.applicationEventReaper = applicationEventReaper; + } + + public static Supplier supplier(LogContext logContext, + Supplier networkClientDelegateSupplier, + BackgroundEventHandler backgroundEventHandler, + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + CompletableEventReaper applicationEventReaper) { + return new CachedSupplier<>() { + @Override + protected CompositePollEventProcessorContext create() { + NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); + + return new CompositePollEventProcessorContext( + logContext, + networkClientDelegate, + backgroundEventHandler, + offsetCommitCallbackInvoker, + applicationEventReaper + ); + } + }; + }; + + public void trackExpirableEvent(CompletableFuture future, long deadlineMs) { + CompletableEvent event = new CompletableEvent<>() { + @Override + public CompletableFuture future() { + return future; + } + + @Override + public long deadlineMs() { + return deadlineMs; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{future=" + future + ", deadlineMs=" + deadlineMs + '}'; + } + }; + + applicationEventReaper.add(event); + } + + public boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { + // If there are background events to process or enqueued callbacks to invoke, exit to + // the application thread. + if (backgroundEventHandler.size() > 0 || offsetCommitCallbackInvoker.size() > 0) { + CompositePollEvent.State state = CompositePollEvent.State.CALLBACKS_REQUIRED; + log.debug("Pausing event processing for {} with {} as next step", state, nextEventType); + event.complete(state, Optional.of(nextEventType)); + return true; + } + + return false; + } + + public boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { + if (maybeFailCompositePoll(event)) + return true; + + if (t == null) + return false; + + if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { + log.debug("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); + return false; + } + + if (t instanceof CompletionException) { + t = t.getCause(); + } + + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + event.completeExceptionally(e); + log.debug("Failing event processing for {}", event, e); + return true; + } + + public boolean maybeFailCompositePoll(CompositePollEvent event) { + Optional exception = networkClientDelegate.getAndClearMetadataError(); + + if (exception.isPresent()) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(exception.get()); + event.completeExceptionally(e); + log.debug("Failing event processing for {}", event, e); + return true; + } + + return false; + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index dcf7ea03bc9e6..9a9bfe36e59a2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -92,8 +92,6 @@ public class ApplicationEventProcessorTest { private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock(StreamsGroupHeartbeatRequestManager.class); private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); - private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); - private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private ApplicationEventProcessor processor; private void setupProcessor(boolean withGroupId) { @@ -114,9 +112,7 @@ private void setupProcessor(boolean withGroupId) { requestManagers, metadata, subscriptionState, - networkClientDelegate, - Optional::empty, - applicationEventReaper + Optional.empty() ); } @@ -138,9 +134,7 @@ private void setupStreamProcessor(boolean withGroupId) { requestManagers, metadata, subscriptionState, - networkClientDelegate, - Optional::empty, - applicationEventReaper + Optional.empty() ); } From bfcd7ec0f80e9c6b51ea3b45f202f7f42e83e602 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 12:16:54 -0700 Subject: [PATCH 040/123] More clean up and refactoring --- .../internals/AsyncKafkaConsumer.java | 10 +++---- .../internals/NetworkClientDelegate.java | 12 ++++----- .../OffsetCommitCallbackInvoker.java | 6 +++++ .../consumer/internals/ShareConsumerImpl.java | 6 ++--- .../events/BackgroundEventHandler.java | 6 +++++ .../internals/AsyncKafkaConsumerTest.java | 26 +------------------ .../events/ApplicationEventProcessorTest.java | 2 +- 7 files changed, 27 insertions(+), 41 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e98e631b2d396..c552cc3bbd74d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -462,7 +462,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + final Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( logContext, networkClientDelegateSupplier, backgroundEventHandler, @@ -500,8 +500,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, applicationEventHandler, () -> { - processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); + processBackgroundEvents(); } ); @@ -583,8 +583,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, applicationEventHandler, () -> { - processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); + processBackgroundEvents(); } ); this.backgroundEventHandler = new BackgroundEventHandler( @@ -681,7 +681,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, Optional.empty() ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + final Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( logContext, networkClientDelegateSupplier, backgroundEventHandler, @@ -711,8 +711,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, applicationEventHandler, () -> { - processBackgroundEvents(); offsetCommitCallbackInvoker.executeCallbacks(); + processBackgroundEvents(); } ); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 5f71cd3fbc74f..31c402df2a6db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -51,7 +51,6 @@ import java.util.Optional; import java.util.Queue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Supplier; @@ -71,7 +70,7 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private final AtomicReference metadataError; + private Optional metadataError; private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; @@ -92,7 +91,7 @@ public NetworkClientDelegate( this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - this.metadataError = new AtomicReference<>(); + this.metadataError = Optional.empty(); this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -164,7 +163,7 @@ private void maybePropagateMetadataError() { if (notifyMetadataErrorsViaErrorQueue) { backgroundEventHandler.add(new ErrorEvent(e)); } else { - metadataError.compareAndSet(null, e); + metadataError = Optional.of(e); } } } @@ -250,8 +249,9 @@ private ClientRequest makeClientRequest( } public Optional getAndClearMetadataError() { - Exception exception = metadataError.getAndSet(null); - return Optional.ofNullable(exception); + Optional metadataError = this.metadataError; + this.metadataError = Optional.empty(); + return metadataError; } public Node leastLoadedNode() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java index acc0f277d0d0e..77b8f6f81fe72 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java @@ -50,6 +50,12 @@ public void enqueueInterceptorInvocation(final Mappause + * itself to return to the application thread for processing. + * + * @return Current size of queue + */ public int size() { return callbackQueue.size(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index cc6efc67e98e6..ecdc9b8a04844 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -297,7 +297,6 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { clientTelemetryReporter, metrics ); - final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, @@ -309,7 +308,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, time, applicationEventQueue, - applicationEventReaper, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -404,7 +403,6 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { metrics ); - final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, @@ -416,7 +414,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { logContext, time, applicationEventQueue, - applicationEventReaper, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index ab790ba20556d..935220bf70d62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -45,6 +45,12 @@ public BackgroundEventHandler(final BlockingQueue backgroundEve this.asyncConsumerMetrics = asyncConsumerMetrics; } + /** + * Returns the current size of the queue. Used by the background thread to determine if it needs to pause + * itself to return to the application thread for processing. + * + * @return Current size of queue + */ public int size() { return backgroundEventQueue.size(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 0e7df79022351..d8ae8183f8683 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -428,7 +428,6 @@ public void testWakeupBeforeCallingPoll() { consumer.wakeup(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -449,7 +448,6 @@ public void testWakeupAfterEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -474,7 +472,6 @@ public void testWakeupAfterNonEmptyFetch() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); // the previously ignored wake-up should not be ignored in the next call @@ -512,7 +509,6 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( callbackExecuted::get, "Consumer.poll() did not execute callback within timeout" @@ -537,7 +533,6 @@ public void testClearWakeupTriggerAfterPoll() { consumer.assign(singleton(tp)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); @@ -684,7 +679,6 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( () -> callback.invoked == 1 && callback.exception == null, "Consumer.poll() did not execute the callback once (without error) in allottec timeout" @@ -1488,7 +1482,6 @@ public void testListenerCallbacksInvoke(List e.getMessage().equals(expectedException.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" @@ -1588,7 +1580,6 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPollException( e -> e.getMessage().equals(expectedException1.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" @@ -1673,7 +1664,6 @@ public void testEnsurePollEventSentOnConsumerPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); consumer.poll(Duration.ofMillis(100)); verify(applicationEventHandler).add(any(CompositePollEvent.class)); } @@ -1692,7 +1682,6 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); } @@ -1727,7 +1716,6 @@ public void testLongPollWaitIsLimited() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); assertEquals(2, returnedRecords.count()); @@ -1832,7 +1820,6 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { try { Thread.currentThread().interrupt(); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); assertThrows(InterruptException.class, () -> consumer.poll(Duration.ZERO)); } finally { // clear interrupted state again since this thread may be reused by JUnit @@ -1865,8 +1852,6 @@ void testReaperInvokedInPoll() { consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); - markResultForCompositePollEvent(CompositePollEvent.State.CALLBACKS_REQUIRED); waitForConsumerPoll( () -> backgroundEventReaper.size() == 0, @@ -1934,7 +1919,6 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer.assign(singleton(new TopicPartition("topic1", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - markResultForCompositePollEvent(); consumer.poll(Duration.ZERO); verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); @@ -2316,17 +2300,9 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { event.markReconcileAndAutoCommitComplete(); return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); - } - - private void markResultForCompositePollEvent() { - doAnswer(invocation -> null) - .when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); - } - - private void markResultForCompositePollEvent(CompositePollEvent.State state) { doAnswer(invocation -> { CompositePollEvent event = invocation.getArgument(0); - event.complete(state, Optional.empty()); + event.complete(CompositePollEvent.State.CALLBACKS_REQUIRED, Optional.empty()); return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 9a9bfe36e59a2..d7feb04930a32 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -81,7 +81,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@SuppressWarnings({"ClassDataAbstractionCoupling", "ClassFanOutComplexity"}) +@SuppressWarnings("ClassDataAbstractionCoupling") public class ApplicationEventProcessorTest { private final Time time = new MockTime(); private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); From f45b70e6886eef97ca5d0a838ef39231d354ffc0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 12:26:11 -0700 Subject: [PATCH 041/123] Refactoring --- .../events/ApplicationEventProcessor.java | 124 +++++++++--------- .../CompositePollEventProcessorContext.java | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 1 - 3 files changed, 66 insertions(+), 61 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index db93b82c7676d..6314f762b1408 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -226,10 +226,35 @@ public void process(ApplicationEvent event) { } } - private void process(final PollEvent event) { - processPollEvent(event.pollTimeMs()); - event.markReconcileAndAutoCommitComplete(); + // Trigger a reconciliation that can safely commit offsets if needed to rebalance, + // as we're processing before any new fetching starts in the app thread + requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> + consumerMembershipManager.maybeReconcile(true)); + if (requestManagers.commitRequestManager.isPresent()) { + CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); + commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); + // all commit request generation points have been passed, + // so it's safe to notify the app thread could proceed and start fetching + event.markReconcileAndAutoCommitComplete(); + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + } else { + // safe to unblock - no auto-commit risk here: + // 1. commitRequestManager is not present + // 2. shareConsumer has no auto-commit mechanism + event.markReconcileAndAutoCommitComplete(); + requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + } } private void process(final CreateFetchRequestsEvent event) { @@ -393,7 +418,12 @@ private void process(final TopicRe2JPatternSubscriptionChangeEvent event) { * This will make the consumer send the updated subscription on the next poll. */ private void process(final UpdatePatternSubscriptionEvent event) { - processUpdatePatternSubscriptionEvent(); + if (subscriptions.hasPatternSubscription()) { + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + updatePatternSubscription(metadata.fetch()); + } + } event.future().complete(null); } @@ -435,7 +465,7 @@ private void process(final ResetOffsetEvent event) { * them to update positions in the subscription state. */ private void process(final CheckAndUpdatePositionsEvent event) { - CompletableFuture future = processCheckAndUpdatePositionsEvent(event.deadlineMs()); + CompletableFuture future = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -714,16 +744,34 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.POLL) { log.debug("Processing {} logic for {}", nextEventType, event); - processPollEvent(event.pollTimeMs()); - nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - - if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, nextEventType)) - return; - } - - if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("Processing {} logic for {}", nextEventType, event); - processUpdatePatternSubscriptionEvent(); + long pollTimeMs = event.pollTimeMs(); + + // Trigger a reconciliation that can safely commit offsets if needed to rebalance, + // as we're processing before any new fetching starts in the app thread + requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> + consumerMembershipManager.maybeReconcile(true)); + if (requestManagers.commitRequestManager.isPresent()) { + CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); + commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); + // all commit request generation points have been passed, + // so it's safe to notify the app thread could proceed and start fetching + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + } else { + // safe to unblock - no auto-commit risk here: + // 1. commitRequestManager is not present + // 2. shareConsumer has no auto-commit mechanism + requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(pollTimeMs); + }); + } nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, nextEventType)) @@ -732,8 +780,8 @@ private void process(final CompositePollEvent event) { if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { log.debug("Processing {} logic for {}", nextEventType, event); - CompletableFuture updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs()); - context.trackExpirableEvent(updatePositionsFuture, event.deadlineMs()); + CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); + context.trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { if (context.maybeFailCompositePoll(event, updatePositionsError)) @@ -837,46 +885,4 @@ private void updatePatternSubscription(Cluster cluster) { int metadataVersionSnapshot() { return metadataVersionSnapshot; } - - private void processPollEvent(final long pollTimeMs) { - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, - // as we're processing before any new fetching starts in the app thread - requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> - consumerMembershipManager.maybeReconcile(true)); - if (requestManagers.commitRequestManager.isPresent()) { - CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); - // all commit request generation points have been passed, - // so it's safe to notify the app thread could proceed and start fetching - requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } else { - // safe to unblock - no auto-commit risk here: - // 1. commitRequestManager is not present - // 2. shareConsumer has no auto-commit mechanism - requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } - } - - private void processUpdatePatternSubscriptionEvent() { - if (subscriptions.hasPatternSubscription()) { - if (this.metadataVersionSnapshot < metadata.updateVersion()) { - this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(metadata.fetch()); - } - } - } - - private CompletableFuture processCheckAndUpdatePositionsEvent(final long deadlineMs) { - return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs); - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java index 7ce050a7bbc13..646bca4e8e17c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java @@ -71,7 +71,7 @@ protected CompositePollEventProcessorContext create() { }; }; - public void trackExpirableEvent(CompletableFuture future, long deadlineMs) { + public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture future, long deadlineMs) { CompletableEvent event = new CompletableEvent<>() { @Override public CompletableFuture future() { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 631e9f8be4691..b7d33d088579f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -568,7 +568,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { consumer.assign(util.Set.of(tp)) consumer.seekToBeginning(util.Set.of(tp)) - TestUtils.waitUntilTrue(() => { val records = consumer.poll(time.Duration.ofSeconds(3)) expectedNumber == records.count() From 8235ed22569561ef7e6d5fe351668b92a0912980 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 12:58:07 -0700 Subject: [PATCH 042/123] Refactoring and clean up --- .../consumer/ConsumerPollTestUtils.java | 81 +++++++++++++++++++ .../clients/consumer/KafkaConsumerTest.java | 63 +++++---------- .../internals/AsyncKafkaConsumerTest.java | 62 ++++---------- ...aslClientsWithInvalidCredentialsTest.scala | 14 ++-- .../server/GssapiAuthenticationTest.scala | 7 +- 5 files changed, 126 insertions(+), 101 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java new file mode 100644 index 0000000000000..d4a6b0957643c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java @@ -0,0 +1,81 @@ +/* + * 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; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.test.TestUtils; + +import java.time.Duration; +import java.util.function.Function; +import java.util.function.Supplier; + +public class ConsumerPollTestUtils { + + @SuppressWarnings("unchecked") + public static ConsumerRecords waitForRecords(Consumer consumer, Time time) { + Timer timer = time.timer(15000); + + while (timer.notExpired()) { + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); + + if (!records.isEmpty()) + return records; + } + + throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); + } + + public static void waitForCondition(Consumer consumer, + Supplier testCondition, + String conditionDetails) { + try { + TestUtils.waitForCondition( + () -> { + consumer.poll(Duration.ZERO); + return testCondition.get(); + }, + conditionDetails + ); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + + public static void waitForException(Consumer consumer, + Function testCondition, + String conditionDetails) { + try { + TestUtils.waitForCondition( + () -> { + try { + consumer.poll(Duration.ZERO); + return false; + } catch (KafkaException e) { + return testCondition.apply(e); + } + }, + conditionDetails + ); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 16faefe0539b4..3d864a9300aee 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -102,7 +102,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.MockConsumerInterceptor; import org.apache.kafka.test.MockDeserializer; @@ -147,7 +146,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -952,7 +950,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol g client.prepareResponse(listOffsetsResponse(Map.of(tp0, 50L))); client.prepareResponse(fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = pollForRecords(); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -1046,7 +1044,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = pollForRecords(); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1765,7 +1763,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = pollForRecords(); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); @@ -1824,7 +1822,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = pollForRecords(); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -2119,7 +2117,7 @@ public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol gro time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); - final ConsumerRecords records = pollForRecords(); + final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertFalse(records.isEmpty()); assertFalse(records.nextOffsets().isEmpty()); } @@ -2667,8 +2665,11 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept consumer.assign(Set.of(tp0)); // poll once to update with the current metadata - waitForConsumerPoll(() -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), - "No metadata requests sent"); + ConsumerPollTestUtils.waitForCondition( + consumer, + () -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), + "No metadata requests sent" + ); client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, metadata.fetch().nodes().get(0))); // no error for no current position @@ -2682,11 +2683,15 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // poll once again, which should send the list-offset request consumer.seek(tp0, 50L); // requests: list-offset, fetch - waitForConsumerPoll(() -> { - boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); - boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); - return hasListOffsetRequest && hasFetchRequest; - }, "No list-offset & fetch request sent"); + ConsumerPollTestUtils.waitForCondition( + consumer, + () -> { + boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); + boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); + return hasListOffsetRequest && hasFetchRequest; + }, + "No list-offset & fetch request sent" + ); // no error for no end offset (so unknown lag) assertEquals(OptionalLong.empty(), consumer.currentLag(tp0)); @@ -2710,7 +2715,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept final FetchInfo fetchInfo = new FetchInfo(1L, 99L, 50L, 5); client.respondToRequest(fetchRequest, fetchResponse(Map.of(tp0, fetchInfo))); - final ConsumerRecords records = pollForRecords(); + final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -3800,34 +3805,6 @@ private MetricName expectedMetricName(String clientId, String config, Class c return new MetricName(NAME, "plugins", DESCRIPTION, expectedTags); } - @SuppressWarnings("unchecked") - private ConsumerRecords pollForRecords() { - Timer timer = time.timer(15000); - - while (timer.notExpired()) { - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); - - if (!records.isEmpty()) - return records; - } - - throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); - } - - private void waitForConsumerPoll(Supplier testCondition, String conditionDetails) { - try { - TestUtils.waitForCondition( - () -> { - consumer.poll(Duration.ZERO); - return testCondition.get(); - }, - conditionDetails - ); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } - private static final String NAME = "name"; private static final String DESCRIPTION = "description"; private static final LinkedHashMap TAGS = new LinkedHashMap<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index d8ae8183f8683..98a7c34cc637d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.CloseOptions; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerPollTestUtils; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -126,9 +127,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -509,7 +508,8 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); markReconcileAndAutoCommitCompleteForPollEvent(); - waitForConsumerPoll( + ConsumerPollTestUtils.waitForCondition( + consumer, callbackExecuted::get, "Consumer.poll() did not execute callback within timeout" ); @@ -679,7 +679,8 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - waitForConsumerPoll( + ConsumerPollTestUtils.waitForCondition( + consumer, () -> callback.invoked == 1 && callback.exception == null, "Consumer.poll() did not execute the callback once (without error) in allottec timeout" ); @@ -1487,13 +1488,11 @@ public void testListenerCallbacksInvoke(List { - return Objects.equals(e.getClass(), expectedException.getClass()) && - Objects.equals(e.getMessage(), expectedException.getMessage()) && - Objects.equals(e.getCause(), expectedException.getCause()); - }, + ConsumerPollTestUtils.waitForException( + consumer, + e -> Objects.equals(e.getClass(), expectedException.getClass()) && + Objects.equals(e.getMessage(), expectedException.getMessage()) && + Objects.equals(e.getCause(), expectedException.getCause()), "Consumer.poll() did not throw the expected exception " + expectedException ); } else { @@ -1560,7 +1559,8 @@ public void testBackgroundError() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - waitForConsumerPollException( + ConsumerPollTestUtils.waitForException( + consumer, e -> e.getMessage().equals(expectedException.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" ); @@ -1580,7 +1580,8 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); markReconcileAndAutoCommitCompleteForPollEvent(); - waitForConsumerPollException( + ConsumerPollTestUtils.waitForException( + consumer, e -> e.getMessage().equals(expectedException1.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" ); @@ -1853,7 +1854,8 @@ void testReaperInvokedInPoll() { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); markReconcileAndAutoCommitCompleteForPollEvent(); - waitForConsumerPoll( + ConsumerPollTestUtils.waitForCondition( + consumer, () -> backgroundEventReaper.size() == 0, "Consumer.poll() did not reap background events within timeout" ); @@ -2306,36 +2308,4 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } - - private void waitForConsumerPoll(Supplier testCondition, String conditionDetails) { - try { - TestUtils.waitForCondition( - () -> { - consumer.poll(Duration.ZERO); - return testCondition.get(); - }, - conditionDetails - ); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } - - private void waitForConsumerPollException(Function testCondition, String conditionDetails) { - try { - TestUtils.waitForCondition( - () -> { - try { - consumer.poll(Duration.ZERO); - return false; - } catch (KafkaException e) { - return testCondition.apply(e); - } - }, - conditionDetails - ); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } } diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index d984deccec30c..bbf76fdc0784c 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -145,29 +145,29 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { } private def verifyConsumerWithAuthenticationFailure(consumer: Consumer[Array[Byte], Array[Byte]]): Unit = { + val startMs = System.currentTimeMillis TestUtils.waitUntilTrue(() => { try { consumer.poll(Duration.ofMillis(1000)) false } catch { - case _: SaslAuthenticationException => true + case _: Exception => true } - }, s"Consumer.poll() did not throw a ${classOf[SaslAuthenticationException]} exception within the timeout") - + }, s"Consumer.poll() did not throw an exception within the timeout") + val elapsedMs = System.currentTimeMillis - startMs + assertTrue(elapsedMs <= 5000, s"Poll took too long, elapsed=$elapsedMs") verifyAuthenticationException(consumer.partitionsFor(topic)) createClientCredential() val producer = createProducer() verifyWithRetry(sendOneRecord(producer))() - - val expectedNumber = 1 TestUtils.waitUntilTrue(() => { try { - consumer.poll(Duration.ofMillis(1000)).count == expectedNumber + consumer.poll(Duration.ofMillis(1000)).count == 1 } catch { case _: SaslAuthenticationException => false } - }, s"Consumer.poll() did not read the expected number of records ($expectedNumber) within the timeout") + }, s"Consumer.poll() did not read the expected number of records within the timeout") } @Test diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index c4ea3df48510d..0de21d72ec0d4 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -26,7 +26,6 @@ import javax.security.auth.login.LoginContext import kafka.api.{IntegrationTestHarness, SaslSetup} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.consumer.CloseOptions import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs @@ -186,7 +185,6 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { consumer.assign(java.util.List.of(tp)) val startMs = System.currentTimeMillis() - TestUtils.waitUntilTrue(() => { try { consumer.poll(Duration.ofMillis(50)) @@ -194,11 +192,10 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { } catch { case _: SaslAuthenticationException => true } - }, "Client not ready or disconnected within timeout") - + }, "Consumer.poll() did not trigger a SaslAuthenticationException within timeout") val endMs = System.currentTimeMillis() require(endMs - startMs < failedAuthenticationDelayMs, "Failed authentication must not be delayed on the client") - consumer.close(CloseOptions.timeout(Duration.ZERO)) + consumer.close() } /** From a157071a0d4c01ede279ccbaca913e22fb0fbc31 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 13:04:57 -0700 Subject: [PATCH 043/123] Refactor and cleanup --- .../kafka/clients/consumer/ConsumerPollTestUtils.java | 6 +++--- .../kafka/clients/consumer/KafkaConsumerTest.java | 6 +++++- .../consumer/internals/AsyncKafkaConsumerTest.java | 10 +++++----- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java index d4a6b0957643c..116c47c4143b8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java @@ -59,7 +59,7 @@ public static void waitForCondition(Consumer consumer, } public static void waitForException(Consumer consumer, - Function testCondition, + Function testCondition, String conditionDetails) { try { TestUtils.waitForCondition( @@ -67,8 +67,8 @@ public static void waitForException(Consumer consumer, try { consumer.poll(Duration.ZERO); return false; - } catch (KafkaException e) { - return testCondition.apply(e); + } catch (Throwable t) { + return testCondition.apply(t); } }, conditionDetails diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 3d864a9300aee..91796885f2f6a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -3654,7 +3654,11 @@ public void testPreventMultiThread(GroupProtocol groupProtocol) throws Interrupt service.execute(() -> consumer.poll(Duration.ofSeconds(5))); try { TimeUnit.SECONDS.sleep(1); - assertThrows(ConcurrentModificationException.class, () -> consumer.poll(Duration.ofSeconds(5))); + ConsumerPollTestUtils.waitForException( + consumer, + t -> t instanceof ConcurrentModificationException, + "Consumer did not throw ConcurrentModificationException within timeout" + ); client.wakeup(); consumer.wakeup(); } finally { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 98a7c34cc637d..f0f799af99520 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1490,9 +1490,9 @@ public void testListenerCallbacksInvoke(List Objects.equals(e.getClass(), expectedException.getClass()) && - Objects.equals(e.getMessage(), expectedException.getMessage()) && - Objects.equals(e.getCause(), expectedException.getCause()), + t -> Objects.equals(t.getClass(), expectedException.getClass()) && + Objects.equals(t.getMessage(), expectedException.getMessage()) && + Objects.equals(t.getCause(), expectedException.getCause()), "Consumer.poll() did not throw the expected exception " + expectedException ); } else { @@ -1561,7 +1561,7 @@ public void testBackgroundError() { markReconcileAndAutoCommitCompleteForPollEvent(); ConsumerPollTestUtils.waitForException( consumer, - e -> e.getMessage().equals(expectedException.getMessage()), + t -> t.getMessage().equals(expectedException.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" ); } @@ -1582,7 +1582,7 @@ public void testMultipleBackgroundErrors() { markReconcileAndAutoCommitCompleteForPollEvent(); ConsumerPollTestUtils.waitForException( consumer, - e -> e.getMessage().equals(expectedException1.getMessage()), + t -> t.getMessage().equals(expectedException1.getMessage()), "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" ); assertTrue(backgroundEventQueue.isEmpty()); From 164dfdd5ff083a974f9ddf84affee29ebcc971c1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 13:05:10 -0700 Subject: [PATCH 044/123] Remove unused KafkaException import in ConsumerPollTestUtils Eliminated an unused import of KafkaException from ConsumerPollTestUtils.java to clean up the code. --- .../org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java index 116c47c4143b8..aebc4854cef70 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; From 72472c5fc68d92b28e99e82d7b14266e902adc73 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Sep 2025 15:59:20 -0700 Subject: [PATCH 045/123] Refactor composite poll event processing and context Refactors the composite poll event flow to use explicit state transitions and result handling, replacing Optional-based APIs with more direct methods. Introduces clearer state management for CompositePollEvent, updates ApplicationEventProcessor and CompositePollEventProcessorContext to use new completion and error handling methods, and improves documentation and test utilities. Updates tests and utility methods to match the new APIs and behaviors. --- .../internals/AsyncKafkaConsumer.java | 4 +- .../events/ApplicationEventProcessor.java | 132 ++++++++++-------- .../internals/events/CompositePollEvent.java | 106 +++++++++----- .../events/CompositePollEventInvoker.java | 34 ++++- .../CompositePollEventProcessorContext.java | 39 ++++-- .../consumer/ConsumerPollTestUtils.java | 31 +++- .../clients/consumer/KafkaConsumerTest.java | 12 +- .../internals/AsyncKafkaConsumerTest.java | 5 +- .../events/ApplicationEventProcessorTest.java | 6 +- 9 files changed, 236 insertions(+), 133 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index c552cc3bbd74d..b66016a0c3ca1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -473,7 +473,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - Optional.of(compositePollContextSupplier) + compositePollContextSupplier ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, @@ -693,7 +693,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - Optional.of(compositePollContextSupplier) + compositePollContextSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 6314f762b1408..1e81f0fb9525d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -77,6 +77,21 @@ public ApplicationEventProcessor(final LogContext logContext, this.metadataVersionSnapshot = metadata.updateVersion(); } + public ApplicationEventProcessor(final LogContext logContext, + final RequestManagers requestManagers, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions) { + this(logContext, requestManagers, metadata, subscriptions, Optional.empty()); + } + + public ApplicationEventProcessor(final LogContext logContext, + final RequestManagers requestManagers, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final CompositePollEventProcessorContext compositePollContext) { + this(logContext, requestManagers, metadata, subscriptions, Optional.of(compositePollContext)); + } + @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @Override public void process(ApplicationEvent event) { @@ -227,32 +242,35 @@ public void process(ApplicationEvent event) { } private void process(final PollEvent event) { + processPollEvent(event.pollTimeMs()); + event.markReconcileAndAutoCommitComplete(); + } + + private void processPollEvent(final long pollTimeMs) { // Trigger a reconciliation that can safely commit offsets if needed to rebalance, // as we're processing before any new fetching starts in the app thread requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> consumerMembershipManager.maybeReconcile(true)); if (requestManagers.commitRequestManager.isPresent()) { CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); + commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); // all commit request generation points have been passed, // so it's safe to notify the app thread could proceed and start fetching - event.markReconcileAndAutoCommitComplete(); requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); + hrm.resetPollTimer(pollTimeMs); }); requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); + hrm.resetPollTimer(pollTimeMs); }); } else { // safe to unblock - no auto-commit risk here: // 1. commitRequestManager is not present // 2. shareConsumer has no auto-commit mechanism - event.markReconcileAndAutoCommitComplete(); requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); + hrm.resetPollTimer(pollTimeMs); }); } } @@ -418,15 +436,20 @@ private void process(final TopicRe2JPatternSubscriptionChangeEvent event) { * This will make the consumer send the updated subscription on the next poll. */ private void process(final UpdatePatternSubscriptionEvent event) { - if (subscriptions.hasPatternSubscription()) { - if (this.metadataVersionSnapshot < metadata.updateVersion()) { - this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(metadata.fetch()); - } - } + processUpdatePatternSubscriptionEvent(); event.future().complete(null); } + private void processUpdatePatternSubscriptionEvent() { + if (!subscriptions.hasPatternSubscription()) { + return; + } + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + updatePatternSubscription(metadata.fetch()); + } + } + /** * Process event indicating that the consumer unsubscribed from all topics. This will make * the consumer release its assignment and send a request to leave the group. @@ -736,45 +759,26 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { private void process(final CompositePollEvent event) { CompositePollEventProcessorContext context = compositePollContext.orElseThrow(IllegalArgumentException::new); + ApplicationEvent.Type nextEventType = event.nextEventType(); - if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, ApplicationEvent.Type.POLL)) + if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; - ApplicationEvent.Type nextEventType = event.nextEventType(); - if (nextEventType == ApplicationEvent.Type.POLL) { log.debug("Processing {} logic for {}", nextEventType, event); - long pollTimeMs = event.pollTimeMs(); - - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, - // as we're processing before any new fetching starts in the app thread - requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> - consumerMembershipManager.maybeReconcile(true)); - if (requestManagers.commitRequestManager.isPresent()) { - CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); - // all commit request generation points have been passed, - // so it's safe to notify the app thread could proceed and start fetching - requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } else { - // safe to unblock - no auto-commit risk here: - // 1. commitRequestManager is not present - // 2. shareConsumer has no auto-commit mechanism - requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } + processPollEvent(event.pollTimeMs()); + nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; + + if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) + return; + } + + if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { + log.debug("Processing {} logic for {}", nextEventType, event); + processUpdatePatternSubscriptionEvent(); nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - if (context.maybeFailCompositePoll(event) || context.maybePauseCompositePoll(event, nextEventType)) + if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; } @@ -784,25 +788,24 @@ private void process(final CompositePollEvent event) { context.trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (context.maybeFailCompositePoll(event, updatePositionsError)) + if (context.maybeCompleteExceptionally(event, updatePositionsError)) return; log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); // Create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (context.maybeFailCompositePoll(event, fetchError)) + if (context.maybeCompleteExceptionally(event, fetchError)) return; - event.complete(CompositePollEvent.State.COMPLETE, Optional.empty()); - log.debug("Completed CompositePollEvent {}", event); + context.complete(event); }); }); return; } - event.completeExceptionally(new KafkaException("Unknown next step for composite poll: " + nextEventType)); + context.completeExceptionally(event, new KafkaException("Unknown next step for composite poll: " + nextEventType)); } private BiConsumer complete(final CompletableFuture b) { @@ -821,19 +824,16 @@ private void process(final CompositePollEvent event) { public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final Supplier requestManagersSupplier, - final Optional> compositePollEventProcessorContextSupplier) { + final Supplier requestManagersSupplier) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); - Optional compositePollContext = compositePollEventProcessorContextSupplier.map(Supplier::get); return new ApplicationEventProcessor( logContext, requestManagers, metadata, - subscriptions, - compositePollContext + subscriptions ); } }; @@ -846,14 +846,22 @@ protected ApplicationEventProcessor create() { public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final Supplier requestManagersSupplier) { - return supplier( - logContext, - metadata, - subscriptions, - requestManagersSupplier, - Optional.empty() - ); + final Supplier requestManagersSupplier, + final Supplier compositePollEventProcessorContextSupplier) { + return new CachedSupplier<>() { + @Override + protected ApplicationEventProcessor create() { + RequestManagers requestManagers = requestManagersSupplier.get(); + CompositePollEventProcessorContext compositePollContext = compositePollEventProcessorContextSupplier.get(); + return new ApplicationEventProcessor( + logContext, + requestManagers, + metadata, + subscriptions, + compositePollContext + ); + } + }; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index e3304f2106dd9..0153087cb1cb4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,70 +16,115 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.ConsumerInterceptor; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; +import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.common.KafkaException; +import java.time.Duration; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; +/** + * This class represents the non-blocking event that executes logic functionally equivalent to the following: + * + *

    + *
  • {@link PollEvent}
  • + *
  • {@link CheckAndUpdatePositionsEvent}
  • + *
  • {@link CreateFetchRequestsEvent}
  • + *
+ * + * {@link AsyncKafkaConsumer#poll(Duration)} is implemented using a non-blocking design to ensure performance is + * at the same level as {@link ClassicKafkaConsumer#poll(Duration)}. The event is submitted in {@code poll()}, but + * there are no blocking waits for the "result" of the event. Checks are made for the result at certain points, but + * they do not block. The logic for the three previously-mentioned events is executed one after the other on the + * background thread. + * + *

+ * + * When the {@code CompositePollEvent} is created, it exists in the {@link State#STARTED} state. The background + * thread will execute the {@code CompositePollEvent} until it completes successfully ({@link State#SUCCEEDED}), + * hits an error ({@link State#FAILED}), or detects that the application thread needs to execute callbacks + * ({@link State#CALLBACKS_REQUIRED}). + * + *

+ * + * It's possible that the background processing of the polling will need to be "paused" in order to execute a + * {@link ConsumerInterceptor}, {@link ConsumerRebalanceListener}, and/or {@link OffsetCommitCallback} in the + * application thread. The background thread is able to detect when it needs to complete processing so that the + * application thread can execute the awaiting callbacks. + */ public class CompositePollEvent extends ApplicationEvent { public enum State { - CALLBACKS_REQUIRED, - IN_PROGRESS, - COMPLETE + STARTED, + SUCCEEDED, + FAILED, + CALLBACKS_REQUIRED } public static class Result { - private static final Result IN_PROGRESS = new Result(State.IN_PROGRESS, Optional.empty()); + private static final Object COMPLETED = new Object(); + private static final Result STARTED = new Result(State.STARTED, null); private final State state; + private final Object value; - private final Optional nextEventType; - - public Result(State state, Optional nextEventType) { + public Result(State state, Object value) { this.state = state; - this.nextEventType = nextEventType; + this.value = value; } public State state() { return state; } - public Optional nextEventType() { - return nextEventType; + public Type asNextEventType() { + if (!(value instanceof ApplicationEvent.Type)) + throw new KafkaException("The result value for the poll was unexpected: " + value); + + return (ApplicationEvent.Type) value; + } + + public KafkaException asKafkaException() { + if (!(value instanceof KafkaException)) + throw new KafkaException("The result value for the poll was unexpected: " + value); + + return (KafkaException) value; } @Override public String toString() { - return "Result{" + "state=" + state + ", nextEventType=" + nextEventType + '}'; + return "Result{" + "state=" + state + ", value=" + value + '}'; } @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Result result = (Result) o; - return state == result.state && Objects.equals(nextEventType, result.nextEventType); + return state == result.state && Objects.equals(value, result.value); } @Override public int hashCode() { - return Objects.hash(state, nextEventType); + return Objects.hash(state, value); } } private final long deadlineMs; private final long pollTimeMs; private final Type nextEventType; - private final AtomicReference resultOrError; + private final AtomicReference result; public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { super(Type.COMPOSITE_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; this.nextEventType = nextEventType; - this.resultOrError = new AtomicReference<>(Result.IN_PROGRESS); + this.result = new AtomicReference<>(Result.STARTED); } public long deadlineMs() { @@ -94,30 +139,27 @@ public Type nextEventType() { return nextEventType; } - public Result resultOrError() { - Object o = resultOrError.get(); - - if (o instanceof KafkaException) - throw (KafkaException) o; - else - return (Result) o; + public Result result() { + return result.get(); } - public void complete(State state, Optional nextEventType) { - Result result = new Result( - Objects.requireNonNull(state), - Objects.requireNonNull(nextEventType) - ); - - resultOrError.compareAndSet(Result.IN_PROGRESS, result); + public void completeSuccessfully() { + Result r = new Result(State.SUCCEEDED, Result.COMPLETED); + result.compareAndSet(Result.STARTED, r); } public void completeExceptionally(KafkaException e) { - resultOrError.compareAndSet(Result.IN_PROGRESS, Objects.requireNonNull(e)); + Result r = new Result(State.FAILED, Objects.requireNonNull(e)); + result.compareAndSet(Result.STARTED, r); + } + + public void completeWithCallbackRequired(Type nextEventType) { + Result r = new Result(State.CALLBACKS_REQUIRED, Objects.requireNonNull(nextEventType)); + result.compareAndSet(Result.STARTED, r); } @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", resultOrError=" + resultOrError; + return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", result=" + result; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java index 8b927acc22ac2..40d4f96052ab5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -23,8 +24,14 @@ import org.slf4j.Logger; +import java.time.Duration; + import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; +/** + * {@code CompositePollEventInvoker} is executed on the application thread in the + * {@link AsyncKafkaConsumer#poll(Duration)}. + */ public class CompositePollEventInvoker { private final Logger log; @@ -43,9 +50,15 @@ public CompositePollEventInvoker(LogContext logContext, this.applicationThreadCallbacks = applicationThreadCallbacks; } + /** + * {@code poll()} manages the lifetime of the {@link CompositePollEvent} processing. If it is called when + * no event is currently processing, it will start a new event processing asynchronously. A check is made + * during each invocation to see if the inflight event has reached a + * {@link CompositePollEvent.State terminal state}. If it has, the result will be processed accordingly. + */ public void poll(Timer timer) { if (inflight == null) { - log.debug("No existing inflight event, submitting"); + log.trace("No existing inflight event, submitting a new event"); submitEvent(ApplicationEvent.Type.POLL, timer); } @@ -58,18 +71,25 @@ public void poll(Timer timer) { ); } - CompositePollEvent.Result result = inflight.resultOrError(); + CompositePollEvent.Result result = inflight.result(); CompositePollEvent.State state = result.state(); - if (state == CompositePollEvent.State.COMPLETE) { + if (state == CompositePollEvent.State.SUCCEEDED) { // Make sure to clear out the inflight request since it's complete. - log.debug("Event {} completed, clearing inflight", inflight); + log.trace("Event {} completed, clearing inflight", inflight); + inflight = null; + } else if (state == CompositePollEvent.State.FAILED) { + log.trace("Event {} failed, clearing inflight", inflight); inflight = null; + throw result.asKafkaException(); } else if (state == CompositePollEvent.State.CALLBACKS_REQUIRED) { - log.debug("About to process callbacks"); + log.trace("Event {} paused for callbacks, clearing inflight", inflight); + + // Note: this is calling user-supplied code, so make sure to handle possible errors. applicationThreadCallbacks.run(); - log.debug("Done processing callbacks"); - result.nextEventType().ifPresent(t -> submitEvent(t, timer)); + + // The application thread callbacks are complete. Resume the polling + submitEvent(result.asNextEventType(), timer); } } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java index 646bca4e8e17c..17ec55bd03576 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java @@ -69,7 +69,7 @@ protected CompositePollEventProcessorContext create() { ); } }; - }; + } public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture future, long deadlineMs) { CompletableEvent event = new CompletableEvent<>() { @@ -92,21 +92,25 @@ public String toString() { applicationEventReaper.add(event); } - public boolean maybePauseCompositePoll(CompositePollEvent event, ApplicationEvent.Type nextEventType) { + public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, ApplicationEvent.Type nextEventType) { // If there are background events to process or enqueued callbacks to invoke, exit to // the application thread. if (backgroundEventHandler.size() > 0 || offsetCommitCallbackInvoker.size() > 0) { - CompositePollEvent.State state = CompositePollEvent.State.CALLBACKS_REQUIRED; - log.debug("Pausing event processing for {} with {} as next step", state, nextEventType); - event.complete(state, Optional.of(nextEventType)); + log.debug( + "Pausing polling by completing {} with the state of {} and the next stage of {}", + event, + CompositePollEvent.State.CALLBACKS_REQUIRED, + nextEventType + ); + event.completeWithCallbackRequired(nextEventType); return true; } return false; } - public boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { - if (maybeFailCompositePoll(event)) + public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) { + if (maybeCompleteExceptionally(event)) return true; if (t == null) @@ -121,22 +125,29 @@ public boolean maybeFailCompositePoll(CompositePollEvent event, Throwable t) { t = t.getCause(); } - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - event.completeExceptionally(e); - log.debug("Failing event processing for {}", event, e); + completeExceptionally(event, t); return true; } - public boolean maybeFailCompositePoll(CompositePollEvent event) { + public boolean maybeCompleteExceptionally(CompositePollEvent event) { Optional exception = networkClientDelegate.getAndClearMetadataError(); if (exception.isPresent()) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(exception.get()); - event.completeExceptionally(e); - log.debug("Failing event processing for {}", event, e); + completeExceptionally(event, exception.get()); return true; } return false; } + + public void completeExceptionally(CompositePollEvent event, Throwable error) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(error); + event.completeExceptionally(e); + log.debug("Failing event processing for {}", event, e); + } + + public void complete(CompositePollEvent event) { + event.completeSuccessfully(); + log.debug("Completed CompositePollEvent {}", event); + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java index aebc4854cef70..82e479c040c80 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.clients.consumer; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; @@ -25,22 +27,39 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; + +/** + * This class provides utilities for tests to wait for a call to {@link Consumer#poll(Duration)} to produce a + * result (error, records, specific condition, etc.). This is mostly due to the subtle difference in behavior + * of the non-blocking {@link AsyncKafkaConsumer}. A single pass of {@link AsyncKafkaConsumer#poll(Duration)} + * may not be sufficient to provide an immediate result. + */ public class ConsumerPollTestUtils { - @SuppressWarnings("unchecked") - public static ConsumerRecords waitForRecords(Consumer consumer, Time time) { - Timer timer = time.timer(15000); + /** + * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} to return records from the given {@link Consumer}. + */ + public static ConsumerRecords waitForRecords(Consumer consumer) { + Timer timer = Time.SYSTEM.timer(DEFAULT_MAX_WAIT_MS); while (timer.notExpired()) { + @SuppressWarnings("unchecked") ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); if (!records.isEmpty()) return records; + + timer.update(); } - throw new org.apache.kafka.common.errors.TimeoutException("no records to return"); + throw new TimeoutException("no records to return"); } + /** + * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} for the {@link Consumer} to produce the side effect + * that causes {@link Supplier condition} to evaluate to {@code true}. + */ public static void waitForCondition(Consumer consumer, Supplier testCondition, String conditionDetails) { @@ -57,6 +76,10 @@ public static void waitForCondition(Consumer consumer, } } + /** + * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} for the {@link Consumer} to throw an exception that, + * when tested against the {@link Function condition}, will evaluate to {@code true}. + */ public static void waitForException(Consumer consumer, Function testCondition, String conditionDetails) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 91796885f2f6a..4cc2d67c39125 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -950,7 +950,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol g client.prepareResponse(listOffsetsResponse(Map.of(tp0, 50L))); client.prepareResponse(fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -1044,7 +1044,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1763,7 +1763,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); @@ -1822,7 +1822,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -2117,7 +2117,7 @@ public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol gro time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); - final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertFalse(records.isEmpty()); assertFalse(records.nextOffsets().isEmpty()); } @@ -2715,7 +2715,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept final FetchInfo fetchInfo = new FetchInfo(1L, 99L, 50L, 5); client.respondToRequest(fetchRequest, fetchResponse(Map.of(tp0, fetchInfo))); - final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer, time); + final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index f0f799af99520..172d9bf5438fd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -156,6 +156,7 @@ import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -1666,7 +1667,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { consumer.subscribe(singletonList("topic1")); markReconcileAndAutoCommitCompleteForPollEvent(); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(CompositePollEvent.class)); + verify(applicationEventHandler, atLeastOnce()).add(any(CompositePollEvent.class)); } private Properties requiredConsumerConfigAndGroupId(final String groupId) { @@ -2304,7 +2305,7 @@ private void markReconcileAndAutoCommitCompleteForPollEvent() { }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); doAnswer(invocation -> { CompositePollEvent event = invocation.getArgument(0); - event.complete(CompositePollEvent.State.CALLBACKS_REQUIRED, Optional.empty()); + event.completeWithCallbackRequired(ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS); return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index d7feb04930a32..dde3f567132fc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -111,8 +111,7 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState, - Optional.empty() + subscriptionState ); } @@ -133,8 +132,7 @@ private void setupStreamProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState, - Optional.empty() + subscriptionState ); } From 0aed4aff8912d96013790f7ba92d790b9ab8c5e3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 30 Sep 2025 10:50:07 -0700 Subject: [PATCH 046/123] Minor refactoring and added documentation Replaces nextEventType with startingEventType in CompositePollEvent and related classes for improved clarity and correctness. Adds validation for allowed starting event types, updates method names, improves logging, and enhances documentation for event processing context and state transitions. --- .../events/ApplicationEventProcessor.java | 2 +- .../internals/events/CompositePollEvent.java | 53 ++++++++++++++--- .../events/CompositePollEventInvoker.java | 21 +++++-- .../CompositePollEventProcessorContext.java | 57 ++++++++++++++++--- 4 files changed, 112 insertions(+), 21 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 1e81f0fb9525d..7c78f10bbdbad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -759,7 +759,7 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { private void process(final CompositePollEvent event) { CompositePollEventProcessorContext context = compositePollContext.orElseThrow(IllegalArgumentException::new); - ApplicationEvent.Type nextEventType = event.nextEventType(); + ApplicationEvent.Type nextEventType = event.startingEventType(); if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 0153087cb1cb4..9a825461a75a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerInterceptor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.OffsetCommitCallback; @@ -24,6 +25,7 @@ import org.apache.kafka.common.KafkaException; import java.time.Duration; +import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; @@ -68,7 +70,15 @@ public enum State { public static class Result { - private static final Object COMPLETED = new Object(); + /** + * This string value is used when the {@code Result} represents a completed event. This is used so that + * {@code null} isn't used for {@link #value}. + */ + private static final Object COMPLETED_SENTINEL = "COMPLETED"; + + /** + * Used as the initial state/result until the terminal state is achieved. + */ private static final Result STARTED = new Result(State.STARTED, null); private final State state; private final Object value; @@ -83,6 +93,9 @@ public State state() { } public Type asNextEventType() { + if (state != State.CALLBACKS_REQUIRED) + throw new KafkaException("The usage of asNextEventType is unexpected for state: " + state); + if (!(value instanceof ApplicationEvent.Type)) throw new KafkaException("The result value for the poll was unexpected: " + value); @@ -90,6 +103,9 @@ public Type asNextEventType() { } public KafkaException asKafkaException() { + if (state != State.FAILED) + throw new KafkaException("The usage of asKafkaException is unexpected for state: " + state); + if (!(value instanceof KafkaException)) throw new KafkaException("The result value for the poll was unexpected: " + value); @@ -114,16 +130,33 @@ public int hashCode() { } } + private static final List ALLOWED_STARTING_EVENT_TYPES = List.of( + Type.CHECK_AND_UPDATE_POSITIONS, + Type.POLL, + Type.UPDATE_SUBSCRIPTION_METADATA + ); private final long deadlineMs; private final long pollTimeMs; - private final Type nextEventType; + private final Type startingEventType; private final AtomicReference result; - public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextEventType) { + /** + * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. + * + * @param deadlineMs Time, in milliseconds, at which point the event must be completed; based on the + * {@link Duration} passed to {@link Consumer#poll(Duration)} + * @param pollTimeMs Time, in milliseconds, at which point the event was created + * @param startingEventType {@link ApplicationEvent.Type} that serves as the starting point for the event processing + */ + public CompositePollEvent(long deadlineMs, long pollTimeMs, Type startingEventType) { super(Type.COMPOSITE_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; - this.nextEventType = nextEventType; + + if (!ALLOWED_STARTING_EVENT_TYPES.contains(startingEventType)) + throw new KafkaException("The starting event type " + startingEventType + " is not valid. Should be one of " + ALLOWED_STARTING_EVENT_TYPES); + + this.startingEventType = startingEventType; this.result = new AtomicReference<>(Result.STARTED); } @@ -135,8 +168,8 @@ public long pollTimeMs() { return pollTimeMs; } - public Type nextEventType() { - return nextEventType; + public Type startingEventType() { + return startingEventType; } public Result result() { @@ -144,7 +177,7 @@ public Result result() { } public void completeSuccessfully() { - Result r = new Result(State.SUCCEEDED, Result.COMPLETED); + Result r = new Result(State.SUCCEEDED, Result.COMPLETED_SENTINEL); result.compareAndSet(Result.STARTED, r); } @@ -160,6 +193,10 @@ public void completeWithCallbackRequired(Type nextEventType) { @Override protected String toStringBase() { - return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextEventType=" + nextEventType + ", result=" + result; + return super.toStringBase() + + ", deadlineMs=" + deadlineMs + + ", pollTimeMs=" + pollTimeMs + + ", startingEventType=" + startingEventType + + ", result=" + result; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java index 40d4f96052ab5..f5013d2fbf4f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java @@ -71,24 +71,34 @@ public void poll(Timer timer) { ); } + // Result should be non-null and starts off as State.STARTED. CompositePollEvent.Result result = inflight.result(); CompositePollEvent.State state = result.state(); if (state == CompositePollEvent.State.SUCCEEDED) { - // Make sure to clear out the inflight request since it's complete. + // The composite event has completed all the requisite stages, though it does not imply that + // there is data in the FetchBuffer yet. Make sure to clear out the inflight request. log.trace("Event {} completed, clearing inflight", inflight); inflight = null; } else if (state == CompositePollEvent.State.FAILED) { + // The composite event failed at one of the stages. Make sure to clear out the inflight request + // before the underlying error is surfaced to the user. log.trace("Event {} failed, clearing inflight", inflight); inflight = null; + throw result.asKafkaException(); } else if (state == CompositePollEvent.State.CALLBACKS_REQUIRED) { + // The background thread detected that it needed to yield to the application thread to invoke + // callbacks. Even though the inflight reference _should_ be overwritten when the next stage of + // the event is submitted, go ahead and clear out the inflight request just to be sure. log.trace("Event {} paused for callbacks, clearing inflight", inflight); + inflight = null; // Note: this is calling user-supplied code, so make sure to handle possible errors. applicationThreadCallbacks.run(); - // The application thread callbacks are complete. Resume the polling + // The application thread callbacks are complete. Create another event to resume the polling at + // the next stage. submitEvent(result.asNextEventType(), timer); } } catch (Throwable t) { @@ -102,8 +112,11 @@ public void poll(Timer timer) { private void submitEvent(ApplicationEvent.Type type, Timer timer) { long deadlineMs = calculateDeadlineMs(timer); - inflight = new CompositePollEvent(deadlineMs, time.milliseconds(), type); + long pollTimeMs = time.milliseconds(); + inflight = new CompositePollEvent(deadlineMs, pollTimeMs, type); applicationEventHandler.add(inflight); - log.debug("Submitted new {} with {} remaining on timer", inflight, timer.remainingMs()); + + if (log.isTraceEnabled()) + log.trace("Submitted new {} with {} remaining on timer", inflight, timer.remainingMs()); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java index 17ec55bd03576..6356e04f009ce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.CachedSupplier; +import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; @@ -30,6 +32,11 @@ import java.util.concurrent.CompletionException; import java.util.function.Supplier; +/** + * This provides the context for the {@link ApplicationEventProcessor#process(ApplicationEvent)} that invokes the + * {@link CompositePollEvent} process method. This is mostly to avoid polluting the {@link ApplicationEventProcessor} + * with instance variables and logic that's specific only to the background {@link CompositePollEvent} processing. + */ public class CompositePollEventProcessorContext { private final Logger log; @@ -50,6 +57,10 @@ private CompositePollEventProcessorContext(LogContext logContext, this.applicationEventReaper = applicationEventReaper; } + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ public static Supplier supplier(LogContext logContext, Supplier networkClientDelegateSupplier, BackgroundEventHandler backgroundEventHandler, @@ -71,11 +82,17 @@ protected CompositePollEventProcessorContext create() { }; } - public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture future, long deadlineMs) { - CompletableEvent event = new CompletableEvent<>() { + /** + * To maintain the flow from {@link ClassicKafkaConsumer}, the logic to check and update positions should be + * allowed to time out before moving on to the logic for sending fetch requests. This achieves that by reusing + * the {@link CompletableEventReaper} and allowing it to expire the {@link CompletableFuture} for the check and + * update positions stage. + */ + public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture updatePositionsFuture, long deadlineMs) { + CompletableEvent event = new CompletableEvent<>() { @Override - public CompletableFuture future() { - return future; + public CompletableFuture future() { + return updatePositionsFuture; } @Override @@ -85,18 +102,23 @@ public long deadlineMs() { @Override public String toString() { - return getClass().getSimpleName() + "{future=" + future + ", deadlineMs=" + deadlineMs + '}'; + return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + deadlineMs + '}'; } }; applicationEventReaper.add(event); } + /** + * Helper method that will check if any application thread user callbacks need to be executed. If so, the + * current event will be completed with {@link CompositePollEvent.State#CALLBACKS_REQUIRED} and this method + * will return {@code true}. Otherwise, it will return {@code false}. + */ public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, ApplicationEvent.Type nextEventType) { // If there are background events to process or enqueued callbacks to invoke, exit to // the application thread. if (backgroundEventHandler.size() > 0 || offsetCommitCallbackInvoker.size() > 0) { - log.debug( + log.trace( "Pausing polling by completing {} with the state of {} and the next stage of {}", event, CompositePollEvent.State.CALLBACKS_REQUIRED, @@ -109,6 +131,13 @@ public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, Appli return false; } + /** + * Helper method that checks if there's a non-null error from + * {@link NetworkClientDelegate#getAndClearMetadataError()} or if the provided exception is not a timeout-based + * exception. If there's an error to report to the user, the current event will be completed with + * {@link CompositePollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will + * return {@code false}. + */ public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) { if (maybeCompleteExceptionally(event)) return true; @@ -129,6 +158,12 @@ public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) return true; } + /** + * Helper method that checks if there's a non-null error from + * {@link NetworkClientDelegate#getAndClearMetadataError()}, and if so, reports it to the user by completing the + * current event with {@link CompositePollEvent.State#FAILED} and returning {@code true}. Otherwise, it will + * return {@code false}. + */ public boolean maybeCompleteExceptionally(CompositePollEvent event) { Optional exception = networkClientDelegate.getAndClearMetadataError(); @@ -140,14 +175,20 @@ public boolean maybeCompleteExceptionally(CompositePollEvent event) { return false; } + /** + * Helper method to complete the given event with {@link CompositePollEvent.State#FAILED}. + */ public void completeExceptionally(CompositePollEvent event, Throwable error) { KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(error); event.completeExceptionally(e); - log.debug("Failing event processing for {}", event, e); + log.trace("Failing event processing for {}", event, e); } + /** + * Helper method to complete the given event with {@link CompositePollEvent.State#SUCCEEDED}. + */ public void complete(CompositePollEvent event) { event.completeSuccessfully(); - log.debug("Completed CompositePollEvent {}", event); + log.trace("Completed event processing for {}", event); } } \ No newline at end of file From ab42fca4f4ae61e3c2e63d628afe9ffc32df68f3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 30 Sep 2025 13:20:05 -0700 Subject: [PATCH 047/123] Fixed streams tests in PlaintextAdminIntegrationTest Surfaced isPendingCallbacks() at the SubscriptionState API so that partitions that are pending callback invocation are not returned. This is also in agreement with the JavaDoc which states: "This will give the set of topic partitions currently assigned to the consumer (which may be none if . . . the partitions are in the process of getting reassigned)." --- .../consumer/internals/AsyncKafkaConsumer.java | 4 +++- .../clients/consumer/internals/SubscriptionState.java | 11 ++++++++++- .../kafka/api/PlaintextAdminIntegrationTest.scala | 9 +++------ 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b66016a0c3ca1..a2a505d038214 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1689,7 +1689,9 @@ public Uuid clientInstanceId(Duration timeout) { public Set assignment() { acquireAndEnsureOpen(); try { - return Collections.unmodifiableSet(subscriptions.assignedPartitions()); + return subscriptions.assignedPartitions().stream() + .filter(tp -> !subscriptions.isPendingCallbacks(tp)) + .collect(Collectors.toUnmodifiableSet()); } finally { release(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 9d44e98ed3954..678318ced1f0e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -896,6 +896,11 @@ synchronized boolean isFetchable(TopicPartition tp) { return tps != null && isFetchableAndSubscribed(tp, tps); } + public synchronized boolean isPendingCallbacks(TopicPartition tp) { + TopicPartitionState assignedOrNull = assignedStateOrNull(tp); + return assignedOrNull != null && assignedOrNull.isPendingCallbacks(); + } + public synchronized boolean hasValidPosition(TopicPartition tp) { TopicPartitionState assignedOrNull = assignedStateOrNull(tp); return assignedOrNull != null && assignedOrNull.hasValidPosition(); @@ -1212,7 +1217,11 @@ private boolean shouldInitialize() { } private boolean isFetchable() { - return !paused && !pendingRevocation && !pendingOnAssignedCallback && hasValidPosition(); + return !paused && !isPendingCallbacks() && hasValidPosition(); + } + + private boolean isPendingCallbacks() { + return pendingRevocation || pendingOnAssignedCallback; } private void highWatermark(Long highWatermark) { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index cf3f8fa782135..091aceedc9bd0 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -4618,10 +4618,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { TestUtils.waitUntilTrue(() => { streams.poll(JDuration.ofMillis(100L)) - !streams.assignment().isEmpty + streams.assignment().size() == testNumPartitions }, "Consumer not assigned to partitions") - streams.poll(JDuration.ofMillis(1000L)) streams.commitSync() TestUtils.waitUntilTrue(() => { @@ -4678,10 +4677,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { TestUtils.waitUntilTrue(() => { streams.poll(JDuration.ofMillis(100L)) - !streams.assignment().isEmpty + streams.assignment().size() == testNumPartitions }, "Consumer not assigned to partitions") - streams.poll(JDuration.ofMillis(1000L)) streams.commitSync() // List streams group offsets @@ -4755,10 +4753,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { TestUtils.waitUntilTrue(() => { streams.poll(JDuration.ofMillis(100L)) - !streams.assignment().isEmpty + streams.assignment().size() == testNumPartitions }, "Consumer not assigned to partitions") - streams.poll(JDuration.ofMillis(1000L)) streams.commitSync() // List streams group offsets From cc49db81babe30abf31be1e24622ba17331716fe Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 30 Sep 2025 13:35:16 -0700 Subject: [PATCH 048/123] Update CompositePollEvent.java --- .../clients/consumer/internals/events/CompositePollEvent.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 9a825461a75a5..16180cf6b125c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -34,6 +34,7 @@ * *
    *
  • {@link PollEvent}
  • + *
  • {@link UpdatePatternSubscriptionEvent}
  • *
  • {@link CheckAndUpdatePositionsEvent}
  • *
  • {@link CreateFetchRequestsEvent}
  • *
From 7ba60474c054edc05106ca523e3699fd646104da Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 30 Sep 2025 16:25:50 -0700 Subject: [PATCH 049/123] Improve logging and comments in poll event classes Updated comments in CompositePollEvent for clarity and improved the debug log message in CompositePollEventInvoker to better describe event failure handling. --- .../consumer/internals/events/CompositePollEvent.java | 5 ++--- .../consumer/internals/events/CompositePollEventInvoker.java | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java index 16180cf6b125c..f5aa9602253c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java @@ -42,8 +42,7 @@ * {@link AsyncKafkaConsumer#poll(Duration)} is implemented using a non-blocking design to ensure performance is * at the same level as {@link ClassicKafkaConsumer#poll(Duration)}. The event is submitted in {@code poll()}, but * there are no blocking waits for the "result" of the event. Checks are made for the result at certain points, but - * they do not block. The logic for the three previously-mentioned events is executed one after the other on the - * background thread. + * they do not block. The logic for the previously-mentioned events is executed sequentially on the background thread. * *

* @@ -198,6 +197,6 @@ protected String toStringBase() { ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", startingEventType=" + startingEventType + - ", result=" + result; + ", result=" + result.get(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java index f5013d2fbf4f1..0feb1c0d6647e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java @@ -104,7 +104,7 @@ public void poll(Timer timer) { } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request // because the error effectively renders it complete. - log.debug("Event {} \"completed\" via error ({}), clearing inflight", inflight, String.valueOf(t)); + log.debug("Event {} failed due to {}, clearing inflight", inflight, String.valueOf(t)); inflight = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); } From bc660d6462676450d7c7ccba5f9988e314954cdc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 30 Sep 2025 16:27:39 -0700 Subject: [PATCH 050/123] Reverting changes made to omit pending-callback partitions from being returned --- .../consumer/internals/AsyncKafkaConsumer.java | 4 +--- .../clients/consumer/internals/SubscriptionState.java | 11 +---------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index a2a505d038214..b66016a0c3ca1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1689,9 +1689,7 @@ public Uuid clientInstanceId(Duration timeout) { public Set assignment() { acquireAndEnsureOpen(); try { - return subscriptions.assignedPartitions().stream() - .filter(tp -> !subscriptions.isPendingCallbacks(tp)) - .collect(Collectors.toUnmodifiableSet()); + return Collections.unmodifiableSet(subscriptions.assignedPartitions()); } finally { release(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 678318ced1f0e..9d44e98ed3954 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -896,11 +896,6 @@ synchronized boolean isFetchable(TopicPartition tp) { return tps != null && isFetchableAndSubscribed(tp, tps); } - public synchronized boolean isPendingCallbacks(TopicPartition tp) { - TopicPartitionState assignedOrNull = assignedStateOrNull(tp); - return assignedOrNull != null && assignedOrNull.isPendingCallbacks(); - } - public synchronized boolean hasValidPosition(TopicPartition tp) { TopicPartitionState assignedOrNull = assignedStateOrNull(tp); return assignedOrNull != null && assignedOrNull.hasValidPosition(); @@ -1217,11 +1212,7 @@ private boolean shouldInitialize() { } private boolean isFetchable() { - return !paused && !isPendingCallbacks() && hasValidPosition(); - } - - private boolean isPendingCallbacks() { - return pendingRevocation || pendingOnAssignedCallback; + return !paused && !pendingRevocation && !pendingOnAssignedCallback && hasValidPosition(); } private void highWatermark(Long highWatermark) { From e9dbc61bff376e1bd9f7ec53967121389a4f16cb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 Oct 2025 13:08:19 -0700 Subject: [PATCH 051/123] Notify FetchBuffer from background thread if pausing due to application thread callbacks FetchBuffer is now provided to CompositePollEventProcessorContext and its supplier, allowing the context to call fetchBuffer.wakeup() after completing an event. The wakeup method in FetchBuffer is made public to support this usage. --- .../consumer/internals/AsyncKafkaConsumer.java | 6 ++++-- .../clients/consumer/internals/FetchBuffer.java | 2 +- .../events/CompositePollEventProcessorContext.java | 13 ++++++++++--- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b66016a0c3ca1..9a411826da356 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -467,7 +467,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, networkClientDelegateSupplier, backgroundEventHandler, offsetCommitCallbackInvoker, - applicationEventReaper + applicationEventReaper, + fetchBuffer ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -686,7 +687,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, networkClientDelegateSupplier, backgroundEventHandler, offsetCommitCallbackInvoker, - applicationEventReaper + applicationEventReaper, + fetchBuffer ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java index 6cf5bc301b370..312e455b1aa81 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java @@ -193,7 +193,7 @@ void awaitWakeup(Timer timer) { } } - void wakeup() { + public void wakeup() { try { lock.lock(); wokenup.set(true); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java index 6356e04f009ce..8996f189cd25b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.clients.consumer.internals.FetchBuffer; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; import org.apache.kafka.common.KafkaException; @@ -44,17 +45,20 @@ public class CompositePollEventProcessorContext { private final BackgroundEventHandler backgroundEventHandler; private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final CompletableEventReaper applicationEventReaper; + private final FetchBuffer fetchBuffer; private CompositePollEventProcessorContext(LogContext logContext, NetworkClientDelegate networkClientDelegate, BackgroundEventHandler backgroundEventHandler, OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper) { + CompletableEventReaper applicationEventReaper, + FetchBuffer fetchBuffer) { this.log = logContext.logger(getClass()); this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; this.applicationEventReaper = applicationEventReaper; + this.fetchBuffer = fetchBuffer; } /** @@ -65,7 +69,8 @@ public static Supplier supplier(LogContext l Supplier networkClientDelegateSupplier, BackgroundEventHandler backgroundEventHandler, OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper) { + CompletableEventReaper applicationEventReaper, + FetchBuffer fetchBuffer) { return new CachedSupplier<>() { @Override protected CompositePollEventProcessorContext create() { @@ -76,7 +81,8 @@ protected CompositePollEventProcessorContext create() { networkClientDelegate, backgroundEventHandler, offsetCommitCallbackInvoker, - applicationEventReaper + applicationEventReaper, + fetchBuffer ); } }; @@ -125,6 +131,7 @@ public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, Appli nextEventType ); event.completeWithCallbackRequired(nextEventType); + fetchBuffer.wakeup(); return true; } From 5d1a34d27f90970d4242813aa6c2b5566f6c15d3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 Oct 2025 13:09:15 -0700 Subject: [PATCH 052/123] Relying on reading records to determine group membership for streams group offsets tests Refactored tests to use a counter for consumed records and added explicit consumer configuration with AUTO_OFFSET_RESET set to 'earliest'. This ensures all produced records are consumed and improves reliability of partition assignment checks. --- .../api/PlaintextAdminIntegrationTest.scala | 42 +++++++++++++++---- 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 091aceedc9bd0..1f2c003b789b8 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -4601,7 +4601,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { prepareRecords(testTopicName) // Producer sends messages - for (i <- 1 to 20) { + val numRecords = 20 + + for (i <- 1 to numRecords) { TestUtils.waitUntilTrue(() => { val producerRecord = producer.send( new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) @@ -4610,15 +4612,21 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }, "Fail to produce record to topic") } + val consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + val streams = createStreamsGroup( + configOverrides = consumerConfig, inputTopic = testTopicName, streamsGroupId = streamsGroupId, ) try { + var counter = 0 + TestUtils.waitUntilTrue(() => { - streams.poll(JDuration.ofMillis(100L)) - streams.assignment().size() == testNumPartitions + counter += streams.poll(JDuration.ofMillis(100L)).count() + counter >= numRecords }, "Consumer not assigned to partitions") streams.commitSync() @@ -4660,7 +4668,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { prepareTopics(List(testTopicName), testNumPartitions) prepareRecords(testTopicName) // Producer sends messages - for (i <- 1 to 20) { + val numRecords = 20 + + for (i <- 1 to numRecords) { TestUtils.waitUntilTrue(() => { val producerRecord = producer.send( new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) @@ -4669,15 +4679,21 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }, "Fail to produce record to topic") } + val consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + val streams = createStreamsGroup( + configOverrides = consumerConfig, inputTopic = testTopicName, streamsGroupId = streamsGroupId, ) try { + var counter = 0 + TestUtils.waitUntilTrue(() => { - streams.poll(JDuration.ofMillis(100L)) - streams.assignment().size() == testNumPartitions + counter += streams.poll(JDuration.ofMillis(100L)).count() + counter >= numRecords }, "Consumer not assigned to partitions") streams.commitSync() @@ -4736,7 +4752,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { prepareRecords(testTopicName) // Producer sends messages - for (i <- 1 to 20) { + val numRecords = 20 + + for (i <- 1 to numRecords) { TestUtils.waitUntilTrue(() => { val producerRecord = producer.send( new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) @@ -4745,15 +4763,21 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }, "Fail to produce record to topic") } + val consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + val streams = createStreamsGroup( + configOverrides = consumerConfig, inputTopic = testTopicName, streamsGroupId = streamsGroupId, ) try { + var counter = 0 + TestUtils.waitUntilTrue(() => { - streams.poll(JDuration.ofMillis(100L)) - streams.assignment().size() == testNumPartitions + counter += streams.poll(JDuration.ofMillis(100L)).count() + counter >= numRecords }, "Consumer not assigned to partitions") streams.commitSync() From f6864a3ac46c23a530b6a32d05d548c403cd96ea Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 2 Oct 2025 14:20:15 -0700 Subject: [PATCH 053/123] Refactor POLL into ASYNC_POLL and SHARE_POLL --- .../internals/AsyncKafkaConsumer.java | 114 +++++++++++----- .../consumer/internals/ShareConsumerImpl.java | 4 +- .../internals/events/ApplicationEvent.java | 4 +- .../events/ApplicationEventProcessor.java | 92 ++++++------- ...sitePollEvent.java => AsyncPollEvent.java} | 25 +++- ...va => AsyncPollEventProcessorContext.java} | 61 ++++----- .../events/CompositePollEventInvoker.java | 122 ------------------ .../{PollEvent.java => SharePollEvent.java} | 30 +---- .../ApplicationEventHandlerTest.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 56 ++++---- .../internals/ConsumerNetworkThreadTest.java | 4 +- .../internals/ShareConsumerImplTest.java | 4 +- .../events/ApplicationEventProcessorTest.java | 9 +- 13 files changed, 215 insertions(+), 314 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CompositePollEvent.java => AsyncPollEvent.java} (88%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CompositePollEventProcessorContext.java => AsyncPollEventProcessorContext.java} (74%) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{PollEvent.java => SharePollEvent.java} (54%) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 9a411826da356..e3f9eabbe98ef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -41,6 +41,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEventProcessorContext; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; @@ -49,8 +51,6 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEventInvoker; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEventProcessorContext; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent; @@ -326,7 +326,7 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - private final CompositePollEventInvoker pollInvoker; + private AsyncPollEvent inflightPoll; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -462,7 +462,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - final Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + final Supplier asyncPollContextSupplier = AsyncPollEventProcessorContext.supplier( logContext, networkClientDelegateSupplier, backgroundEventHandler, @@ -474,7 +474,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - compositePollContextSupplier + asyncPollContextSupplier ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, @@ -496,15 +496,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, new StreamsRebalanceListenerInvoker(logContext, s)); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); - this.pollInvoker = new CompositePollEventInvoker( - logContext, - time, - applicationEventHandler, - () -> { - offsetCommitCallbackInvoker.executeCallbacks(); - processBackgroundEvents(); - } - ); // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, @@ -579,15 +570,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.clientTelemetryReporter = Optional.empty(); this.autoCommitEnabled = autoCommitEnabled; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.pollInvoker = new CompositePollEventInvoker( - logContext, - time, - applicationEventHandler, - () -> { - offsetCommitCallbackInvoker.executeCallbacks(); - processBackgroundEvents(); - } - ); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, time, @@ -682,7 +664,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, Optional.empty() ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - final Supplier compositePollContextSupplier = CompositePollEventProcessorContext.supplier( + final Supplier asyncPollContextSupplier = AsyncPollEventProcessorContext.supplier( logContext, networkClientDelegateSupplier, backgroundEventHandler, @@ -695,7 +677,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, subscriptions, requestManagersSupplier, - compositePollContextSupplier + asyncPollContextSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, @@ -708,15 +690,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.streamsRebalanceListenerInvoker = Optional.empty(); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = new CompletableEventReaper(logContext); - this.pollInvoker = new CompositePollEventInvoker( - logContext, - time, - applicationEventHandler, - () -> { - offsetCommitCallbackInvoker.executeCallbacks(); - processBackgroundEvents(); - } - ); } // auxiliary interface for testing @@ -887,7 +860,7 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - pollInvoker.poll(timer); + checkInflightPollResult(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -915,6 +888,77 @@ public ConsumerRecords poll(final Duration timeout) { } } + /** + * {@code checkInflightPollResult()} manages the lifetime of the {@link AsyncPollEvent} processing. If it is + * called when no event is currently processing, it will start a new event processing asynchronously. A check + * is made during each invocation to see if the inflight event has reached a + * {@link AsyncPollEvent.State terminal state}. If it has, the result will be processed accordingly. + */ + public void checkInflightPollResult(Timer timer) { + if (inflightPoll == null) { + log.trace("No existing inflight async poll event, submitting a new event"); + submitEvent(ApplicationEvent.Type.ASYNC_POLL, timer); + } + + try { + if (log.isTraceEnabled()) { + log.trace( + "Attempting to retrieve result from previously submitted {} with {} remaining on timer", + inflightPoll, + timer.remainingMs() + ); + } + + // Result should be non-null and starts off as State.STARTED. + AsyncPollEvent.Result result = inflightPoll.result(); + AsyncPollEvent.State state = result.state(); + + if (state == AsyncPollEvent.State.SUCCEEDED) { + // The async poll event has completed all the requisite stages, though it does not imply that + // there is data in the FetchBuffer yet. Make sure to clear out the inflight request. + log.trace("Event {} completed, clearing inflight", inflightPoll); + inflightPoll = null; + } else if (state == AsyncPollEvent.State.FAILED) { + // The async poll failed at one of the stages. Make sure to clear out the inflight request + // before the underlying error is surfaced to the user. + log.trace("Event {} failed, clearing inflight", inflightPoll); + inflightPoll = null; + + throw result.asKafkaException(); + } else if (state == AsyncPollEvent.State.CALLBACKS_REQUIRED) { + // The background thread detected that it needed to yield to the application thread to invoke + // callbacks. Even though the inflight reference _should_ be overwritten when the next stage of + // the event is submitted, go ahead and clear out the inflight request just to be sure. + log.trace("Event {} paused for callbacks, clearing inflight", inflightPoll); + inflightPoll = null; + + // Note: this is calling user-supplied code, so make sure to handle possible errors. + offsetCommitCallbackInvoker.executeCallbacks(); + processBackgroundEvents(); + + // The application thread callbacks are complete. Create another event to resume the polling at + // the next stage. + submitEvent(result.asNextEventType(), timer); + } + } catch (Throwable t) { + // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request + // because the error effectively renders it complete. + log.debug("Event {} failed due to {}, clearing inflight", inflightPoll, String.valueOf(t)); + inflightPoll = null; + throw ConsumerUtils.maybeWrapAsKafkaException(t); + } + } + + private void submitEvent(ApplicationEvent.Type type, Timer timer) { + long deadlineMs = calculateDeadlineMs(timer); + long pollTimeMs = time.milliseconds(); + inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs, type); + applicationEventHandler.add(inflightPoll); + + if (log.isTraceEnabled()) + log.trace("Submitted new {} with {} remaining on timer", inflightPoll, timer.remainingMs()); + } + /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index ecdc9b8a04844..6b578f59c028d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -38,13 +38,13 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; 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.PollEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeAsyncEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeSyncEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackRegistrationEvent; import org.apache.kafka.clients.consumer.internals.events.ShareFetchEvent; +import org.apache.kafka.clients.consumer.internals.events.SharePollEvent; import org.apache.kafka.clients.consumer.internals.events.ShareSubscriptionChangeEvent; import org.apache.kafka.clients.consumer.internals.events.ShareUnsubscribeEvent; import org.apache.kafka.clients.consumer.internals.events.StopFindCoordinatorOnCloseEvent; @@ -583,7 +583,7 @@ public synchronized ConsumerRecords poll(final Duration timeout) { do { // Make sure the network thread can tell the application is actively polling - applicationEventHandler.add(new PollEvent(timer.currentTimeMs())); + applicationEventHandler.add(new SharePollEvent(timer.currentTimeMs())); processBackgroundEvents(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index 20e6828777dac..79ca558123a7c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -28,14 +28,14 @@ public abstract class ApplicationEvent { public enum Type { - COMMIT_ASYNC, COMMIT_SYNC, COMPOSITE_POLL, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + COMMIT_ASYNC, COMMIT_SYNC, ASYNC_POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, CHECK_AND_UPDATE_POSITIONS, RESET_OFFSET, TOPIC_METADATA, ALL_TOPICS_METADATA, TOPIC_SUBSCRIPTION_CHANGE, TOPIC_PATTERN_SUBSCRIPTION_CHANGE, TOPIC_RE2J_PATTERN_SUBSCRIPTION_CHANGE, UPDATE_SUBSCRIPTION_METADATA, UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, COMMIT_ON_CLOSE, CREATE_FETCH_REQUESTS, LEAVE_GROUP_ON_CLOSE, STOP_FIND_COORDINATOR_ON_CLOSE, PAUSE_PARTITIONS, RESUME_PARTITIONS, CURRENT_LAG, - SHARE_FETCH, SHARE_ACKNOWLEDGE_ASYNC, SHARE_ACKNOWLEDGE_SYNC, + SHARE_POLL, SHARE_FETCH, SHARE_ACKNOWLEDGE_ASYNC, SHARE_ACKNOWLEDGE_SYNC, SHARE_SUBSCRIPTION_CHANGE, SHARE_UNSUBSCRIBE, SHARE_ACKNOWLEDGE_ON_CLOSE, SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK_REGISTRATION, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 7c78f10bbdbad..c241cd5eb8b0f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -61,19 +61,19 @@ public class ApplicationEventProcessor implements EventProcessor compositePollContext; + private final Optional asyncPollContext; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final Optional compositePollContext) { + final Optional asyncPollContext) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; - this.compositePollContext = compositePollContext; + this.asyncPollContext = asyncPollContext; this.metadataVersionSnapshot = metadata.updateVersion(); } @@ -88,16 +88,20 @@ public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final CompositePollEventProcessorContext compositePollContext) { - this(logContext, requestManagers, metadata, subscriptions, Optional.of(compositePollContext)); + final AsyncPollEventProcessorContext asyncPollContext) { + this(logContext, requestManagers, metadata, subscriptions, Optional.of(asyncPollContext)); } @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @Override public void process(ApplicationEvent event) { switch (event.type()) { - case COMPOSITE_POLL: - process((CompositePollEvent) event); + case ASYNC_POLL: + process((AsyncPollEvent) event); + return; + + case SHARE_POLL: + process((SharePollEvent) event); return; case COMMIT_ASYNC: @@ -108,10 +112,6 @@ public void process(ApplicationEvent event) { process((SyncCommitEvent) event); return; - case POLL: - process((PollEvent) event); - return; - case FETCH_COMMITTED_OFFSETS: process((FetchCommittedOffsetsEvent) event); return; @@ -241,38 +241,13 @@ public void process(ApplicationEvent event) { } } - private void process(final PollEvent event) { - processPollEvent(event.pollTimeMs()); - event.markReconcileAndAutoCommitComplete(); - } - - private void processPollEvent(final long pollTimeMs) { - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, - // as we're processing before any new fetching starts in the app thread + private void process(final SharePollEvent event) { requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> consumerMembershipManager.maybeReconcile(true)); - if (requestManagers.commitRequestManager.isPresent()) { - CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(pollTimeMs); - // all commit request generation points have been passed, - // so it's safe to notify the app thread could proceed and start fetching - requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } else { - // safe to unblock - no auto-commit risk here: - // 1. commitRequestManager is not present - // 2. shareConsumer has no auto-commit mechanism - requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(pollTimeMs); - }); - } + requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); } private void process(final CreateFetchRequestsEvent event) { @@ -757,16 +732,35 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { requestManagers.streamsMembershipManager.get().onAllTasksLostCallbackCompleted(event); } - private void process(final CompositePollEvent event) { - CompositePollEventProcessorContext context = compositePollContext.orElseThrow(IllegalArgumentException::new); + private void process(final AsyncPollEvent event) { + AsyncPollEventProcessorContext context = asyncPollContext.orElseThrow(IllegalArgumentException::new); ApplicationEvent.Type nextEventType = event.startingEventType(); if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; - if (nextEventType == ApplicationEvent.Type.POLL) { + if (nextEventType == ApplicationEvent.Type.ASYNC_POLL) { log.debug("Processing {} logic for {}", nextEventType, event); - processPollEvent(event.pollTimeMs()); + + // Trigger a reconciliation that can safely commit offsets if needed to rebalance, + // as we're processing before any new fetching starts + requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> + consumerMembershipManager.maybeReconcile(true)); + + if (requestManagers.commitRequestManager.isPresent()) { + CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); + commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); + + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + } + nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) @@ -805,7 +799,7 @@ private void process(final CompositePollEvent event) { return; } - context.completeExceptionally(event, new KafkaException("Unknown next step for composite poll: " + nextEventType)); + context.completeExceptionally(event, new KafkaException("Unknown next step for async poll: " + nextEventType)); } private BiConsumer complete(final CompletableFuture b) { @@ -847,18 +841,18 @@ public static Supplier supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, - final Supplier compositePollEventProcessorContextSupplier) { + final Supplier asyncPollContextSupplier) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); - CompositePollEventProcessorContext compositePollContext = compositePollEventProcessorContextSupplier.get(); + AsyncPollEventProcessorContext asyncPollContext = asyncPollContextSupplier.get(); return new ApplicationEventProcessor( logContext, requestManagers, metadata, subscriptions, - compositePollContext + asyncPollContext ); } }; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java similarity index 88% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index f5aa9602253c2..e412016ae886c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -33,7 +33,7 @@ * This class represents the non-blocking event that executes logic functionally equivalent to the following: * *

    - *
  • {@link PollEvent}
  • + *
  • {@link SharePollEvent}
  • *
  • {@link UpdatePatternSubscriptionEvent}
  • *
  • {@link CheckAndUpdatePositionsEvent}
  • *
  • {@link CreateFetchRequestsEvent}
  • @@ -46,8 +46,8 @@ * *

    * - * When the {@code CompositePollEvent} is created, it exists in the {@link State#STARTED} state. The background - * thread will execute the {@code CompositePollEvent} until it completes successfully ({@link State#SUCCEEDED}), + * When the {@code AsyncPollEvent} is created, it exists in the {@link State#STARTED} state. The background + * thread will execute the {@code AsyncPollEvent} until it completes successfully ({@link State#SUCCEEDED}), * hits an error ({@link State#FAILED}), or detects that the application thread needs to execute callbacks * ({@link State#CALLBACKS_REQUIRED}). * @@ -58,7 +58,7 @@ * application thread. The background thread is able to detect when it needs to complete processing so that the * application thread can execute the awaiting callbacks. */ -public class CompositePollEvent extends ApplicationEvent { +public class AsyncPollEvent extends ApplicationEvent { public enum State { @@ -131,8 +131,8 @@ public int hashCode() { } private static final List ALLOWED_STARTING_EVENT_TYPES = List.of( + Type.ASYNC_POLL, Type.CHECK_AND_UPDATE_POSITIONS, - Type.POLL, Type.UPDATE_SUBSCRIPTION_METADATA ); private final long deadlineMs; @@ -140,6 +140,17 @@ public int hashCode() { private final Type startingEventType; private final AtomicReference result; + /** + * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. + * + * @param deadlineMs Time, in milliseconds, at which point the event must be completed; based on the + * {@link Duration} passed to {@link Consumer#poll(Duration)} + * @param pollTimeMs Time, in milliseconds, at which point the event was created + */ + public AsyncPollEvent(long deadlineMs, long pollTimeMs) { + this(deadlineMs, pollTimeMs, Type.ASYNC_POLL); + } + /** * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. * @@ -148,8 +159,8 @@ public int hashCode() { * @param pollTimeMs Time, in milliseconds, at which point the event was created * @param startingEventType {@link ApplicationEvent.Type} that serves as the starting point for the event processing */ - public CompositePollEvent(long deadlineMs, long pollTimeMs, Type startingEventType) { - super(Type.COMPOSITE_POLL); + public AsyncPollEvent(long deadlineMs, long pollTimeMs, Type startingEventType) { + super(Type.ASYNC_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java similarity index 74% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java index 8996f189cd25b..e08bbd9064598 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java @@ -35,10 +35,10 @@ /** * This provides the context for the {@link ApplicationEventProcessor#process(ApplicationEvent)} that invokes the - * {@link CompositePollEvent} process method. This is mostly to avoid polluting the {@link ApplicationEventProcessor} - * with instance variables and logic that's specific only to the background {@link CompositePollEvent} processing. + * {@link AsyncPollEvent} process method. This is mostly to avoid polluting the {@link ApplicationEventProcessor} + * with instance variables and logic that's specific only to the background {@link AsyncPollEvent} processing. */ -public class CompositePollEventProcessorContext { +public class AsyncPollEventProcessorContext { private final Logger log; private final NetworkClientDelegate networkClientDelegate; @@ -47,12 +47,12 @@ public class CompositePollEventProcessorContext { private final CompletableEventReaper applicationEventReaper; private final FetchBuffer fetchBuffer; - private CompositePollEventProcessorContext(LogContext logContext, - NetworkClientDelegate networkClientDelegate, - BackgroundEventHandler backgroundEventHandler, - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper, - FetchBuffer fetchBuffer) { + private AsyncPollEventProcessorContext(LogContext logContext, + NetworkClientDelegate networkClientDelegate, + BackgroundEventHandler backgroundEventHandler, + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + CompletableEventReaper applicationEventReaper, + FetchBuffer fetchBuffer) { this.log = logContext.logger(getClass()); this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; @@ -65,18 +65,18 @@ private CompositePollEventProcessorContext(LogContext logContext, * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. */ - public static Supplier supplier(LogContext logContext, - Supplier networkClientDelegateSupplier, - BackgroundEventHandler backgroundEventHandler, - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper, - FetchBuffer fetchBuffer) { + public static Supplier supplier(LogContext logContext, + Supplier networkClientDelegateSupplier, + BackgroundEventHandler backgroundEventHandler, + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + CompletableEventReaper applicationEventReaper, + FetchBuffer fetchBuffer) { return new CachedSupplier<>() { @Override - protected CompositePollEventProcessorContext create() { + protected AsyncPollEventProcessorContext create() { NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - return new CompositePollEventProcessorContext( + return new AsyncPollEventProcessorContext( logContext, networkClientDelegate, backgroundEventHandler, @@ -117,20 +117,23 @@ public String toString() { /** * Helper method that will check if any application thread user callbacks need to be executed. If so, the - * current event will be completed with {@link CompositePollEvent.State#CALLBACKS_REQUIRED} and this method + * current event will be completed with {@link AsyncPollEvent.State#CALLBACKS_REQUIRED} and this method * will return {@code true}. Otherwise, it will return {@code false}. */ - public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, ApplicationEvent.Type nextEventType) { + public boolean maybeCompleteWithCallbackRequired(AsyncPollEvent event, ApplicationEvent.Type nextEventType) { // If there are background events to process or enqueued callbacks to invoke, exit to // the application thread. if (backgroundEventHandler.size() > 0 || offsetCommitCallbackInvoker.size() > 0) { log.trace( "Pausing polling by completing {} with the state of {} and the next stage of {}", event, - CompositePollEvent.State.CALLBACKS_REQUIRED, + AsyncPollEvent.State.CALLBACKS_REQUIRED, nextEventType ); event.completeWithCallbackRequired(nextEventType); + + // This to ensure that the application thread doesn't needlessly wait for the full time out if it's + // been detected that a callback is required. fetchBuffer.wakeup(); return true; } @@ -142,10 +145,10 @@ public boolean maybeCompleteWithCallbackRequired(CompositePollEvent event, Appli * Helper method that checks if there's a non-null error from * {@link NetworkClientDelegate#getAndClearMetadataError()} or if the provided exception is not a timeout-based * exception. If there's an error to report to the user, the current event will be completed with - * {@link CompositePollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will + * {@link AsyncPollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will * return {@code false}. */ - public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) { + public boolean maybeCompleteExceptionally(AsyncPollEvent event, Throwable t) { if (maybeCompleteExceptionally(event)) return true; @@ -153,7 +156,7 @@ public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) return false; if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { - log.debug("Ignoring timeout for CompositePollEvent {}: {}", event, t.getMessage()); + log.trace("Ignoring timeout for {}: {}", event, t.getMessage()); return false; } @@ -168,10 +171,10 @@ public boolean maybeCompleteExceptionally(CompositePollEvent event, Throwable t) /** * Helper method that checks if there's a non-null error from * {@link NetworkClientDelegate#getAndClearMetadataError()}, and if so, reports it to the user by completing the - * current event with {@link CompositePollEvent.State#FAILED} and returning {@code true}. Otherwise, it will + * current event with {@link AsyncPollEvent.State#FAILED} and returning {@code true}. Otherwise, it will * return {@code false}. */ - public boolean maybeCompleteExceptionally(CompositePollEvent event) { + public boolean maybeCompleteExceptionally(AsyncPollEvent event) { Optional exception = networkClientDelegate.getAndClearMetadataError(); if (exception.isPresent()) { @@ -183,18 +186,18 @@ public boolean maybeCompleteExceptionally(CompositePollEvent event) { } /** - * Helper method to complete the given event with {@link CompositePollEvent.State#FAILED}. + * Helper method to complete the given event with {@link AsyncPollEvent.State#FAILED}. */ - public void completeExceptionally(CompositePollEvent event, Throwable error) { + public void completeExceptionally(AsyncPollEvent event, Throwable error) { KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(error); event.completeExceptionally(e); log.trace("Failing event processing for {}", event, e); } /** - * Helper method to complete the given event with {@link CompositePollEvent.State#SUCCEEDED}. + * Helper method to complete the given event with {@link AsyncPollEvent.State#SUCCEEDED}. */ - public void complete(CompositePollEvent event) { + public void complete(AsyncPollEvent event) { event.completeSuccessfully(); log.trace("Completed event processing for {}", event); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java deleted file mode 100644 index 0feb1c0d6647e..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompositePollEventInvoker.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.AsyncKafkaConsumer; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; - -import org.slf4j.Logger; - -import java.time.Duration; - -import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; - -/** - * {@code CompositePollEventInvoker} is executed on the application thread in the - * {@link AsyncKafkaConsumer#poll(Duration)}. - */ -public class CompositePollEventInvoker { - - private final Logger log; - private final Time time; - private final ApplicationEventHandler applicationEventHandler; - private final Runnable applicationThreadCallbacks; - private CompositePollEvent inflight; - - public CompositePollEventInvoker(LogContext logContext, - Time time, - ApplicationEventHandler applicationEventHandler, - Runnable applicationThreadCallbacks) { - this.log = logContext.logger(getClass()); - this.time = time; - this.applicationEventHandler = applicationEventHandler; - this.applicationThreadCallbacks = applicationThreadCallbacks; - } - - /** - * {@code poll()} manages the lifetime of the {@link CompositePollEvent} processing. If it is called when - * no event is currently processing, it will start a new event processing asynchronously. A check is made - * during each invocation to see if the inflight event has reached a - * {@link CompositePollEvent.State terminal state}. If it has, the result will be processed accordingly. - */ - public void poll(Timer timer) { - if (inflight == null) { - log.trace("No existing inflight event, submitting a new event"); - submitEvent(ApplicationEvent.Type.POLL, timer); - } - - try { - if (log.isTraceEnabled()) { - log.trace( - "Attempting to retrieve result from previously submitted {} with {} remaining on timer", - inflight, - timer.remainingMs() - ); - } - - // Result should be non-null and starts off as State.STARTED. - CompositePollEvent.Result result = inflight.result(); - CompositePollEvent.State state = result.state(); - - if (state == CompositePollEvent.State.SUCCEEDED) { - // The composite event has completed all the requisite stages, though it does not imply that - // there is data in the FetchBuffer yet. Make sure to clear out the inflight request. - log.trace("Event {} completed, clearing inflight", inflight); - inflight = null; - } else if (state == CompositePollEvent.State.FAILED) { - // The composite event failed at one of the stages. Make sure to clear out the inflight request - // before the underlying error is surfaced to the user. - log.trace("Event {} failed, clearing inflight", inflight); - inflight = null; - - throw result.asKafkaException(); - } else if (state == CompositePollEvent.State.CALLBACKS_REQUIRED) { - // The background thread detected that it needed to yield to the application thread to invoke - // callbacks. Even though the inflight reference _should_ be overwritten when the next stage of - // the event is submitted, go ahead and clear out the inflight request just to be sure. - log.trace("Event {} paused for callbacks, clearing inflight", inflight); - inflight = null; - - // Note: this is calling user-supplied code, so make sure to handle possible errors. - applicationThreadCallbacks.run(); - - // The application thread callbacks are complete. Create another event to resume the polling at - // the next stage. - submitEvent(result.asNextEventType(), timer); - } - } catch (Throwable t) { - // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request - // because the error effectively renders it complete. - log.debug("Event {} failed due to {}, clearing inflight", inflight, String.valueOf(t)); - inflight = null; - throw ConsumerUtils.maybeWrapAsKafkaException(t); - } - } - - private void submitEvent(ApplicationEvent.Type type, Timer timer) { - long deadlineMs = calculateDeadlineMs(timer); - long pollTimeMs = time.milliseconds(); - inflight = new CompositePollEvent(deadlineMs, pollTimeMs, type); - applicationEventHandler.add(inflight); - - if (log.isTraceEnabled()) - log.trace("Submitted new {} with {} remaining on timer", inflight, timer.remainingMs()); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SharePollEvent.java similarity index 54% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SharePollEvent.java index 37df5d9ddc210..2db7b18173c01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SharePollEvent.java @@ -16,28 +16,12 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import java.util.concurrent.CompletableFuture; - -public class PollEvent extends ApplicationEvent { +public class SharePollEvent extends ApplicationEvent { private final long pollTimeMs; - /** - * A future that represents the completion of reconciliation and auto-commit - * processing. - * This future is completed when all commit request generation points have - * been passed, including: - *

      - *
    • auto-commit on rebalance
    • - *
    • auto-commit on the interval
    • - *
    - * Once completed, it signals that it's safe for the consumer to proceed with - * fetching new records. - */ - private final CompletableFuture reconcileAndAutoCommit = new CompletableFuture<>(); - - public PollEvent(final long pollTimeMs) { - super(Type.POLL); + public SharePollEvent(final long pollTimeMs) { + super(Type.SHARE_POLL); this.pollTimeMs = pollTimeMs; } @@ -45,14 +29,6 @@ public long pollTimeMs() { return pollTimeMs; } - public CompletableFuture reconcileAndAutoCommit() { - return reconcileAndAutoCommit; - } - - public void markReconcileAndAutoCommitComplete() { - reconcileAndAutoCommit.complete(null); - } - @Override public String toStringBase() { return super.toStringBase() + ", pollTimeMs=" + pollTimeMs; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java index 402697227ee80..891e15846f3e8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.PollEvent; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.LogContext; @@ -61,7 +61,7 @@ public void testRecordApplicationEventQueueSize(String groupName) { asyncConsumerMetrics )) { // add event - applicationEventHandler.add(new PollEvent(time.milliseconds())); + applicationEventHandler.add(new AsyncPollEvent(time.milliseconds() + 10, time.milliseconds())); verify(asyncConsumerMetrics).recordApplicationEventQueueSize(1); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 07a4e0869a67f..47036f5d6eb19 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; import org.apache.kafka.clients.consumer.internals.events.CommitEvent; @@ -43,14 +44,12 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; 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.LeaveGroupOnCloseEvent; 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.ResetOffsetEvent; import org.apache.kafka.clients.consumer.internals.events.SeekUnvalidatedEvent; import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; @@ -427,7 +426,7 @@ public void testWakeupBeforeCallingPoll() { consumer.wakeup(); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -447,7 +446,7 @@ public void testWakeupAfterEmptyFetch() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } @@ -471,7 +470,7 @@ public void testWakeupAfterNonEmptyFetch() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); // the previously ignored wake-up should not be ignored in the next call @@ -508,7 +507,7 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); ConsumerPollTestUtils.waitForCondition( consumer, callbackExecuted::get, @@ -533,7 +532,7 @@ public void testClearWakeupTriggerAfterPoll() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); consumer.poll(Duration.ZERO); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); @@ -679,7 +678,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { completeAssignmentChangeEventSuccessfully(); consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); ConsumerPollTestUtils.waitForCondition( consumer, () -> callback.invoked == 1 && callback.exception == null, @@ -1483,7 +1482,7 @@ public void testListenerCallbacksInvoke(List t.getMessage().equals(expectedException.getMessage()), @@ -1580,7 +1579,7 @@ public void testMultipleBackgroundErrors() { backgroundEventQueue.add(errorEvent2); completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); ConsumerPollTestUtils.waitForException( consumer, t -> t.getMessage().equals(expectedException1.getMessage()), @@ -1665,9 +1664,9 @@ public void testEnsurePollEventSentOnConsumerPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler, atLeastOnce()).add(any(CompositePollEvent.class)); + verify(applicationEventHandler, atLeastOnce()).add(any(AsyncPollEvent.class)); } private Properties requiredConsumerConfigAndGroupId(final String groupId) { @@ -1683,7 +1682,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); consumer.poll(Duration.ZERO); } @@ -1717,7 +1716,7 @@ public void testLongPollWaitIsLimited() { ).when(fetchCollector).collectFetch(any(FetchBuffer.class)); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); assertEquals(2, returnedRecords.count()); @@ -1821,7 +1820,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { // interrupt the thread and call poll try { Thread.currentThread().interrupt(); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); assertThrows(InterruptException.class, () -> consumer.poll(Duration.ZERO)); } finally { // clear interrupted state again since this thread may be reused by JUnit @@ -1853,7 +1852,7 @@ void testReaperInvokedInPoll() { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); ConsumerPollTestUtils.waitForCondition( consumer, @@ -1921,7 +1920,7 @@ public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { completeUnsubscribeApplicationEventSuccessfully(); consumer.assign(singleton(new TopicPartition("topic1", 0))); - markReconcileAndAutoCommitCompleteForPollEvent(); + completeAsyncPollEventSuccessfully(); consumer.poll(Duration.ZERO); verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); @@ -2217,6 +2216,14 @@ private void completeSeekUnvalidatedEventSuccessfully() { }).when(applicationEventHandler).addAndGet(ArgumentMatchers.isA(SeekUnvalidatedEvent.class)); } + private void completeAsyncPollEventSuccessfully() { + doAnswer(invocation -> { + AsyncPollEvent event = invocation.getArgument(0); + event.completeWithCallbackRequired(ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncPollEvent.class)); + } + private void forceCommitCallbackInvocation() { // Invokes callback consumer.commitAsync(); @@ -2294,17 +2301,4 @@ public void testCloseWrapsStreamsRebalanceListenerException() { verify(mockStreamsListener).onTasksRevoked(any()); } } - - private void markReconcileAndAutoCommitCompleteForPollEvent() { - doAnswer(invocation -> { - PollEvent event = invocation.getArgument(0); - event.markReconcileAndAutoCommitComplete(); - return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(PollEvent.class)); - doAnswer(invocation -> { - CompositePollEvent event = invocation.getArgument(0); - event.completeWithCallbackRequired(ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS); - return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CompositePollEvent.class)); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 35ccb17dfab43..88004ebbcd74b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -18,8 +18,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.PollEvent; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.metrics.Metrics; @@ -258,7 +258,7 @@ public void testRunOnceRecordApplicationEventQueueSizeAndApplicationEventQueueTi )) { consumerNetworkThread.initializeResources(); - PollEvent event = new PollEvent(0); + AsyncPollEvent event = new AsyncPollEvent(10, 0); event.setEnqueuedMs(time.milliseconds()); applicationEventQueue.add(event); asyncConsumerMetrics.recordApplicationEventQueueSize(1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java index 5dddd0772df2f..b3833098d6692 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java @@ -24,11 +24,11 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; -import org.apache.kafka.clients.consumer.internals.events.PollEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackEvent; import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackRegistrationEvent; import org.apache.kafka.clients.consumer.internals.events.ShareFetchEvent; +import org.apache.kafka.clients.consumer.internals.events.SharePollEvent; import org.apache.kafka.clients.consumer.internals.events.ShareSubscriptionChangeEvent; import org.apache.kafka.clients.consumer.internals.events.ShareUnsubscribeEvent; import org.apache.kafka.clients.consumer.internals.events.StopFindCoordinatorOnCloseEvent; @@ -680,7 +680,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { consumer.subscribe(subscriptionTopic); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(PollEvent.class)); + verify(applicationEventHandler).add(any(SharePollEvent.class)); verify(applicationEventHandler).addAndGet(any(ShareSubscriptionChangeEvent.class)); completeShareAcknowledgeOnCloseApplicationEventSuccessfully(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index dde3f567132fc..654c4772d7c61 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -111,7 +111,8 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState + subscriptionState, + Optional.of(mock(AsyncPollEventProcessorContext.class)) ); } @@ -171,7 +172,7 @@ public void testApplicationEventIsProcessed(ApplicationEvent e) { private static Stream applicationEvents() { return Stream.of( - Arguments.of(new PollEvent(100)), + Arguments.of(new AsyncPollEvent(calculateDeadlineMs(12345, 100), 100)), Arguments.of(new CreateFetchRequestsEvent(calculateDeadlineMs(12345, 100))), Arguments.of(new CheckAndUpdatePositionsEvent(500)), Arguments.of(new TopicMetadataEvent("topic", Long.MAX_VALUE)), @@ -265,12 +266,12 @@ public void testSeekUnvalidatedEventWithException() { @Test public void testPollEvent() { - PollEvent event = new PollEvent(12345); + AsyncPollEvent event = new AsyncPollEvent(12346, 12345); setupProcessor(true); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); + when(offsetsRequestManager.updateFetchPositions(anyLong())).thenReturn(new CompletableFuture<>()); processor.process(event); - assertTrue(event.reconcileAndAutoCommit().isDone()); verify(commitRequestManager).updateTimerAndMaybeCommit(12345); verify(membershipManager).onConsumerPoll(); verify(heartbeatRequestManager).resetPollTimer(12345); From 0256bdf274d22567769a1193bde9face2dd057f7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 2 Oct 2025 15:54:21 -0700 Subject: [PATCH 054/123] Moving the regex subscription check out of the poll path --- .../internals/AbstractMembershipManager.java | 11 +++++++++++ .../events/ApplicationEventProcessor.java | 17 +++++++---------- .../internals/events/AsyncPollEvent.java | 3 +-- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index ffe01c089e7bf..e1ad2f6889af0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -43,6 +43,7 @@ import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static java.util.Collections.unmodifiableList; @@ -487,6 +488,16 @@ public void onSubscriptionUpdated() { * active call to {@link org.apache.kafka.clients.consumer.KafkaConsumer#poll(Duration)}" */ public void onConsumerPoll() { + if (subscriptions.hasPatternSubscription()) { + final Set topicsToSubscribe = metadata.fetch().topics().stream() + .filter(subscriptions::matchesSubscribedPattern) + .collect(Collectors.toSet()); + if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { + metadata.requestUpdateForNewTopics(); + } + subscriptionUpdated.compareAndSet(false, true); + } + if (subscriptionUpdated.compareAndSet(true, false) && state == MemberState.UNSUBSCRIBED) { transitionToJoining(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index c241cd5eb8b0f..c664aa7f98d8d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -411,7 +411,13 @@ private void process(final TopicRe2JPatternSubscriptionChangeEvent event) { * This will make the consumer send the updated subscription on the next poll. */ private void process(final UpdatePatternSubscriptionEvent event) { - processUpdatePatternSubscriptionEvent(); + if (!subscriptions.hasPatternSubscription()) { + return; + } + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + updatePatternSubscription(metadata.fetch()); + } event.future().complete(null); } @@ -761,15 +767,6 @@ private void process(final AsyncPollEvent event) { }); } - nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA; - - if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) - return; - } - - if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) { - log.debug("Processing {} logic for {}", nextEventType, event); - processUpdatePatternSubscriptionEvent(); nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index e412016ae886c..bcc0f742f2f7c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -132,8 +132,7 @@ public int hashCode() { private static final List ALLOWED_STARTING_EVENT_TYPES = List.of( Type.ASYNC_POLL, - Type.CHECK_AND_UPDATE_POSITIONS, - Type.UPDATE_SUBSCRIPTION_METADATA + Type.CHECK_AND_UPDATE_POSITIONS ); private final long deadlineMs; private final long pollTimeMs; From a233e90d5d4ea2bd23276180db3c30901d91b78f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 2 Oct 2025 21:25:42 -0700 Subject: [PATCH 055/123] Remove unused processUpdatePatternSubscriptionEvent method Deleted the private processUpdatePatternSubscriptionEvent method from ApplicationEventProcessor as it is no longer used. --- .../internals/events/ApplicationEventProcessor.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index c664aa7f98d8d..7965257f7745d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -421,16 +421,6 @@ private void process(final UpdatePatternSubscriptionEvent event) { event.future().complete(null); } - private void processUpdatePatternSubscriptionEvent() { - if (!subscriptions.hasPatternSubscription()) { - return; - } - if (this.metadataVersionSnapshot < metadata.updateVersion()) { - this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(metadata.fetch()); - } - } - /** * Process event indicating that the consumer unsubscribed from all topics. This will make * the consumer release its assignment and send a request to leave the group. From 2b2f70c36e688bed61ce0ce0c6eb6cd98f8b8e40 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 3 Oct 2025 10:40:52 -0700 Subject: [PATCH 056/123] Using Consumer.poll() helper methods from TestUtils --- .../api/PlaintextAdminIntegrationTest.scala | 46 +++++++++++++------ ...aslClientsWithInvalidCredentialsTest.scala | 33 +++++++------ .../server/GssapiAuthenticationTest.scala | 15 +++--- .../scala/unit/kafka/utils/TestUtils.scala | 15 ++++++ 4 files changed, 70 insertions(+), 39 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 1f2c003b789b8..3e40e80feff87 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -35,7 +35,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer -import org.apache.kafka.clients.consumer.{CommitFailedException, Consumer, ConsumerConfig, GroupProtocol, KafkaConsumer, OffsetAndMetadata, ShareConsumer} +import org.apache.kafka.clients.consumer.{CommitFailedException, Consumer, ConsumerConfig, ConsumerRecords, GroupProtocol, KafkaConsumer, OffsetAndMetadata, ShareConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, SslConfigs, TopicConfig} @@ -568,10 +568,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { consumer.assign(util.Set.of(tp)) consumer.seekToBeginning(util.Set.of(tp)) - TestUtils.waitUntilTrue(() => { - val records = consumer.poll(time.Duration.ofSeconds(3)) + def verifyRecordCount(records: ConsumerRecords[Array[Byte], Array[Byte]]): Boolean = { expectedNumber == records.count() - }, s"Consumer.poll() did not return the expected number of records ($expectedNumber) within the timeout") + } + TestUtils.pollRecordsUntilTrue( + consumer, + verifyRecordCount, + s"Consumer.poll() did not return the expected number of records ($expectedNumber) within the timeout", + pollTimeoutMs = 3000 + ) } finally consumer.close() } @@ -4624,10 +4629,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { var counter = 0 - TestUtils.waitUntilTrue(() => { - counter += streams.poll(JDuration.ofMillis(100L)).count() + def verifyRecordCount(records: ConsumerRecords[Nothing, Nothing]): Boolean = { + counter += records.count() counter >= numRecords - }, "Consumer not assigned to partitions") + } + TestUtils.pollRecordsUntilTrue( + streams, + verifyRecordCount, + s"Consumer not assigned to partitions" + ) streams.commitSync() @@ -4691,10 +4701,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { var counter = 0 - TestUtils.waitUntilTrue(() => { - counter += streams.poll(JDuration.ofMillis(100L)).count() + def verifyRecordCount(records: ConsumerRecords[Nothing, Nothing]): Boolean = { + counter += records.count() counter >= numRecords - }, "Consumer not assigned to partitions") + } + TestUtils.pollRecordsUntilTrue( + streams, + verifyRecordCount, + s"Consumer not assigned to partitions" + ) streams.commitSync() @@ -4775,10 +4790,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { var counter = 0 - TestUtils.waitUntilTrue(() => { - counter += streams.poll(JDuration.ofMillis(100L)).count() + def verifyRecordCount(records: ConsumerRecords[Nothing, Nothing]): Boolean = { + counter += records.count() counter >= numRecords - }, "Consumer not assigned to partitions") + } + TestUtils.pollRecordsUntilTrue( + streams, + verifyRecordCount, + s"Consumer not assigned to partitions" + ) streams.commitSync() diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index bbf76fdc0784c..e1a81736b1db7 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -14,11 +14,10 @@ package kafka.api import kafka.security.JaasTestUtils -import java.time.Duration import java.util.Properties import java.util.concurrent.{ExecutionException, TimeUnit} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.SaslAuthenticationException @@ -146,14 +145,12 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { private def verifyConsumerWithAuthenticationFailure(consumer: Consumer[Array[Byte], Array[Byte]]): Unit = { val startMs = System.currentTimeMillis - TestUtils.waitUntilTrue(() => { - try { - consumer.poll(Duration.ofMillis(1000)) - false - } catch { - case _: Exception => true - } - }, s"Consumer.poll() did not throw an exception within the timeout") + TestUtils.pollUntilException( + consumer, + _ => true, + s"Consumer.poll() did not throw an exception within the timeout", + pollTimeoutMs = 1000 + ) val elapsedMs = System.currentTimeMillis - startMs assertTrue(elapsedMs <= 5000, s"Poll took too long, elapsed=$elapsedMs") verifyAuthenticationException(consumer.partitionsFor(topic)) @@ -161,13 +158,15 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { createClientCredential() val producer = createProducer() verifyWithRetry(sendOneRecord(producer))() - TestUtils.waitUntilTrue(() => { - try { - consumer.poll(Duration.ofMillis(1000)).count == 1 - } catch { - case _: SaslAuthenticationException => false - } - }, s"Consumer.poll() did not read the expected number of records within the timeout") + def verifyRecordCount(records: ConsumerRecords[Array[Byte], Array[Byte]]): Boolean = { + records.count() == 1 + } + TestUtils.pollRecordsUntilTrue( + consumer, + verifyRecordCount, + s"Consumer.poll() did not read the expected number of records within the timeout", + pollTimeoutMs = 1000 + ) } @Test diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 0de21d72ec0d4..fe7d8fb441b00 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -19,7 +19,6 @@ package kafka.server import java.net.InetSocketAddress -import java.time.Duration import java.util.Properties import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import javax.security.auth.login.LoginContext @@ -185,14 +184,12 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { consumer.assign(java.util.List.of(tp)) val startMs = System.currentTimeMillis() - TestUtils.waitUntilTrue(() => { - try { - consumer.poll(Duration.ofMillis(50)) - false - } catch { - case _: SaslAuthenticationException => true - } - }, "Consumer.poll() did not trigger a SaslAuthenticationException within timeout") + TestUtils.pollUntilException( + consumer, + t => t.isInstanceOf[SaslAuthenticationException], + "Consumer.poll() did not trigger a SaslAuthenticationException within timeout", + pollTimeoutMs = 50 + ) val endMs = System.currentTimeMillis() require(endMs - startMs < failedAuthenticationDelayMs, "Failed authentication must not be delayed on the client") consumer.close() diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 8b0affae9eab9..63d0c3e49a961 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -690,6 +690,21 @@ object TestUtils extends Logging { }, msg = msg, pause = 0L, waitTimeMs = waitTimeMs) } + def pollUntilException(consumer: Consumer[_, _], + action: Throwable => Boolean, + msg: => String, + waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, + pollTimeoutMs: Long = 100): Unit = { + waitUntilTrue(() => { + try { + consumer.poll(Duration.ofMillis(pollTimeoutMs)) + false + } catch { + case t: Throwable => action(t) + } + }, msg = msg, pause = 0L, waitTimeMs = waitTimeMs) + } + def pollRecordsUntilTrue[K, V](consumer: Consumer[K, V], action: ConsumerRecords[K, V] => Boolean, msg: => String, From 461ffdd9b00a0ee9cb2b54066e5d89d717ce6d6e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 3 Oct 2025 10:41:11 -0700 Subject: [PATCH 057/123] Change debug to trace in ApplicationEventProcessor --- .../internals/events/ApplicationEventProcessor.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 7965257f7745d..619f670bd2c7f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -736,7 +736,7 @@ private void process(final AsyncPollEvent event) { return; if (nextEventType == ApplicationEvent.Type.ASYNC_POLL) { - log.debug("Processing {} logic for {}", nextEventType, event); + log.trace("Processing {} logic for {}", nextEventType, event); // Trigger a reconciliation that can safely commit offsets if needed to rebalance, // as we're processing before any new fetching starts @@ -764,7 +764,7 @@ private void process(final AsyncPollEvent event) { } if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.debug("Processing {} logic for {}", nextEventType, event); + log.trace("Processing {} logic for {}", nextEventType, event); CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); context.trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); @@ -772,7 +772,7 @@ private void process(final AsyncPollEvent event) { if (context.maybeCompleteExceptionally(event, updatePositionsError)) return; - log.debug("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); + log.trace("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); // Create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { From 9fb9ee9e6e250fbdc075f64fbfdbefc1c7cdbb39 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 3 Oct 2025 16:35:46 -0700 Subject: [PATCH 058/123] Refactoring --- .../internals/ConsumerNetworkThread.java | 51 ++++++++++---- .../events/AbstractTopicMetadataEvent.java | 6 +- .../events/ApplicationEventProcessor.java | 4 +- .../internals/events/AsyncPollEvent.java | 8 ++- .../events/CheckAndUpdatePositionsEvent.java | 11 +-- .../events/CompletableApplicationEvent.java | 4 -- .../internals/events/ListOffsetsEvent.java | 6 +- .../events/MetadataErrorNotifiable.java | 28 ++++++++ .../clients/consumer/KafkaConsumerTest.java | 67 +++++++------------ 9 files changed, 113 insertions(+), 72 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index d2d178a88c38b..1dbe15db9d9a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,9 +20,9 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; +import org.apache.kafka.clients.consumer.internals.events.MetadataErrorNotifiable; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; @@ -40,6 +40,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; @@ -191,13 +192,17 @@ private void processApplicationEvents() { for (ApplicationEvent event : events) { asyncConsumerMetrics.recordApplicationEventQueueTime(time.milliseconds() - event.enqueuedMs()); try { - if (event instanceof CompletableEvent) { + if (event instanceof CompletableEvent) applicationEventReaper.add((CompletableEvent) event); - // Check if there are any metadata errors and fail the CompletableEvent if an error is present. - // This call is meant to handle "immediately completed events" which may not enter the awaiting state, - // so metadata errors need to be checked and handled right away. - maybeFailOnMetadataError(List.of((CompletableEvent) event)); + + // Check if there are any metadata errors and fail the CompletableEvent if an error is present. + // This call is meant to handle "immediately completed events" which may not enter the awaiting state, + // so metadata errors need to be checked and handled right away. + if (event instanceof MetadataErrorNotifiable) { + if (maybeFailOnMetadataError((MetadataErrorNotifiable) event)) + continue; } + applicationEventProcessor.process(event); } catch (Throwable t) { log.warn("Error processing event {}", t.getMessage(), t); @@ -369,17 +374,37 @@ void cleanup() { * If there is a metadata error, complete all uncompleted events that require subscription metadata. */ private void maybeFailOnMetadataError(List> events) { - List> subscriptionMetadataEvent = new ArrayList<>(); + List notifiables = new ArrayList<>(); for (CompletableEvent ce : events) { - if (ce instanceof CompletableApplicationEvent && ((CompletableApplicationEvent) ce).requireSubscriptionMetadata()) - subscriptionMetadataEvent.add((CompletableApplicationEvent) ce); + if (ce instanceof MetadataErrorNotifiable) { + notifiables.add((MetadataErrorNotifiable) ce); + } } - if (subscriptionMetadataEvent.isEmpty()) + if (notifiables.isEmpty()) return; - networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError -> - subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) - ); + + Optional metadataErrorOpt = networkClientDelegate.getAndClearMetadataError(); + + if (metadataErrorOpt.isEmpty()) + return; + + Exception metadataError = metadataErrorOpt.get(); + notifiables.forEach(n -> n.metadataError(metadataError)); + } + + /** + * If there is a metadata error, complete all uncompleted events that require subscription metadata. + */ + private boolean maybeFailOnMetadataError(MetadataErrorNotifiable notifiable) { + Optional metadataErrorOpt = networkClientDelegate.getAndClearMetadataError(); + + if (metadataErrorOpt.isEmpty()) + return false; + + Exception metadataError = metadataErrorOpt.get(); + notifiable.metadataError(metadataError); + return true; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index cb23e6aaf2826..5c605d7efa2e7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -21,14 +21,14 @@ import java.util.List; import java.util.Map; -public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { +public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> implements MetadataErrorNotifiable { protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) { super(type, deadlineMs); } @Override - public boolean requireSubscriptionMetadata() { - return true; + public void metadataError(Exception metadataException) { + future().completeExceptionally(metadataException); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 619f670bd2c7f..abd05597e64aa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -732,7 +732,7 @@ private void process(final AsyncPollEvent event) { AsyncPollEventProcessorContext context = asyncPollContext.orElseThrow(IllegalArgumentException::new); ApplicationEvent.Type nextEventType = event.startingEventType(); - if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) + if (context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; if (nextEventType == ApplicationEvent.Type.ASYNC_POLL) { @@ -759,7 +759,7 @@ private void process(final AsyncPollEvent event) { nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; - if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType)) + if (context.maybeCompleteWithCallbackRequired(event, nextEventType)) return; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index bcc0f742f2f7c..2a5b4e7cf7102 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; import java.time.Duration; @@ -58,7 +59,7 @@ * application thread. The background thread is able to detect when it needs to complete processing so that the * application thread can execute the awaiting callbacks. */ -public class AsyncPollEvent extends ApplicationEvent { +public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiable { public enum State { @@ -201,6 +202,11 @@ public void completeWithCallbackRequired(Type nextEventType) { result.compareAndSet(Result.STARTED, r); } + @Override + public void metadataError(Exception metadataException) { + completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(metadataException)); + } + @Override protected String toStringBase() { return super.toStringBase() + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java index 5f1ced33e3a09..9e0a60beadc92 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.TopicPartition; @@ -30,7 +31,7 @@ * The event completes with a boolean indicating if all assigned partitions have valid fetch positions * (based on {@link SubscriptionState#hasAllFetchPositions()}). */ -public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent { +public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent implements MetadataErrorNotifiable { public CheckAndUpdatePositionsEvent(long deadlineMs) { super(Type.CHECK_AND_UPDATE_POSITIONS, deadlineMs); @@ -39,11 +40,11 @@ public CheckAndUpdatePositionsEvent(long deadlineMs) { /** * Indicates that this event requires subscription metadata to be present * for its execution. This is used to ensure that metadata errors are - * handled correctly during the {@link org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer#poll(Duration) poll} - * or {@link org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer#position(TopicPartition) position} process. + * handled correctly during the {@link Consumer#poll(Duration) poll} + * or {@link Consumer#position(TopicPartition) position} process. */ @Override - public boolean requireSubscriptionMetadata() { - return true; + public void metadataError(Exception metadataException) { + future().completeExceptionally(metadataException); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 51b2d1ffbdb90..8cd17d19feb66 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -52,8 +52,4 @@ public long deadlineMs() { protected String toStringBase() { return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } - - public boolean requireSubscriptionMetadata() { - return false; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 605a2ff30c24a..1bcd73d07596b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -32,7 +32,7 @@ * {@link OffsetAndTimestamp} found (offset of the first message whose timestamp is greater than * or equals to the target timestamp) */ -public class ListOffsetsEvent extends CompletableApplicationEvent> { +public class ListOffsetsEvent extends CompletableApplicationEvent> implements MetadataErrorNotifiable { private final Map timestampsToSearch; private final boolean requireTimestamps; @@ -65,8 +65,8 @@ public boolean requireTimestamps() { } @Override - public boolean requireSubscriptionMetadata() { - return true; + public void metadataError(Exception metadataException) { + future().completeExceptionally(metadataException); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java new file mode 100644 index 0000000000000..153bd47ace8c4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java @@ -0,0 +1,28 @@ +/* + * 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.NetworkClientDelegate; + +/** + * This interface is used for events that need to be notified when the + * {@link NetworkClientDelegate#getAndClearMetadataError()} has an error. + */ +public interface MetadataErrorNotifiable { + + void metadataError(Exception metadataException); +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 4cc2d67c39125..d4281f6b29a84 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1063,7 +1063,7 @@ private void initMetadata(MockClient mockClient, Map partitionC @ParameterizedTest @EnumSource(value = GroupProtocol.class) - public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) throws InterruptedException { + public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) { SubscriptionState subscription = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE); ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1079,15 +1079,14 @@ public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) throws I true, groupId, groupInstanceId, false); consumer.assign(List.of(tp0)); - if (groupProtocol == GroupProtocol.CONSUMER) { - // New consumer poll(ZERO) needs to wait for the offset fetch event added by a call to poll, to be processed - // by the background thread, so it can realize there are no committed offsets and then - // throw the NoOffsetForPartitionException - assertPollEventuallyThrows(consumer, NoOffsetForPartitionException.class, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); - } else { - assertThrows(NoOffsetForPartitionException.class, () -> consumer.poll(Duration.ZERO)); - } + // Consumer.poll(0) needs to wait for the offset fetch event added by a call to poll, to be processed + // by the background thread, so it can realize there are no committed offsets and then + // throw the NoOffsetForPartitionException. + ConsumerPollTestUtils.waitForException( + consumer, + NoOffsetForPartitionException.class::isInstance, + "Consumer was not able to update fetch positions on continuous calls with 0 timeout" + ); } @ParameterizedTest @@ -2267,19 +2266,18 @@ public void testEndOffsetsAuthenticationFailure(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testPollAuthenticationFailure(GroupProtocol groupProtocol) throws InterruptedException { + public void testPollAuthenticationFailure(GroupProtocol groupProtocol) { final KafkaConsumer consumer = consumerWithPendingAuthenticationError(groupProtocol); consumer.subscribe(Set.of(topic)); - if (groupProtocol == GroupProtocol.CONSUMER) { - // New consumer poll(ZERO) needs to wait for the event added by a call to poll, to be processed - // by the background thread, so it can realize there is authentication fail and then - // throw the AuthenticationException - assertPollEventuallyThrows(consumer, AuthenticationException.class, - "this consumer was not able to discover metadata errors during continuous polling."); - } else { - assertThrows(AuthenticationException.class, () -> consumer.poll(Duration.ZERO)); - } + // Consumer.poll(0) needs to wait for the event added by a call to poll, to be processed + // by the background thread, so it can realize there is authentication fail and then + // throw the AuthenticationException. + ConsumerPollTestUtils.waitForException( + consumer, + AuthenticationException.class::isInstance, + "this consumer was not able to discover metadata errors during continuous polling." + ); } // TODO: this test triggers a bug with the CONSUMER group protocol implementation. @@ -3194,27 +3192,14 @@ public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws I KafkaConsumer consumer = newConsumer(groupProtocol, time, client, subscription, metadata, assignor, true, groupInstanceId); consumer.subscribe(Set.of(invalidTopicName), getConsumerRebalanceListener(consumer)); - if (groupProtocol == GroupProtocol.CONSUMER) { - // New consumer poll(ZERO) needs to wait for the event added by a call to poll, to be processed - // by the background thread, so it can realize there is invalid topics and then - // throw the InvalidTopicException - assertPollEventuallyThrows(consumer, InvalidTopicException.class, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); - } else { - assertThrows(InvalidTopicException.class, () -> consumer.poll(Duration.ZERO)); - } - } - - private static void assertPollEventuallyThrows(KafkaConsumer consumer, - Class expectedException, String errMsg) throws InterruptedException { - TestUtils.waitForCondition(() -> { - try { - consumer.poll(Duration.ZERO); - return false; - } catch (Throwable exception) { - return expectedException.isInstance(exception); - } - }, errMsg); + // Consumer.poll(0) needs to wait for the event added by a call to poll, to be processed + // by the background thread, so it can realize there is invalid topics and then + // throw the InvalidTopicException. + ConsumerPollTestUtils.waitForException( + consumer, + InvalidTopicException.class::isInstance, + "Consumer was not able to update fetch positions on continuous calls with 0 timeout" + ); } @ParameterizedTest From a8ccdb6f48dd959195a1046993c73dc4f220e4a6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 3 Oct 2025 19:30:22 -0700 Subject: [PATCH 059/123] Replace pollRecordsUntilTrue with waitUntilTrue in verifyConsumerWithAuthenticationFailure --- ...aslClientsWithInvalidCredentialsTest.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index e1a81736b1db7..7c9c8e577eee5 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -13,11 +13,12 @@ package kafka.api import kafka.security.JaasTestUtils +import kafka.utils.TestUtils.waitUntilTrue import java.util.Properties import java.util.concurrent.{ExecutionException, TimeUnit} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.SaslAuthenticationException @@ -32,6 +33,7 @@ import org.apache.kafka.metadata.storage.Formatter import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{MethodSource, ValueSource} +import java.time.Duration import scala.jdk.javaapi.OptionConverters import scala.util.Using @@ -158,15 +160,13 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { createClientCredential() val producer = createProducer() verifyWithRetry(sendOneRecord(producer))() - def verifyRecordCount(records: ConsumerRecords[Array[Byte], Array[Byte]]): Boolean = { - records.count() == 1 - } - TestUtils.pollRecordsUntilTrue( - consumer, - verifyRecordCount, - s"Consumer.poll() did not read the expected number of records within the timeout", - pollTimeoutMs = 1000 - ) + waitUntilTrue(() => { + try { + consumer.poll(Duration.ofMillis(1000)).count() == 1 + } catch { + case _:Throwable => false + } + }, msg = s"Consumer.poll() did not read the expected number of records within the timeout") } @Test From e1cf7b70564ef44cb5d7450f4bc3033586243839 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 3 Oct 2025 19:44:58 -0700 Subject: [PATCH 060/123] Replace awaitNonEmptyRecords with waitForRecords in PlaintextConsumerTest --- .../consumer/PlaintextConsumerTest.java | 35 +++++-------------- 1 file changed, 8 insertions(+), 27 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index 1782dd13c8dd8..323be036e08eb 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -66,7 +66,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.IntStream; import static org.apache.kafka.clients.ClientsTestUtils.BaseConsumerTestcase.BROKER_COUNT; @@ -810,7 +809,7 @@ private void testPerPartitionLeadMetricsCleanUpWithSubscribe( // Create a consumer and consumer some messages. var listener = new TestConsumerReassignmentListener(); consumer.subscribe(List.of(TOPIC, topic2), listener); - var records = awaitNonEmptyRecords(consumer, TP); + var records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(1, listener.callsToAssigned, "should be assigned once"); // Verify the metric exist. @@ -877,7 +876,7 @@ private void testPerPartitionLagMetricsCleanUpWithSubscribe( // Create a consumer and consumer some messages. var listener = new TestConsumerReassignmentListener(); consumer.subscribe(List.of(TOPIC, topic2), listener); - var records = awaitNonEmptyRecords(consumer, TP); + var records = ConsumerPollTestUtils.waitForRecords(consumer); assertEquals(1, listener.callsToAssigned, "should be assigned once"); // Verify the metric exist. @@ -944,7 +943,7 @@ private void testPerPartitionLeadMetricsCleanUpWithAssign( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - var records = awaitNonEmptyRecords(consumer, TP); + var records = ConsumerPollTestUtils.waitForRecords(consumer); // Verify the metric exist. Map tags = Map.of( @@ -958,7 +957,7 @@ private void testPerPartitionLeadMetricsCleanUpWithAssign( assertEquals((double) records.count(), fetchLead.metricValue(), "The lead should be " + records.count()); consumer.assign(List.of(tp2)); - awaitNonEmptyRecords(consumer, tp2); + ConsumerPollTestUtils.waitForRecords(consumer); assertNull(consumer.metrics().get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags))); } } @@ -999,7 +998,7 @@ private void testPerPartitionLagMetricsCleanUpWithAssign( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - var records = awaitNonEmptyRecords(consumer, TP); + var records = ConsumerPollTestUtils.waitForRecords(consumer); // Verify the metric exist. Map tags = Map.of( @@ -1014,7 +1013,7 @@ private void testPerPartitionLagMetricsCleanUpWithAssign( var expectedLag = numMessages - records.count(); assertEquals(expectedLag, (double) fetchLag.metricValue(), EPSILON, "The lag should be " + expectedLag); consumer.assign(List.of(tp2)); - awaitNonEmptyRecords(consumer, tp2); + ConsumerPollTestUtils.waitForRecords(consumer); assertNull(consumer.metrics().get(new MetricName(TP + ".records-lag", "consumer-fetch-manager-metrics", "", tags))); assertNull(consumer.metrics().get(new MetricName("records-lag", "consumer-fetch-manager-metrics", "", tags))); } @@ -1058,7 +1057,7 @@ private void testPerPartitionLagMetricsWhenReadCommitted( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - awaitNonEmptyRecords(consumer, TP); + ConsumerPollTestUtils.waitForRecords(consumer); // Verify the metric exist. Map tags = Map.of( @@ -1655,7 +1654,7 @@ private void testStallBetweenPoll(GroupProtocol groupProtocol) throws Exception consumer.subscribe(List.of(testTopic)); // This is here to allow the consumer time to settle the group membership/assignment. - awaitNonEmptyRecords(consumer, new TopicPartition(testTopic, 0)); + ConsumerPollTestUtils.waitForRecords(consumer); // Keep track of the last time the poll is invoked to ensure the deltas between invocations don't // exceed the delay threshold defined above. @@ -1675,24 +1674,6 @@ private void testStallBetweenPoll(GroupProtocol groupProtocol) throws Exception } } - private ConsumerRecords awaitNonEmptyRecords( - Consumer consumer, - TopicPartition tp - ) throws Exception { - AtomicReference> result = new AtomicReference<>(); - - TestUtils.waitForCondition(() -> { - var polledRecords = consumer.poll(Duration.ofSeconds(10)); - boolean hasRecords = !polledRecords.isEmpty(); - if (hasRecords) { - result.set(polledRecords); - } - return hasRecords; - }, "Timed out waiting for non-empty records from topic " + tp.topic() + " partition " + tp.partition()); - - return result.get(); - } - public static class SerializerImpl implements Serializer { private final ByteArraySerializer serializer = new ByteArraySerializer(); From eace3ee1d60a369e0c3b4c622bdbc1363a5c3954 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 5 Oct 2025 19:26:07 -0700 Subject: [PATCH 061/123] Add detailed Javadoc to MetadataErrorNotifiable Expanded the interface documentation to describe how metadata errors are detected and handled in the ConsumerNetworkThread loop, including when and how the metadataError method is invoked. --- .../events/MetadataErrorNotifiable.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java index 153bd47ace8c4..169f0b0b57648 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; /** @@ -24,5 +26,31 @@ */ public interface MetadataErrorNotifiable { + /** + * The background thread detects metadata errors on every call to {@link NetworkClientDelegate#poll(long, long)}. + * {@link NetworkClientDelegate} calls {@link Metadata#maybeThrowAnyException()} and stores the result. + * The presence of a metadata error is checked in the {@link ConsumerNetworkThread}'s loop by calling + * {@link NetworkClientDelegate#getAndClearMetadataError()}. There are two places in the loop in which the + * metadata error is checked: + * + *
      + *
    • + * At the very top of the {@link ConsumerNetworkThread}'s loop, the {@link ApplicationEventHandler}'s + * queue is drained. Before processing each event via + * {@link ApplicationEventProcessor#process(ApplicationEvent)}, if a metadata error occurred, this method + * will be invoked on the event if it implements this interface. + *

      + * Note: for an event on which this method is invoked, it will not be passed to the + * {@link ApplicationEventProcessor#process(ApplicationEvent)} method. + *

    • + *
    • + * At the very bottom of the {@link ConsumerNetworkThread}'s loop, the {@link CompletableEventReaper} + * is executed and any outstanding event is returned. If a metadata error occurred, this method + * will be invoked on all unexpired events if it implements this interface. + *
    • + *
    + * + * @param metadataException Error that originally came from {@link Metadata#maybeThrowAnyException()} + */ void metadataError(Exception metadataException); } From 767316ba60e2a329500c20ed810d9a1d2cf1e636 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 5 Oct 2025 21:06:08 -0700 Subject: [PATCH 062/123] Refactoring to remove interim callback step --- .../consumer/ConsumerIntegrationTest.java | 2 +- .../internals/AsyncKafkaConsumer.java | 47 +--- .../events/ApplicationEventProcessor.java | 141 +++++++----- .../internals/events/AsyncPollEvent.java | 94 ++------ .../AsyncPollEventProcessorContext.java | 204 ------------------ .../internals/AsyncKafkaConsumerTest.java | 2 +- .../events/ApplicationEventProcessorTest.java | 3 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- 8 files changed, 116 insertions(+), 379 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java index 220866c240f4a..f8621bc9133a9 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -122,7 +122,7 @@ public void onPartitionsAssigned(Collection partitions) { } }); - TestUtils.waitForCondition(() -> consumer.poll(Duration.ofSeconds(1)).count() == 1, + TestUtils.waitForCondition(() -> consumer.poll(Duration.ofMillis(100)).count() == 1, 5000, "failed to poll data"); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e3f9eabbe98ef..2ab5f24d17839 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -42,7 +42,6 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncPollEventProcessorContext; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; @@ -462,19 +461,11 @@ public AsyncKafkaConsumer(final ConsumerConfig config, streamsRebalanceData ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - final Supplier asyncPollContextSupplier = AsyncPollEventProcessorContext.supplier( - logContext, - networkClientDelegateSupplier, - backgroundEventHandler, - offsetCommitCallbackInvoker, - applicationEventReaper, - fetchBuffer - ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, subscriptions, requestManagersSupplier, - asyncPollContextSupplier + applicationEventReaper ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, @@ -664,20 +655,12 @@ public AsyncKafkaConsumer(final ConsumerConfig config, Optional.empty() ); final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); - final Supplier asyncPollContextSupplier = AsyncPollEventProcessorContext.supplier( - logContext, - networkClientDelegateSupplier, - backgroundEventHandler, - offsetCommitCallbackInvoker, - applicationEventReaper, - fetchBuffer - ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, subscriptions, requestManagersSupplier, - asyncPollContextSupplier + applicationEventReaper ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, @@ -897,10 +880,14 @@ public ConsumerRecords poll(final Duration timeout) { public void checkInflightPollResult(Timer timer) { if (inflightPoll == null) { log.trace("No existing inflight async poll event, submitting a new event"); - submitEvent(ApplicationEvent.Type.ASYNC_POLL, timer); + submitEvent(timer); } try { + // Note: this is calling user-supplied code, so make sure to handle possible errors. + offsetCommitCallbackInvoker.executeCallbacks(); + processBackgroundEvents(); + if (log.isTraceEnabled()) { log.trace( "Attempting to retrieve result from previously submitted {} with {} remaining on timer", @@ -924,21 +911,7 @@ public void checkInflightPollResult(Timer timer) { log.trace("Event {} failed, clearing inflight", inflightPoll); inflightPoll = null; - throw result.asKafkaException(); - } else if (state == AsyncPollEvent.State.CALLBACKS_REQUIRED) { - // The background thread detected that it needed to yield to the application thread to invoke - // callbacks. Even though the inflight reference _should_ be overwritten when the next stage of - // the event is submitted, go ahead and clear out the inflight request just to be sure. - log.trace("Event {} paused for callbacks, clearing inflight", inflightPoll); - inflightPoll = null; - - // Note: this is calling user-supplied code, so make sure to handle possible errors. - offsetCommitCallbackInvoker.executeCallbacks(); - processBackgroundEvents(); - - // The application thread callbacks are complete. Create another event to resume the polling at - // the next stage. - submitEvent(result.asNextEventType(), timer); + throw result.error(); } } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request @@ -949,10 +922,10 @@ public void checkInflightPollResult(Timer timer) { } } - private void submitEvent(ApplicationEvent.Type type, Timer timer) { + private void submitEvent(Timer timer) { long deadlineMs = calculateDeadlineMs(timer); long pollTimeMs = time.milliseconds(); - inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs, type); + inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs); applicationEventHandler.add(inflightPoll); if (log.isTraceEnabled()) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index abd05597e64aa..04cd7ec6abdd7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -19,9 +19,11 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.Acknowledgements; import org.apache.kafka.clients.consumer.internals.CachedSupplier; +import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; @@ -46,6 +48,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -61,19 +64,19 @@ public class ApplicationEventProcessor implements EventProcessor asyncPollContext; + private final Optional applicationEventReaper; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final Optional asyncPollContext) { + final Optional applicationEventReaper) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; - this.asyncPollContext = asyncPollContext; + this.applicationEventReaper = applicationEventReaper; this.metadataVersionSnapshot = metadata.updateVersion(); } @@ -88,8 +91,8 @@ public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final AsyncPollEventProcessorContext asyncPollContext) { - this(logContext, requestManagers, metadata, subscriptions, Optional.of(asyncPollContext)); + final CompletableEventReaper applicationEventReaper) { + this(logContext, requestManagers, metadata, subscriptions, Optional.of(applicationEventReaper)); } @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @@ -729,64 +732,99 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { } private void process(final AsyncPollEvent event) { - AsyncPollEventProcessorContext context = asyncPollContext.orElseThrow(IllegalArgumentException::new); - ApplicationEvent.Type nextEventType = event.startingEventType(); + log.trace("Processing poll logic for {}", event); - if (context.maybeCompleteWithCallbackRequired(event, nextEventType)) - return; + // Trigger a reconciliation that can safely commit offsets if needed to rebalance, + // as we're processing before any new fetching starts + requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> + consumerMembershipManager.maybeReconcile(true)); - if (nextEventType == ApplicationEvent.Type.ASYNC_POLL) { - log.trace("Processing {} logic for {}", nextEventType, event); - - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, - // as we're processing before any new fetching starts - requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> - consumerMembershipManager.maybeReconcile(true)); - - if (requestManagers.commitRequestManager.isPresent()) { - CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); - commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); - - requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); - }); - requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); - hrm.resetPollTimer(event.pollTimeMs()); - }); - } + if (requestManagers.commitRequestManager.isPresent()) { + CommitRequestManager commitRequestManager = requestManagers.commitRequestManager.get(); + commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); - nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS; + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { + hrm.membershipManager().onConsumerPoll(); + hrm.resetPollTimer(event.pollTimeMs()); + }); + } - if (context.maybeCompleteWithCallbackRequired(event, nextEventType)) + log.trace("Processing check and update positions logic for {}", event); + CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); + trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); + + updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) return; - } - if (nextEventType == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) { - log.trace("Processing {} logic for {}", nextEventType, event); - CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); - context.trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); + log.trace("Processing create fetch requests logic for {}", event); - updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (context.maybeCompleteExceptionally(event, updatePositionsError)) + // Create a fetch request if there's no data in the FetchBuffer. + requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { + if (maybeCompleteAsyncPollEventExceptionally(event, fetchError)) return; - log.trace("Processing {} logic for {}", ApplicationEvent.Type.CREATE_FETCH_REQUESTS, event); + event.completeSuccessfully(); + log.trace("Completed event processing for {}", event); + }); + }); + } - // Create a fetch request if there's no data in the FetchBuffer. - requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (context.maybeCompleteExceptionally(event, fetchError)) - return; + /** + * To maintain the flow from {@link ClassicKafkaConsumer}, the logic to check and update positions should be + * allowed to time out before moving on to the logic for sending fetch requests. This achieves that by reusing + * the {@link CompletableEventReaper} and allowing it to expire the {@link CompletableFuture} for the check and + * update positions stage. + */ + public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture updatePositionsFuture, long deadlineMs) { + applicationEventReaper.ifPresent(reaper -> { + CompletableEvent event = new CompletableEvent<>() { + @Override + public CompletableFuture future() { + return updatePositionsFuture; + } - context.complete(event); - }); - }); + @Override + public long deadlineMs() { + return deadlineMs; + } - return; + @Override + public String toString() { + return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + deadlineMs + '}'; + } + }; + + reaper.add(event); + }); + } + + /** + * If there's an error to report to the user, the current event will be completed with + * {@link AsyncPollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will + * return {@code false}. + */ + private boolean maybeCompleteAsyncPollEventExceptionally(AsyncPollEvent event, Throwable t) { + if (t == null) + return false; + + if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { + log.trace("Ignoring timeout for {}: {}", event, t.getMessage()); + return false; + } + + if (t instanceof CompletionException) { + t = t.getCause(); } - context.completeExceptionally(event, new KafkaException("Unknown next step for async poll: " + nextEventType)); + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + event.completeExceptionally(e); + log.trace("Failing event processing for {}", event, e); + return true; } private BiConsumer complete(final CompletableFuture b) { @@ -828,18 +866,17 @@ public static Supplier supplier(final LogContext logC final ConsumerMetadata metadata, final SubscriptionState subscriptions, final Supplier requestManagersSupplier, - final Supplier asyncPollContextSupplier) { + final CompletableEventReaper applicationEventReaper) { return new CachedSupplier<>() { @Override protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); - AsyncPollEventProcessorContext asyncPollContext = asyncPollContextSupplier.get(); return new ApplicationEventProcessor( logContext, requestManagers, metadata, subscriptions, - asyncPollContext + applicationEventReaper ); } }; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 2a5b4e7cf7102..145612d6fa011 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -17,16 +17,12 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerInterceptor; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; import java.time.Duration; -import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; @@ -34,8 +30,7 @@ * This class represents the non-blocking event that executes logic functionally equivalent to the following: * *
      - *
    • {@link SharePollEvent}
    • - *
    • {@link UpdatePatternSubscriptionEvent}
    • + *
    • Polling
    • *
    • {@link CheckAndUpdatePositionsEvent}
    • *
    • {@link CreateFetchRequestsEvent}
    • *
    @@ -48,16 +43,8 @@ *

    * * When the {@code AsyncPollEvent} is created, it exists in the {@link State#STARTED} state. The background - * thread will execute the {@code AsyncPollEvent} until it completes successfully ({@link State#SUCCEEDED}), - * hits an error ({@link State#FAILED}), or detects that the application thread needs to execute callbacks - * ({@link State#CALLBACKS_REQUIRED}). - * - *

    - * - * It's possible that the background processing of the polling will need to be "paused" in order to execute a - * {@link ConsumerInterceptor}, {@link ConsumerRebalanceListener}, and/or {@link OffsetCommitCallback} in the - * application thread. The background thread is able to detect when it needs to complete processing so that the - * application thread can execute the awaiting callbacks. + * thread will execute the {@code AsyncPollEvent} until it completes successfully ({@link State#SUCCEEDED}) + * or hits an error ({@link State#FAILED}). */ public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiable { @@ -65,79 +52,51 @@ public enum State { STARTED, SUCCEEDED, - FAILED, - CALLBACKS_REQUIRED + FAILED } public static class Result { - /** - * This string value is used when the {@code Result} represents a completed event. This is used so that - * {@code null} isn't used for {@link #value}. - */ - private static final Object COMPLETED_SENTINEL = "COMPLETED"; - /** * Used as the initial state/result until the terminal state is achieved. */ private static final Result STARTED = new Result(State.STARTED, null); private final State state; - private final Object value; + private final KafkaException error; - public Result(State state, Object value) { + public Result(State state, KafkaException error) { this.state = state; - this.value = value; + this.error = error; } public State state() { return state; } - public Type asNextEventType() { - if (state != State.CALLBACKS_REQUIRED) - throw new KafkaException("The usage of asNextEventType is unexpected for state: " + state); - - if (!(value instanceof ApplicationEvent.Type)) - throw new KafkaException("The result value for the poll was unexpected: " + value); - - return (ApplicationEvent.Type) value; - } - - public KafkaException asKafkaException() { - if (state != State.FAILED) - throw new KafkaException("The usage of asKafkaException is unexpected for state: " + state); - - if (!(value instanceof KafkaException)) - throw new KafkaException("The result value for the poll was unexpected: " + value); - - return (KafkaException) value; + public KafkaException error() { + return error; } @Override public String toString() { - return "Result{" + "state=" + state + ", value=" + value + '}'; + return "Result{" + "state=" + state + ", error=" + error + '}'; } @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Result result = (Result) o; - return state == result.state && Objects.equals(value, result.value); + return state == result.state && Objects.equals(error, result.error); } @Override public int hashCode() { - return Objects.hash(state, value); + return Objects.hash(state, error); } } - private static final List ALLOWED_STARTING_EVENT_TYPES = List.of( - Type.ASYNC_POLL, - Type.CHECK_AND_UPDATE_POSITIONS - ); private final long deadlineMs; private final long pollTimeMs; - private final Type startingEventType; private final AtomicReference result; /** @@ -148,26 +107,9 @@ public int hashCode() { * @param pollTimeMs Time, in milliseconds, at which point the event was created */ public AsyncPollEvent(long deadlineMs, long pollTimeMs) { - this(deadlineMs, pollTimeMs, Type.ASYNC_POLL); - } - - /** - * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. - * - * @param deadlineMs Time, in milliseconds, at which point the event must be completed; based on the - * {@link Duration} passed to {@link Consumer#poll(Duration)} - * @param pollTimeMs Time, in milliseconds, at which point the event was created - * @param startingEventType {@link ApplicationEvent.Type} that serves as the starting point for the event processing - */ - public AsyncPollEvent(long deadlineMs, long pollTimeMs, Type startingEventType) { super(Type.ASYNC_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; - - if (!ALLOWED_STARTING_EVENT_TYPES.contains(startingEventType)) - throw new KafkaException("The starting event type " + startingEventType + " is not valid. Should be one of " + ALLOWED_STARTING_EVENT_TYPES); - - this.startingEventType = startingEventType; this.result = new AtomicReference<>(Result.STARTED); } @@ -179,16 +121,12 @@ public long pollTimeMs() { return pollTimeMs; } - public Type startingEventType() { - return startingEventType; - } - public Result result() { return result.get(); } public void completeSuccessfully() { - Result r = new Result(State.SUCCEEDED, Result.COMPLETED_SENTINEL); + Result r = new Result(State.SUCCEEDED, null); result.compareAndSet(Result.STARTED, r); } @@ -197,11 +135,6 @@ public void completeExceptionally(KafkaException e) { result.compareAndSet(Result.STARTED, r); } - public void completeWithCallbackRequired(Type nextEventType) { - Result r = new Result(State.CALLBACKS_REQUIRED, Objects.requireNonNull(nextEventType)); - result.compareAndSet(Result.STARTED, r); - } - @Override public void metadataError(Exception metadataException) { completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(metadataException)); @@ -212,7 +145,6 @@ protected String toStringBase() { return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + - ", startingEventType=" + startingEventType + ", result=" + result.get(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java deleted file mode 100644 index e08bbd9064598..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEventProcessorContext.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * 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.CachedSupplier; -import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; -import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.clients.consumer.internals.FetchBuffer; -import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; -import org.apache.kafka.clients.consumer.internals.OffsetCommitCallbackInvoker; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.utils.LogContext; - -import org.slf4j.Logger; - -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.function.Supplier; - -/** - * This provides the context for the {@link ApplicationEventProcessor#process(ApplicationEvent)} that invokes the - * {@link AsyncPollEvent} process method. This is mostly to avoid polluting the {@link ApplicationEventProcessor} - * with instance variables and logic that's specific only to the background {@link AsyncPollEvent} processing. - */ -public class AsyncPollEventProcessorContext { - - private final Logger log; - private final NetworkClientDelegate networkClientDelegate; - private final BackgroundEventHandler backgroundEventHandler; - private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; - private final CompletableEventReaper applicationEventReaper; - private final FetchBuffer fetchBuffer; - - private AsyncPollEventProcessorContext(LogContext logContext, - NetworkClientDelegate networkClientDelegate, - BackgroundEventHandler backgroundEventHandler, - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper, - FetchBuffer fetchBuffer) { - this.log = logContext.logger(getClass()); - this.networkClientDelegate = networkClientDelegate; - this.backgroundEventHandler = backgroundEventHandler; - this.offsetCommitCallbackInvoker = offsetCommitCallbackInvoker; - this.applicationEventReaper = applicationEventReaper; - this.fetchBuffer = fetchBuffer; - } - - /** - * Creates a {@link Supplier} for deferred creation during invocation by - * {@link ConsumerNetworkThread}. - */ - public static Supplier supplier(LogContext logContext, - Supplier networkClientDelegateSupplier, - BackgroundEventHandler backgroundEventHandler, - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - CompletableEventReaper applicationEventReaper, - FetchBuffer fetchBuffer) { - return new CachedSupplier<>() { - @Override - protected AsyncPollEventProcessorContext create() { - NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - - return new AsyncPollEventProcessorContext( - logContext, - networkClientDelegate, - backgroundEventHandler, - offsetCommitCallbackInvoker, - applicationEventReaper, - fetchBuffer - ); - } - }; - } - - /** - * To maintain the flow from {@link ClassicKafkaConsumer}, the logic to check and update positions should be - * allowed to time out before moving on to the logic for sending fetch requests. This achieves that by reusing - * the {@link CompletableEventReaper} and allowing it to expire the {@link CompletableFuture} for the check and - * update positions stage. - */ - public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture updatePositionsFuture, long deadlineMs) { - CompletableEvent event = new CompletableEvent<>() { - @Override - public CompletableFuture future() { - return updatePositionsFuture; - } - - @Override - public long deadlineMs() { - return deadlineMs; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + deadlineMs + '}'; - } - }; - - applicationEventReaper.add(event); - } - - /** - * Helper method that will check if any application thread user callbacks need to be executed. If so, the - * current event will be completed with {@link AsyncPollEvent.State#CALLBACKS_REQUIRED} and this method - * will return {@code true}. Otherwise, it will return {@code false}. - */ - public boolean maybeCompleteWithCallbackRequired(AsyncPollEvent event, ApplicationEvent.Type nextEventType) { - // If there are background events to process or enqueued callbacks to invoke, exit to - // the application thread. - if (backgroundEventHandler.size() > 0 || offsetCommitCallbackInvoker.size() > 0) { - log.trace( - "Pausing polling by completing {} with the state of {} and the next stage of {}", - event, - AsyncPollEvent.State.CALLBACKS_REQUIRED, - nextEventType - ); - event.completeWithCallbackRequired(nextEventType); - - // This to ensure that the application thread doesn't needlessly wait for the full time out if it's - // been detected that a callback is required. - fetchBuffer.wakeup(); - return true; - } - - return false; - } - - /** - * Helper method that checks if there's a non-null error from - * {@link NetworkClientDelegate#getAndClearMetadataError()} or if the provided exception is not a timeout-based - * exception. If there's an error to report to the user, the current event will be completed with - * {@link AsyncPollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will - * return {@code false}. - */ - public boolean maybeCompleteExceptionally(AsyncPollEvent event, Throwable t) { - if (maybeCompleteExceptionally(event)) - return true; - - if (t == null) - return false; - - if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException) { - log.trace("Ignoring timeout for {}: {}", event, t.getMessage()); - return false; - } - - if (t instanceof CompletionException) { - t = t.getCause(); - } - - completeExceptionally(event, t); - return true; - } - - /** - * Helper method that checks if there's a non-null error from - * {@link NetworkClientDelegate#getAndClearMetadataError()}, and if so, reports it to the user by completing the - * current event with {@link AsyncPollEvent.State#FAILED} and returning {@code true}. Otherwise, it will - * return {@code false}. - */ - public boolean maybeCompleteExceptionally(AsyncPollEvent event) { - Optional exception = networkClientDelegate.getAndClearMetadataError(); - - if (exception.isPresent()) { - completeExceptionally(event, exception.get()); - return true; - } - - return false; - } - - /** - * Helper method to complete the given event with {@link AsyncPollEvent.State#FAILED}. - */ - public void completeExceptionally(AsyncPollEvent event, Throwable error) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(error); - event.completeExceptionally(e); - log.trace("Failing event processing for {}", event, e); - } - - /** - * Helper method to complete the given event with {@link AsyncPollEvent.State#SUCCEEDED}. - */ - public void complete(AsyncPollEvent event) { - event.completeSuccessfully(); - log.trace("Completed event processing for {}", event); - } -} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 47036f5d6eb19..de086fef71b87 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -2219,7 +2219,7 @@ private void completeSeekUnvalidatedEventSuccessfully() { private void completeAsyncPollEventSuccessfully() { doAnswer(invocation -> { AsyncPollEvent event = invocation.getArgument(0); - event.completeWithCallbackRequired(ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS); + event.completeSuccessfully(); return null; }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncPollEvent.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 654c4772d7c61..bcd42abe2ae90 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -111,8 +111,7 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), requestManagers, metadata, - subscriptionState, - Optional.of(mock(AsyncPollEventProcessorContext.class)) + subscriptionState ); } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index bfcc0bb0d4fca..7e17b15374638 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -1358,7 +1358,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val consumer = createConsumer() consumer.assign(java.util.List.of(tp)) - assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) + assertThrows(classOf[AuthorizationException], () => consumeRecords(consumer)) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) From f4eb1619fda12628b5c00975a4b0560d2bf9e44d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 5 Oct 2025 21:13:10 -0700 Subject: [PATCH 063/123] Inline trackCheckAndUpdatePositionsForTimeout in process(AsyncPollEvent) Moved the timeout tracking for check-and-update-positions logic directly into the main event processing flow, eliminating the separate trackCheckAndUpdatePositionsForTimeout method. This streamlines the handling of timeouts and improves code clarity by keeping related logic together. --- .../events/ApplicationEventProcessor.java | 53 ++++++++----------- 1 file changed, 23 insertions(+), 30 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 04cd7ec6abdd7..c76453a09922f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.Acknowledgements; import org.apache.kafka.clients.consumer.internals.CachedSupplier; -import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; @@ -755,34 +754,12 @@ private void process(final AsyncPollEvent event) { log.trace("Processing check and update positions logic for {}", event); CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); - trackCheckAndUpdatePositionsForTimeout(updatePositionsFuture, event.deadlineMs()); - updatePositionsFuture.whenComplete((__, updatePositionsError) -> { - if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) - return; - - log.trace("Processing create fetch requests logic for {}", event); - - // Create a fetch request if there's no data in the FetchBuffer. - requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { - if (maybeCompleteAsyncPollEventExceptionally(event, fetchError)) - return; - - event.completeSuccessfully(); - log.trace("Completed event processing for {}", event); - }); - }); - } - - /** - * To maintain the flow from {@link ClassicKafkaConsumer}, the logic to check and update positions should be - * allowed to time out before moving on to the logic for sending fetch requests. This achieves that by reusing - * the {@link CompletableEventReaper} and allowing it to expire the {@link CompletableFuture} for the check and - * update positions stage. - */ - public void trackCheckAndUpdatePositionsForTimeout(CompletableFuture updatePositionsFuture, long deadlineMs) { + // To maintain the flow from ClassicKafkaConsumer, the check-and-update-positions logic should be allowed + // to time out before moving on to the logic for sending fetch requests. This is achieved by using the event + // reaper and allowing it to expire the check-and-update-positions future. applicationEventReaper.ifPresent(reaper -> { - CompletableEvent event = new CompletableEvent<>() { + CompletableEvent pseudoEvent = new CompletableEvent<>() { @Override public CompletableFuture future() { return updatePositionsFuture; @@ -790,16 +767,32 @@ public CompletableFuture future() { @Override public long deadlineMs() { - return deadlineMs; + return event.deadlineMs(); } @Override public String toString() { - return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + deadlineMs + '}'; + return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + event.deadlineMs() + '}'; } }; - reaper.add(event); + reaper.add(pseudoEvent); + }); + + updatePositionsFuture.whenComplete((__, updatePositionsError) -> { + if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) + return; + + log.trace("Processing create fetch requests logic for {}", event); + + // Create a fetch request if there's no data in the FetchBuffer. + requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { + if (maybeCompleteAsyncPollEventExceptionally(event, fetchError)) + return; + + event.completeSuccessfully(); + log.trace("Completed event processing for {}", event); + }); }); } From ddf042306c92069407948c603808993001ef6569 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 5 Oct 2025 21:19:16 -0700 Subject: [PATCH 064/123] Revert wakeup() method visibility back to package-private --- .../apache/kafka/clients/consumer/internals/FetchBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java index 312e455b1aa81..6cf5bc301b370 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java @@ -193,7 +193,7 @@ void awaitWakeup(Timer timer) { } } - public void wakeup() { + void wakeup() { try { lock.lock(); wokenup.set(true); From b5f4c8683cc2c2b9d77335c3dc7778396a5d287f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 09:01:55 -0700 Subject: [PATCH 065/123] Minor refactoring, cleanup, extraneous whitespace removal --- .../internals/AsyncKafkaConsumer.java | 3 ++- .../internals/ConsumerNetworkThread.java | 25 +++++++++---------- .../events/AbstractTopicMetadataEvent.java | 4 +-- .../internals/events/AsyncPollEvent.java | 4 +-- .../events/CheckAndUpdatePositionsEvent.java | 4 +-- .../internals/events/ListOffsetsEvent.java | 4 +-- ...java => MetadataErrorNotifiableEvent.java} | 4 +-- .../internals/AsyncKafkaConsumerTest.java | 17 ++++++------- ...aslClientsWithInvalidCredentialsTest.scala | 2 +- 9 files changed, 33 insertions(+), 34 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{MetadataErrorNotifiable.java => MetadataErrorNotifiableEvent.java} (95%) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 2ab5f24d17839..6c89bb3ecd5f7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -884,7 +884,8 @@ public void checkInflightPollResult(Timer timer) { } try { - // Note: this is calling user-supplied code, so make sure to handle possible errors. + // Note: this is calling user-supplied code, so make sure that any errors thrown here are caught and + // the inflight event is cleared. offsetCommitCallbackInvoker.executeCallbacks(); processBackgroundEvents(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 1dbe15db9d9a7..0f1c24829d978 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -22,7 +22,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; -import org.apache.kafka.clients.consumer.internals.events.MetadataErrorNotifiable; +import org.apache.kafka.clients.consumer.internals.events.MetadataErrorNotifiableEvent; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; @@ -192,17 +192,16 @@ private void processApplicationEvents() { for (ApplicationEvent event : events) { asyncConsumerMetrics.recordApplicationEventQueueTime(time.milliseconds() - event.enqueuedMs()); try { - if (event instanceof CompletableEvent) + if (event instanceof CompletableEvent) { applicationEventReaper.add((CompletableEvent) event); - + } // Check if there are any metadata errors and fail the CompletableEvent if an error is present. // This call is meant to handle "immediately completed events" which may not enter the awaiting state, // so metadata errors need to be checked and handled right away. - if (event instanceof MetadataErrorNotifiable) { - if (maybeFailOnMetadataError((MetadataErrorNotifiable) event)) + if (event instanceof MetadataErrorNotifiableEvent) { + if (maybeFailOnMetadataError((MetadataErrorNotifiableEvent) event)) continue; } - applicationEventProcessor.process(event); } catch (Throwable t) { log.warn("Error processing event {}", t.getMessage(), t); @@ -374,11 +373,11 @@ void cleanup() { * If there is a metadata error, complete all uncompleted events that require subscription metadata. */ private void maybeFailOnMetadataError(List> events) { - List notifiables = new ArrayList<>(); + List notifiables = new ArrayList<>(); for (CompletableEvent ce : events) { - if (ce instanceof MetadataErrorNotifiable) { - notifiables.add((MetadataErrorNotifiable) ce); + if (ce instanceof MetadataErrorNotifiableEvent) { + notifiables.add((MetadataErrorNotifiableEvent) ce); } } @@ -391,20 +390,20 @@ private void maybeFailOnMetadataError(List> events) { return; Exception metadataError = metadataErrorOpt.get(); - notifiables.forEach(n -> n.metadataError(metadataError)); + notifiables.forEach(n -> n.completeExceptionallyWithMetadataError(metadataError)); } /** - * If there is a metadata error, complete all uncompleted events that require subscription metadata. + * If there is a metadata error, complete this event exceptionally. */ - private boolean maybeFailOnMetadataError(MetadataErrorNotifiable notifiable) { + private boolean maybeFailOnMetadataError(MetadataErrorNotifiableEvent notifiable) { Optional metadataErrorOpt = networkClientDelegate.getAndClearMetadataError(); if (metadataErrorOpt.isEmpty()) return false; Exception metadataError = metadataErrorOpt.get(); - notifiable.metadataError(metadataError); + notifiable.completeExceptionallyWithMetadataError(metadataError); return true; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index 5c605d7efa2e7..a23c5828c194f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -21,14 +21,14 @@ import java.util.List; import java.util.Map; -public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> implements MetadataErrorNotifiable { +public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> implements MetadataErrorNotifiableEvent { protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) { super(type, deadlineMs); } @Override - public void metadataError(Exception metadataException) { + public void completeExceptionallyWithMetadataError(Exception metadataException) { future().completeExceptionally(metadataException); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 145612d6fa011..e2a8143ac7f68 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -46,7 +46,7 @@ * thread will execute the {@code AsyncPollEvent} until it completes successfully ({@link State#SUCCEEDED}) * or hits an error ({@link State#FAILED}). */ -public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiable { +public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiableEvent { public enum State { @@ -136,7 +136,7 @@ public void completeExceptionally(KafkaException e) { } @Override - public void metadataError(Exception metadataException) { + public void completeExceptionallyWithMetadataError(Exception metadataException) { completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(metadataException)); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java index 9e0a60beadc92..c9cfa0219c01d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java @@ -31,7 +31,7 @@ * The event completes with a boolean indicating if all assigned partitions have valid fetch positions * (based on {@link SubscriptionState#hasAllFetchPositions()}). */ -public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent implements MetadataErrorNotifiable { +public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent implements MetadataErrorNotifiableEvent { public CheckAndUpdatePositionsEvent(long deadlineMs) { super(Type.CHECK_AND_UPDATE_POSITIONS, deadlineMs); @@ -44,7 +44,7 @@ public CheckAndUpdatePositionsEvent(long deadlineMs) { * or {@link Consumer#position(TopicPartition) position} process. */ @Override - public void metadataError(Exception metadataException) { + public void completeExceptionallyWithMetadataError(Exception metadataException) { future().completeExceptionally(metadataException); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 1bcd73d07596b..6789571267300 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -32,7 +32,7 @@ * {@link OffsetAndTimestamp} found (offset of the first message whose timestamp is greater than * or equals to the target timestamp) */ -public class ListOffsetsEvent extends CompletableApplicationEvent> implements MetadataErrorNotifiable { +public class ListOffsetsEvent extends CompletableApplicationEvent> implements MetadataErrorNotifiableEvent { private final Map timestampsToSearch; private final boolean requireTimestamps; @@ -65,7 +65,7 @@ public boolean requireTimestamps() { } @Override - public void metadataError(Exception metadataException) { + public void completeExceptionallyWithMetadataError(Exception metadataException) { future().completeExceptionally(metadataException); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java index 169f0b0b57648..7fabf276be476 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiable.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java @@ -24,7 +24,7 @@ * This interface is used for events that need to be notified when the * {@link NetworkClientDelegate#getAndClearMetadataError()} has an error. */ -public interface MetadataErrorNotifiable { +public interface MetadataErrorNotifiableEvent { /** * The background thread detects metadata errors on every call to {@link NetworkClientDelegate#poll(long, long)}. @@ -52,5 +52,5 @@ public interface MetadataErrorNotifiable { * * @param metadataException Error that originally came from {@link Metadata#maybeThrowAnyException()} */ - void metadataError(Exception metadataException); + void completeExceptionallyWithMetadataError(Exception metadataException); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index de086fef71b87..1cfe218519101 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1483,7 +1483,6 @@ public void testListenerCallbacksInvoke(List { - AsyncPollEvent event = invocation.getArgument(0); - event.completeSuccessfully(); - return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncPollEvent.class)); - } - private void forceCommitCallbackInvocation() { // Invokes callback consumer.commitAsync(); @@ -2301,4 +2292,12 @@ public void testCloseWrapsStreamsRebalanceListenerException() { verify(mockStreamsListener).onTasksRevoked(any()); } } + + private void completeAsyncPollEventSuccessfully() { + doAnswer(invocation -> { + AsyncPollEvent event = invocation.getArgument(0); + event.completeSuccessfully(); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncPollEvent.class)); + } } diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index 7c9c8e577eee5..4d2c5ab4f0cc6 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -15,6 +15,7 @@ package kafka.api import kafka.security.JaasTestUtils import kafka.utils.TestUtils.waitUntilTrue +import java.time.Duration import java.util.Properties import java.util.concurrent.{ExecutionException, TimeUnit} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} @@ -33,7 +34,6 @@ import org.apache.kafka.metadata.storage.Formatter import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{MethodSource, ValueSource} -import java.time.Duration import scala.jdk.javaapi.OptionConverters import scala.util.Using From cac80ef38f95b8b3f7da8bc3ff5be60d45fd325f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 09:33:49 -0700 Subject: [PATCH 066/123] Reverted minor import change with SaslClientsWithInvalidCredentialsTest --- .../kafka/api/SaslClientsWithInvalidCredentialsTest.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index 4d2c5ab4f0cc6..382f548ed5231 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -13,7 +13,6 @@ package kafka.api import kafka.security.JaasTestUtils -import kafka.utils.TestUtils.waitUntilTrue import java.time.Duration import java.util.Properties @@ -160,7 +159,7 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { createClientCredential() val producer = createProducer() verifyWithRetry(sendOneRecord(producer))() - waitUntilTrue(() => { + TestUtils.waitUntilTrue(() => { try { consumer.poll(Duration.ofMillis(1000)).count() == 1 } catch { From fa784e5f253f611094819ed5f299a5c9fc7b1049 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 10:16:09 -0700 Subject: [PATCH 067/123] Refactored AsyncPollEvent to remove State --- .../internals/AsyncKafkaConsumer.java | 51 +++++++++---------- .../internals/events/AsyncPollEvent.java | 50 +++++------------- 2 files changed, 39 insertions(+), 62 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 6c89bb3ecd5f7..f75d64141237b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -874,13 +874,23 @@ public ConsumerRecords poll(final Duration timeout) { /** * {@code checkInflightPollResult()} manages the lifetime of the {@link AsyncPollEvent} processing. If it is * called when no event is currently processing, it will start a new event processing asynchronously. A check - * is made during each invocation to see if the inflight event has reached a - * {@link AsyncPollEvent.State terminal state}. If it has, the result will be processed accordingly. + * is made during each invocation to see if the inflight event has completed. If it has, the result + * will be processed accordingly. */ public void checkInflightPollResult(Timer timer) { if (inflightPoll == null) { - log.trace("No existing inflight async poll event, submitting a new event"); - submitEvent(timer); + long deadlineMs = calculateDeadlineMs(timer); + long pollTimeMs = time.milliseconds(); + inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs); + applicationEventHandler.add(inflightPoll); + + if (log.isTraceEnabled()) { + log.trace( + "No existing inflight async poll event, submitted new {} with {} remaining on timer", + inflightPoll, + timer.remainingMs() + ); + } } try { @@ -898,21 +908,20 @@ public void checkInflightPollResult(Timer timer) { } // Result should be non-null and starts off as State.STARTED. - AsyncPollEvent.Result result = inflightPoll.result(); - AsyncPollEvent.State state = result.state(); + Optional resultOpt = inflightPoll.result(); - if (state == AsyncPollEvent.State.SUCCEEDED) { - // The async poll event has completed all the requisite stages, though it does not imply that - // there is data in the FetchBuffer yet. Make sure to clear out the inflight request. - log.trace("Event {} completed, clearing inflight", inflightPoll); - inflightPoll = null; - } else if (state == AsyncPollEvent.State.FAILED) { - // The async poll failed at one of the stages. Make sure to clear out the inflight request - // before the underlying error is surfaced to the user. - log.trace("Event {} failed, clearing inflight", inflightPoll); + if (resultOpt.isPresent()) { + AsyncPollEvent.Result result = resultOpt.get(); + + // The async poll event has completed, either successfully or not. In either case, clear out the + // inflight request. + log.trace("Event {} completed with result {}, clearing inflight", inflightPoll, result); inflightPoll = null; + Optional errorOpt = result.error(); - throw result.error(); + if (errorOpt.isPresent()) { + throw errorOpt.get(); + } } } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request @@ -923,16 +932,6 @@ public void checkInflightPollResult(Timer timer) { } } - private void submitEvent(Timer timer) { - long deadlineMs = calculateDeadlineMs(timer); - long pollTimeMs = time.milliseconds(); - inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs); - applicationEventHandler.add(inflightPoll); - - if (log.isTraceEnabled()) - log.trace("Submitted new {} with {} remaining on timer", inflightPoll, timer.remainingMs()); - } - /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index e2a8143ac7f68..49c7bc31fef4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -24,6 +24,7 @@ import java.time.Duration; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; /** @@ -39,59 +40,36 @@ * at the same level as {@link ClassicKafkaConsumer#poll(Duration)}. The event is submitted in {@code poll()}, but * there are no blocking waits for the "result" of the event. Checks are made for the result at certain points, but * they do not block. The logic for the previously-mentioned events is executed sequentially on the background thread. - * - *

    - * - * When the {@code AsyncPollEvent} is created, it exists in the {@link State#STARTED} state. The background - * thread will execute the {@code AsyncPollEvent} until it completes successfully ({@link State#SUCCEEDED}) - * or hits an error ({@link State#FAILED}). */ public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiableEvent { - public enum State { - - STARTED, - SUCCEEDED, - FAILED - } - public static class Result { - /** - * Used as the initial state/result until the terminal state is achieved. - */ - private static final Result STARTED = new Result(State.STARTED, null); - private final State state; - private final KafkaException error; + private final Optional error; - public Result(State state, KafkaException error) { - this.state = state; + public Result(Optional error) { this.error = error; } - public State state() { - return state; - } - - public KafkaException error() { + public Optional error() { return error; } @Override public String toString() { - return "Result{" + "state=" + state + ", error=" + error + '}'; + return "Result{error=" + error + '}'; } @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Result result = (Result) o; - return state == result.state && Objects.equals(error, result.error); + return Objects.equals(error, result.error); } @Override public int hashCode() { - return Objects.hash(state, error); + return error.hashCode(); } } @@ -110,7 +88,7 @@ public AsyncPollEvent(long deadlineMs, long pollTimeMs) { super(Type.ASYNC_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; - this.result = new AtomicReference<>(Result.STARTED); + this.result = new AtomicReference<>(); } public long deadlineMs() { @@ -121,18 +99,18 @@ public long pollTimeMs() { return pollTimeMs; } - public Result result() { - return result.get(); + public Optional result() { + return Optional.ofNullable(result.get()); } public void completeSuccessfully() { - Result r = new Result(State.SUCCEEDED, null); - result.compareAndSet(Result.STARTED, r); + Result r = new Result(Optional.empty()); + result.compareAndSet(null, r); } public void completeExceptionally(KafkaException e) { - Result r = new Result(State.FAILED, Objects.requireNonNull(e)); - result.compareAndSet(Result.STARTED, r); + Result r = new Result(Optional.of(e)); + result.compareAndSet(null, r); } @Override From 5be72105ba0a80288e41c9f7cf73e6df2555b04a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 10:21:06 -0700 Subject: [PATCH 068/123] Update ApplicationEventProcessor.java --- .../consumer/internals/events/ApplicationEventProcessor.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index c76453a09922f..35ef048289609 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -797,9 +797,8 @@ public String toString() { } /** - * If there's an error to report to the user, the current event will be completed with - * {@link AsyncPollEvent.State#FAILED} and this method will return {@code true}. Otherwise, it will - * return {@code false}. + * If there's an error to report to the user, the current event will be completed and this method will + * return {@code true}. Otherwise, it will return {@code false}. */ private boolean maybeCompleteAsyncPollEventExceptionally(AsyncPollEvent event, Throwable t) { if (t == null) From 19ce01afa30d021ed811bd13ae5dd39a369e0a2c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 14:41:03 -0700 Subject: [PATCH 069/123] Removed check-and-update-positions future timeout and removed AsyncPollEvent.Result inner class --- .../internals/AsyncKafkaConsumer.java | 68 +++++++++--------- .../internals/NetworkClientDelegate.java | 22 ++++++ .../events/ApplicationEventProcessor.java | 69 +------------------ .../internals/events/AsyncPollEvent.java | 52 ++++---------- .../clients/consumer/KafkaConsumerTest.java | 17 ++--- 5 files changed, 80 insertions(+), 148 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index f75d64141237b..5327cec66aecc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -460,18 +460,16 @@ public AsyncKafkaConsumer(final ConsumerConfig config, memberStateListener, streamsRebalanceData ); - final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, subscriptions, - requestManagersSupplier, - applicationEventReaper + requestManagersSupplier ); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, time, applicationEventQueue, - applicationEventReaper, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -654,18 +652,16 @@ public AsyncKafkaConsumer(final ConsumerConfig config, memberStateListener, Optional.empty() ); - final CompletableEventReaper applicationEventReaper = new CompletableEventReaper(logContext); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, subscriptions, - requestManagersSupplier, - applicationEventReaper + requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, - applicationEventReaper, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, @@ -843,7 +839,7 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - checkInflightPollResult(timer); + checkInflightPoll(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -872,25 +868,29 @@ public ConsumerRecords poll(final Duration timeout) { } /** - * {@code checkInflightPollResult()} manages the lifetime of the {@link AsyncPollEvent} processing. If it is + * {@code checkInflightPoll()} manages the lifetime of the {@link AsyncPollEvent} processing. If it is * called when no event is currently processing, it will start a new event processing asynchronously. A check - * is made during each invocation to see if the inflight event has completed. If it has, the result - * will be processed accordingly. + * is made during each invocation to see if the inflight event has completed. If it has, it will be + * processed accordingly. */ - public void checkInflightPollResult(Timer timer) { + public void checkInflightPoll(Timer timer) { + boolean newlySubmittedEvent = false; + if (inflightPoll == null) { - long deadlineMs = calculateDeadlineMs(timer); long pollTimeMs = time.milliseconds(); + long deadlineMs = calculateDeadlineMs(time, pollTimeMs); inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs); - applicationEventHandler.add(inflightPoll); + newlySubmittedEvent = true; if (log.isTraceEnabled()) { log.trace( - "No existing inflight async poll event, submitted new {} with {} remaining on timer", + "Submitting new inflight event {} with {} remaining on timer", inflightPoll, timer.remainingMs() ); } + + applicationEventHandler.add(inflightPoll); } try { @@ -899,34 +899,36 @@ public void checkInflightPollResult(Timer timer) { offsetCommitCallbackInvoker.executeCallbacks(); processBackgroundEvents(); - if (log.isTraceEnabled()) { - log.trace( - "Attempting to retrieve result from previously submitted {} with {} remaining on timer", - inflightPoll, - timer.remainingMs() - ); - } - - // Result should be non-null and starts off as State.STARTED. - Optional resultOpt = inflightPoll.result(); - - if (resultOpt.isPresent()) { - AsyncPollEvent.Result result = resultOpt.get(); + if (inflightPoll.isComplete()) { + Optional errorOpt = inflightPoll.error(); // The async poll event has completed, either successfully or not. In either case, clear out the // inflight request. - log.trace("Event {} completed with result {}, clearing inflight", inflightPoll, result); + log.trace("Inflight event {} completed, clearing", inflightPoll); inflightPoll = null; - Optional errorOpt = result.error(); if (errorOpt.isPresent()) { throw errorOpt.get(); } + } else if (!newlySubmittedEvent) { + if (timer.isExpired()) { + // The inflight event is expired... + log.trace("Inflight event {} expired without completing, clearing", inflightPoll); + inflightPoll = null; + } else { + if (log.isTraceEnabled()) { + log.trace( + "Inflight event {} is incomplete with {} remaining on timer", + inflightPoll, + timer.remainingMs() + ); + } + } } } catch (Throwable t) { // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request // because the error effectively renders it complete. - log.debug("Event {} failed due to {}, clearing inflight", inflightPoll, String.valueOf(t)); + log.debug("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); inflightPoll = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); } @@ -1827,7 +1829,7 @@ private Fetch pollForFetches(Timer timer) { return fetch; } - log.trace("Polling for fetches with timeout {}", pollTimeout); + log.trace("Polling for fetches with timeout {} and {}", pollTimeout, timer.remainingMs()); Timer pollTimer = time.timer(pollTimeout); wakeupTrigger.setFetchAction(fetchBuffer); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 31c402df2a6db..772ca911d11b7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -210,6 +210,28 @@ boolean doSend(final UnsentRequest r, final long currentTimeMs) { } ClientRequest request = makeClientRequest(r, node, currentTimeMs); if (!client.ready(node, currentTimeMs)) { + AuthenticationException authenticationException = client.authenticationException(node); + + if (authenticationException != null) { + request.callback().onComplete( + new ClientResponse( + request.makeHeader( + request.requestBuilder().latestAllowedVersion() + ), + request.callback(), + request.destination(), + request.createdTimeMs(), + currentTimeMs, + true, + null, + authenticationException, + null + ) + ); + + return false; + } + // enqueue the request again if the node isn't ready yet. The request will be handled in the next iteration // of the event loop log.debug("Node is not ready, handle the request in the next event loop: node={}, request={}", node, r); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 35ef048289609..b940fb7458687 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -43,7 +43,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -63,37 +62,19 @@ public class ApplicationEventProcessor implements EventProcessor applicationEventReaper; private int metadataVersionSnapshot; public ApplicationEventProcessor(final LogContext logContext, final RequestManagers requestManagers, final ConsumerMetadata metadata, - final SubscriptionState subscriptions, - final Optional applicationEventReaper) { + final SubscriptionState subscriptions) { this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; this.subscriptions = subscriptions; - this.applicationEventReaper = applicationEventReaper; this.metadataVersionSnapshot = metadata.updateVersion(); } - public ApplicationEventProcessor(final LogContext logContext, - final RequestManagers requestManagers, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions) { - this(logContext, requestManagers, metadata, subscriptions, Optional.empty()); - } - - public ApplicationEventProcessor(final LogContext logContext, - final RequestManagers requestManagers, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions, - final CompletableEventReaper applicationEventReaper) { - this(logContext, requestManagers, metadata, subscriptions, Optional.of(applicationEventReaper)); - } - @SuppressWarnings({"CyclomaticComplexity", "JavaNCSSCheck"}) @Override public void process(ApplicationEvent event) { @@ -755,30 +736,6 @@ private void process(final AsyncPollEvent event) { log.trace("Processing check and update positions logic for {}", event); CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); - // To maintain the flow from ClassicKafkaConsumer, the check-and-update-positions logic should be allowed - // to time out before moving on to the logic for sending fetch requests. This is achieved by using the event - // reaper and allowing it to expire the check-and-update-positions future. - applicationEventReaper.ifPresent(reaper -> { - CompletableEvent pseudoEvent = new CompletableEvent<>() { - @Override - public CompletableFuture future() { - return updatePositionsFuture; - } - - @Override - public long deadlineMs() { - return event.deadlineMs(); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{updatePositionsFuture=" + updatePositionsFuture + ", deadlineMs=" + event.deadlineMs() + '}'; - } - }; - - reaper.add(pseudoEvent); - }); - updatePositionsFuture.whenComplete((__, updatePositionsError) -> { if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) return; @@ -850,30 +807,6 @@ protected ApplicationEventProcessor create() { }; } - /** - * Creates a {@link Supplier} for deferred creation during invocation by - * {@link ConsumerNetworkThread}. - */ - public static Supplier supplier(final LogContext logContext, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions, - final Supplier requestManagersSupplier, - final CompletableEventReaper applicationEventReaper) { - return new CachedSupplier<>() { - @Override - protected ApplicationEventProcessor create() { - RequestManagers requestManagers = requestManagersSupplier.get(); - return new ApplicationEventProcessor( - logContext, - requestManagers, - metadata, - subscriptions, - applicationEventReaper - ); - } - }; - } - /** * This function evaluates the regex that the consumer subscribed to * against the list of topic names from metadata, and updates diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 49c7bc31fef4b..93869f2e01c27 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -43,39 +43,10 @@ */ public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNotifiableEvent { - public static class Result { - - private final Optional error; - - public Result(Optional error) { - this.error = error; - } - - public Optional error() { - return error; - } - - @Override - public String toString() { - return "Result{error=" + error + '}'; - } - - @Override - public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) return false; - Result result = (Result) o; - return Objects.equals(error, result.error); - } - - @Override - public int hashCode() { - return error.hashCode(); - } - } - private final long deadlineMs; private final long pollTimeMs; - private final AtomicReference result; + private volatile KafkaException error; + private volatile boolean isComplete; /** * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. @@ -88,7 +59,6 @@ public AsyncPollEvent(long deadlineMs, long pollTimeMs) { super(Type.ASYNC_POLL); this.deadlineMs = deadlineMs; this.pollTimeMs = pollTimeMs; - this.result = new AtomicReference<>(); } public long deadlineMs() { @@ -99,18 +69,21 @@ public long pollTimeMs() { return pollTimeMs; } - public Optional result() { - return Optional.ofNullable(result.get()); + public Optional error() { + return Optional.ofNullable(error); + } + + public boolean isComplete() { + return isComplete; } public void completeSuccessfully() { - Result r = new Result(Optional.empty()); - result.compareAndSet(null, r); + isComplete = true; } public void completeExceptionally(KafkaException e) { - Result r = new Result(Optional.of(e)); - result.compareAndSet(null, r); + isComplete = true; + this.error = e; } @Override @@ -123,6 +96,7 @@ protected String toStringBase() { return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + - ", result=" + result.get(); + ", error=" + error + + ", isComplete=" + isComplete; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index d4281f6b29a84..40db20e7be045 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2680,15 +2680,11 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept } // poll once again, which should send the list-offset request consumer.seek(tp0, 50L); - // requests: list-offset, fetch + // requests: list-offset ConsumerPollTestUtils.waitForCondition( consumer, - () -> { - boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); - boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); - return hasListOffsetRequest && hasFetchRequest; - }, - "No list-offset & fetch request sent" + () -> requestGenerated(client, ApiKeys.LIST_OFFSETS), + "No list-offset sent" ); // no error for no end offset (so unknown lag) @@ -2698,7 +2694,12 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // and hence next call would return correct lag result ClientRequest listOffsetRequest = findRequest(client, ApiKeys.LIST_OFFSETS); client.respondToRequest(listOffsetRequest, listOffsetsResponse(Map.of(tp0, 90L))); - consumer.poll(Duration.ofMillis(0)); + // requests: fetch + ConsumerPollTestUtils.waitForCondition( + consumer, + () -> requestGenerated(client, ApiKeys.FETCH), + "No fetch sent" + ); // For AsyncKafkaConsumer, subscription state is updated in background, so the result will eventually be updated. TestUtils.waitForCondition(() -> { From b04b0a2c739a4ca2cbf15743a005c6ce11b1861d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 14:43:47 -0700 Subject: [PATCH 070/123] Removed superfluous imports --- .../kafka/clients/consumer/internals/events/AsyncPollEvent.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 93869f2e01c27..07b8fffcd1362 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -23,9 +23,7 @@ import org.apache.kafka.common.KafkaException; import java.time.Duration; -import java.util.Objects; import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; /** * This class represents the non-blocking event that executes logic functionally equivalent to the following: From d93967564105b51e85665b3945f1eb663d914c61 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 16:49:22 -0700 Subject: [PATCH 071/123] Added comment to NetworkClientDelegate.doSend() change --- .../kafka/clients/consumer/internals/NetworkClientDelegate.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 772ca911d11b7..ca0492e39b569 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -212,6 +212,8 @@ boolean doSend(final UnsentRequest r, final long currentTimeMs) { if (!client.ready(node, currentTimeMs)) { AuthenticationException authenticationException = client.authenticationException(node); + // The client may not be ready because it hit an unrecoverable authentication error. In that case, there's + // no benefit from retrying, so propagate the error here. if (authenticationException != null) { request.callback().onComplete( new ClientResponse( From 91409bc9d35a27828bdeb0f2c6c20d3aadfa72e6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 16:50:31 -0700 Subject: [PATCH 072/123] Make ConsumerIntegrationTest.testRackAwareAssignment() poll more frequently to reconcile assignment changes more quickly --- .../consumer/ConsumerIntegrationTest.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java index f8621bc9133a9..4112dc58acfbf 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -266,10 +266,11 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec consumer1.subscribe(List.of(topic)); consumer2.subscribe(List.of(topic)); + Duration pollTimeout = Duration.ofMillis(100); TestUtils.waitForCondition(() -> { - consumer0.poll(Duration.ofMillis(1000)); - consumer1.poll(Duration.ofMillis(1000)); - consumer2.poll(Duration.ofMillis(1000)); + consumer0.poll(pollTimeout); + consumer1.poll(pollTimeout); + consumer2.poll(pollTimeout); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().isEmpty() && consumer2.assignment().isEmpty(); @@ -284,9 +285,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec ); clusterInstance.waitTopicCreation(topic, 3); TestUtils.waitForCondition(() -> { - consumer0.poll(Duration.ofMillis(1000)); - consumer1.poll(Duration.ofMillis(1000)); - consumer2.poll(Duration.ofMillis(1000)); + consumer0.poll(pollTimeout); + consumer1.poll(pollTimeout); + consumer2.poll(pollTimeout); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) && consumer2.assignment().isEmpty(); @@ -301,9 +302,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec ); clusterInstance.waitTopicCreation(topic, 6); TestUtils.waitForCondition(() -> { - consumer0.poll(Duration.ofMillis(1000)); - consumer1.poll(Duration.ofMillis(1000)); - consumer2.poll(Duration.ofMillis(1000)); + consumer0.poll(pollTimeout); + consumer1.poll(pollTimeout); + consumer2.poll(pollTimeout); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) && consumer2.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4), new TopicPartition(topic, 5))); @@ -325,9 +326,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec new TopicPartition(topic, 5), Optional.of(new NewPartitionReassignment(List.of(0))) )).all().get(); TestUtils.waitForCondition(() -> { - consumer0.poll(Duration.ofMillis(1000)); - consumer1.poll(Duration.ofMillis(1000)); - consumer2.poll(Duration.ofMillis(1000)); + consumer0.poll(pollTimeout); + consumer1.poll(pollTimeout); + consumer2.poll(pollTimeout); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 5))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4))) && consumer2.assignment().equals(Set.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(topic, 2))); From e7b53865fdf2a94c14d604a1ec49d959c77834b0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Oct 2025 18:24:52 -0700 Subject: [PATCH 073/123] Fix stupid typo for poll timeout --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 5327cec66aecc..cfe726e699334 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -877,9 +877,7 @@ public void checkInflightPoll(Timer timer) { boolean newlySubmittedEvent = false; if (inflightPoll == null) { - long pollTimeMs = time.milliseconds(); - long deadlineMs = calculateDeadlineMs(time, pollTimeMs); - inflightPoll = new AsyncPollEvent(deadlineMs, pollTimeMs); + inflightPoll = new AsyncPollEvent(calculateDeadlineMs(timer), time.milliseconds()); newlySubmittedEvent = true; if (log.isTraceEnabled()) { From 4985c7de178b70c4df709ff2a6d9a2a61fb271b1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 08:52:21 -0700 Subject: [PATCH 074/123] Updates for MetadataErrorNotifiableEvent --- .../internals/ConsumerNetworkThread.java | 43 +++++++------------ .../events/AbstractTopicMetadataEvent.java | 4 +- .../internals/events/AsyncPollEvent.java | 4 +- .../events/CheckAndUpdatePositionsEvent.java | 4 +- .../internals/events/ListOffsetsEvent.java | 4 +- .../events/MetadataErrorNotifiableEvent.java | 4 +- 6 files changed, 26 insertions(+), 37 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 0f1c24829d978..9666041fc64a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -199,7 +199,7 @@ private void processApplicationEvents() { // This call is meant to handle "immediately completed events" which may not enter the awaiting state, // so metadata errors need to be checked and handled right away. if (event instanceof MetadataErrorNotifiableEvent) { - if (maybeFailOnMetadataError((MetadataErrorNotifiableEvent) event)) + if (maybeFailOnMetadataError(List.of(event))) continue; } applicationEventProcessor.process(event); @@ -372,38 +372,27 @@ void cleanup() { /** * If there is a metadata error, complete all uncompleted events that require subscription metadata. */ - private void maybeFailOnMetadataError(List> events) { - List notifiables = new ArrayList<>(); + private boolean maybeFailOnMetadataError(List events) { + List filteredEvents = new ArrayList<>(); - for (CompletableEvent ce : events) { - if (ce instanceof MetadataErrorNotifiableEvent) { - notifiables.add((MetadataErrorNotifiableEvent) ce); + for (Object obj : events) { + if (obj instanceof MetadataErrorNotifiableEvent) { + filteredEvents.add((MetadataErrorNotifiableEvent) obj); } } - if (notifiables.isEmpty()) - return; - - Optional metadataErrorOpt = networkClientDelegate.getAndClearMetadataError(); - - if (metadataErrorOpt.isEmpty()) - return; - - Exception metadataError = metadataErrorOpt.get(); - notifiables.forEach(n -> n.completeExceptionallyWithMetadataError(metadataError)); - } + // Don't get-and-clear the metadata error if there are no events that will be notified. + if (filteredEvents.isEmpty()) + return false; - /** - * If there is a metadata error, complete this event exceptionally. - */ - private boolean maybeFailOnMetadataError(MetadataErrorNotifiableEvent notifiable) { - Optional metadataErrorOpt = networkClientDelegate.getAndClearMetadataError(); + Optional andClearMetadataError = networkClientDelegate.getAndClearMetadataError(); - if (metadataErrorOpt.isEmpty()) + if (andClearMetadataError.isPresent()) { + Exception metadataError = andClearMetadataError.get(); + filteredEvents.forEach(e -> e.onMetadataError(metadataError)); + return true; + } else { return false; - - Exception metadataError = metadataErrorOpt.get(); - notifiable.completeExceptionallyWithMetadataError(metadataError); - return true; + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index a23c5828c194f..2db2b16b1d268 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -28,7 +28,7 @@ protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) { } @Override - public void completeExceptionallyWithMetadataError(Exception metadataException) { - future().completeExceptionally(metadataException); + public void onMetadataError(Exception metadataError) { + future().completeExceptionally(metadataError); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 07b8fffcd1362..e63f85ecc33f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -85,8 +85,8 @@ public void completeExceptionally(KafkaException e) { } @Override - public void completeExceptionallyWithMetadataError(Exception metadataException) { - completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(metadataException)); + public void onMetadataError(Exception metadataError) { + completeExceptionally(ConsumerUtils.maybeWrapAsKafkaException(metadataError)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java index c9cfa0219c01d..4fd834eaf0980 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java @@ -44,7 +44,7 @@ public CheckAndUpdatePositionsEvent(long deadlineMs) { * or {@link Consumer#position(TopicPartition) position} process. */ @Override - public void completeExceptionallyWithMetadataError(Exception metadataException) { - future().completeExceptionally(metadataException); + public void onMetadataError(Exception metadataError) { + future().completeExceptionally(metadataError); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 6789571267300..bce78e4aa2075 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -65,8 +65,8 @@ public boolean requireTimestamps() { } @Override - public void completeExceptionallyWithMetadataError(Exception metadataException) { - future().completeExceptionally(metadataException); + public void onMetadataError(Exception metadataError) { + future().completeExceptionally(metadataError); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java index 7fabf276be476..be8ec46796087 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataErrorNotifiableEvent.java @@ -50,7 +50,7 @@ public interface MetadataErrorNotifiableEvent { * *

* - * @param metadataException Error that originally came from {@link Metadata#maybeThrowAnyException()} + * @param metadataError Error that originally came from {@link Metadata#maybeThrowAnyException()} */ - void completeExceptionallyWithMetadataError(Exception metadataException); + void onMetadataError(Exception metadataError); } From 2e5a982c1d77c0e58164d8c6e5c1a8563c8b65b6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 08:52:34 -0700 Subject: [PATCH 075/123] Revert change to AsyncKafkaConsumer --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cfe726e699334..1380e9c5d6ff5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1827,8 +1827,16 @@ private Fetch pollForFetches(Timer timer) { return fetch; } - log.trace("Polling for fetches with timeout {} and {}", pollTimeout, timer.remainingMs()); + // 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 + // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call + // updateAssignmentMetadataIfNeeded before this method. + if (pollTimeout > retryBackoffMs) { + pollTimeout = retryBackoffMs; + } + + log.trace("Polling for fetches with timeout {}", pollTimeout); Timer pollTimer = time.timer(pollTimeout); wakeupTrigger.setFetchAction(fetchBuffer); From 68f5bc19d36410d4325565003d24f6386796e83e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 08:54:06 -0700 Subject: [PATCH 076/123] Whitespace change --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 1380e9c5d6ff5..807a6545a3d81 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1837,6 +1837,7 @@ private Fetch pollForFetches(Timer timer) { } log.trace("Polling for fetches with timeout {}", pollTimeout); + Timer pollTimer = time.timer(pollTimeout); wakeupTrigger.setFetchAction(fetchBuffer); From 5215030a4c94efb0ba0fc6107e8022cf8404dfdb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 08:57:08 -0700 Subject: [PATCH 077/123] Update AsyncKafkaConsumer.java --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 807a6545a3d81..4f3f9e26d5850 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1829,9 +1829,6 @@ private Fetch pollForFetches(Timer timer) { // 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 - - // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call - // updateAssignmentMetadataIfNeeded before this method. if (pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } From f660b15e52864e85cfd83928da95d28c066c5f1f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 14:21:30 -0700 Subject: [PATCH 078/123] Refactor regex subscription evaluation --- .../internals/AbstractMembershipManager.java | 11 ---- .../events/ApplicationEventProcessor.java | 54 ++++++++++++------- 2 files changed, 35 insertions(+), 30 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index e1ad2f6889af0..ffe01c089e7bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -43,7 +43,6 @@ import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import static java.util.Collections.unmodifiableList; @@ -488,16 +487,6 @@ public void onSubscriptionUpdated() { * active call to {@link org.apache.kafka.clients.consumer.KafkaConsumer#poll(Duration)}" */ public void onConsumerPoll() { - if (subscriptions.hasPatternSubscription()) { - final Set topicsToSubscribe = metadata.fetch().topics().stream() - .filter(subscriptions::matchesSubscribedPattern) - .collect(Collectors.toSet()); - if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { - metadata.requestUpdateForNewTopics(); - } - subscriptionUpdated.compareAndSet(false, true); - } - if (subscriptionUpdated.compareAndSet(true, false) && state == MemberState.UNSUBSCRIBED) { transitionToJoining(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index b940fb7458687..f783c11d55f65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -20,12 +20,15 @@ import org.apache.kafka.clients.consumer.internals.Acknowledgements; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; +import org.apache.kafka.clients.consumer.internals.ConsumerMembershipManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; +import org.apache.kafka.clients.consumer.internals.ShareMembershipManager; +import org.apache.kafka.clients.consumer.internals.StreamsMembershipManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.IsolationLevel; @@ -228,7 +231,9 @@ private void process(final SharePollEvent event) { requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> consumerMembershipManager.maybeReconcile(true)); requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); + ShareMembershipManager membershipManager = hrm.membershipManager(); + maybeUpdatePatternSubscription(membershipManager::onSubscriptionUpdated); + membershipManager.onConsumerPoll(); hrm.resetPollTimer(event.pollTimeMs()); }); } @@ -337,7 +342,7 @@ private void process(final TopicSubscriptionChangeEvent event) { if (subscriptions.subscribe(event.topics(), event.listener())) { this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics(); } - requestManagers.streamsMembershipManager.get().onSubscriptionUpdated(); + requestManagers.streamsGroupHeartbeatRequestManager.get().membershipManager().onSubscriptionUpdated(); event.future().complete(null); } catch (Exception e) { event.future().completeExceptionally(e); @@ -360,7 +365,10 @@ private void process(final TopicPatternSubscriptionChangeEvent event) { try { subscriptions.subscribe(event.pattern(), event.listener()); metadata.requestUpdateForNewTopics(); - updatePatternSubscription(metadata.fetch()); + requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { + ConsumerMembershipManager membershipManager = hrm.membershipManager(); + updatePatternSubscription(membershipManager::onSubscriptionUpdated, metadata.fetch()); + }); event.future().complete(null); } catch (Exception e) { event.future().completeExceptionally(e); @@ -394,13 +402,7 @@ private void process(final TopicRe2JPatternSubscriptionChangeEvent event) { * This will make the consumer send the updated subscription on the next poll. */ private void process(final UpdatePatternSubscriptionEvent event) { - if (!subscriptions.hasPatternSubscription()) { - return; - } - if (this.metadataVersionSnapshot < metadata.updateVersion()) { - this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(metadata.fetch()); - } + requestManagers.consumerMembershipManager.ifPresent(mm -> maybeUpdatePatternSubscription(mm::onSubscriptionUpdated)); event.future().complete(null); } @@ -724,11 +726,15 @@ private void process(final AsyncPollEvent event) { commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs()); requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); + ConsumerMembershipManager membershipManager = hrm.membershipManager(); + maybeUpdatePatternSubscription(membershipManager::onSubscriptionUpdated); + membershipManager.onConsumerPoll(); hrm.resetPollTimer(event.pollTimeMs()); }); requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm -> { - hrm.membershipManager().onConsumerPoll(); + StreamsMembershipManager membershipManager = hrm.membershipManager(); + maybeUpdatePatternSubscription(membershipManager::onSubscriptionUpdated); + membershipManager.onConsumerPoll(); hrm.resetPollTimer(event.pollTimeMs()); }); } @@ -807,6 +813,16 @@ protected ApplicationEventProcessor create() { }; } + private void maybeUpdatePatternSubscription(OnSubscriptionUpdatedCallback callback) { + if (!subscriptions.hasPatternSubscription()) { + return; + } + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + updatePatternSubscription(callback, metadata.fetch()); + } + } + /** * This function evaluates the regex that the consumer subscribed to * against the list of topic names from metadata, and updates @@ -814,26 +830,26 @@ protected ApplicationEventProcessor create() { * * @param cluster Cluster from which we get the topics */ - private void updatePatternSubscription(Cluster cluster) { - if (requestManagers.consumerHeartbeatRequestManager.isEmpty()) { - log.warn("Group membership manager not present when processing a subscribe event"); - return; - } + private void updatePatternSubscription(OnSubscriptionUpdatedCallback callback, Cluster cluster) { final Set topicsToSubscribe = cluster.topics().stream() .filter(subscriptions::matchesSubscribedPattern) .collect(Collectors.toSet()); if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics(); - } // Join the group if not already part of it, or just send the updated subscription // to the broker on the next poll. Note that this is done even if no topics matched // the regex, to ensure the member joins the group if needed (with empty subscription). - requestManagers.consumerHeartbeatRequestManager.get().membershipManager().onSubscriptionUpdated(); + callback.onSubscriptionUpdated(); } // Visible for testing int metadataVersionSnapshot() { return metadataVersionSnapshot; } + + private interface OnSubscriptionUpdatedCallback { + + void onSubscriptionUpdated(); + } } From dfdd693c7b2df7476afd4280a398b0bbad588e2f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 18:01:35 -0700 Subject: [PATCH 079/123] Removed logging from process(AsyncPollEvent) path --- .../internals/events/ApplicationEventProcessor.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index f783c11d55f65..bd1f7dbee60a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -714,8 +714,6 @@ private void process(final StreamsOnAllTasksLostCallbackCompletedEvent event) { } private void process(final AsyncPollEvent event) { - log.trace("Processing poll logic for {}", event); - // Trigger a reconciliation that can safely commit offsets if needed to rebalance, // as we're processing before any new fetching starts requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> @@ -739,22 +737,18 @@ private void process(final AsyncPollEvent event) { }); } - log.trace("Processing check and update positions logic for {}", event); CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) return; - log.trace("Processing create fetch requests logic for {}", event); - // Create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { if (maybeCompleteAsyncPollEventExceptionally(event, fetchError)) return; event.completeSuccessfully(); - log.trace("Completed event processing for {}", event); }); }); } From 626bb67707c60ddf46d12b3edafb5ec53b76a02d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 18:02:47 -0700 Subject: [PATCH 080/123] Fixed variable name typo --- .../clients/consumer/internals/ConsumerNetworkThread.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 9666041fc64a5..4e86f40aedfca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -385,11 +385,10 @@ private boolean maybeFailOnMetadataError(List events) { if (filteredEvents.isEmpty()) return false; - Optional andClearMetadataError = networkClientDelegate.getAndClearMetadataError(); + Optional metadataError = networkClientDelegate.getAndClearMetadataError(); - if (andClearMetadataError.isPresent()) { - Exception metadataError = andClearMetadataError.get(); - filteredEvents.forEach(e -> e.onMetadataError(metadataError)); + if (metadataError.isPresent()) { + filteredEvents.forEach(e -> e.onMetadataError(metadataError.get())); return true; } else { return false; From ef5e4da4b8e697d7fd4029b90c9639786c2cadf3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 18:09:59 -0700 Subject: [PATCH 081/123] Added brief documentation for the inner OnSubscriptionUpdatedCallback interface --- .../internals/events/ApplicationEventProcessor.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index bd1f7dbee60a4..9148aa85a95f8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.internals.AbstractMembershipManager; import org.apache.kafka.clients.consumer.internals.Acknowledgements; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -830,6 +831,7 @@ private void updatePatternSubscription(OnSubscriptionUpdatedCallback callback, C .collect(Collectors.toSet()); if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics(); + } // Join the group if not already part of it, or just send the updated subscription // to the broker on the next poll. Note that this is done even if no topics matched @@ -842,6 +844,11 @@ int metadataVersionSnapshot() { return metadataVersionSnapshot; } + /** + * This is only needed because the {@link StreamsMembershipManager} doesn't extend from + * {@link AbstractMembershipManager}, so {@link AbstractMembershipManager#onSubscriptionUpdated()} is not + * available, and this functional interface acts as a shim to support all three membership managers. + */ private interface OnSubscriptionUpdatedCallback { void onSubscriptionUpdated(); From 3595e829e3ad624b639fd0efe505409017642faa Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 18:15:08 -0700 Subject: [PATCH 082/123] Renamed OnSubscriptionUpdatedCallback -> MembershipManagerShim --- .../events/ApplicationEventProcessor.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 9148aa85a95f8..0f57304f956b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -808,13 +808,13 @@ protected ApplicationEventProcessor create() { }; } - private void maybeUpdatePatternSubscription(OnSubscriptionUpdatedCallback callback) { + private void maybeUpdatePatternSubscription(MembershipManagerShim membershipManager) { if (!subscriptions.hasPatternSubscription()) { return; } if (this.metadataVersionSnapshot < metadata.updateVersion()) { this.metadataVersionSnapshot = metadata.updateVersion(); - updatePatternSubscription(callback, metadata.fetch()); + updatePatternSubscription(membershipManager, metadata.fetch()); } } @@ -825,7 +825,7 @@ private void maybeUpdatePatternSubscription(OnSubscriptionUpdatedCallback callba * * @param cluster Cluster from which we get the topics */ - private void updatePatternSubscription(OnSubscriptionUpdatedCallback callback, Cluster cluster) { + private void updatePatternSubscription(MembershipManagerShim membershipManager, Cluster cluster) { final Set topicsToSubscribe = cluster.topics().stream() .filter(subscriptions::matchesSubscribedPattern) .collect(Collectors.toSet()); @@ -836,7 +836,7 @@ private void updatePatternSubscription(OnSubscriptionUpdatedCallback callback, C // Join the group if not already part of it, or just send the updated subscription // to the broker on the next poll. Note that this is done even if no topics matched // the regex, to ensure the member joins the group if needed (with empty subscription). - callback.onSubscriptionUpdated(); + membershipManager.onSubscriptionUpdated(); } // Visible for testing @@ -845,11 +845,12 @@ int metadataVersionSnapshot() { } /** - * This is only needed because the {@link StreamsMembershipManager} doesn't extend from - * {@link AbstractMembershipManager}, so {@link AbstractMembershipManager#onSubscriptionUpdated()} is not - * available, and this functional interface acts as a shim to support all three membership managers. + * Ideally the {@link AbstractMembershipManager#onSubscriptionUpdated()} API could be invoked directly for the + * three membership managers, but unfortunately {@link StreamsMembershipManager} doesn't extend from + * {@link AbstractMembershipManager}, so that method is not directly available. This functional interface acts + * as a shim to support all three membership managers. */ - private interface OnSubscriptionUpdatedCallback { + private interface MembershipManagerShim { void onSubscriptionUpdated(); } From 4e9c3133981ca49f142b084fd51d2cf79a860592 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 19:58:10 -0700 Subject: [PATCH 083/123] Revert unnecessary changes to ConsumerIntegrationTest --- .../consumer/ConsumerIntegrationTest.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java index 4112dc58acfbf..220866c240f4a 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -122,7 +122,7 @@ public void onPartitionsAssigned(Collection partitions) { } }); - TestUtils.waitForCondition(() -> consumer.poll(Duration.ofMillis(100)).count() == 1, + TestUtils.waitForCondition(() -> consumer.poll(Duration.ofSeconds(1)).count() == 1, 5000, "failed to poll data"); } @@ -266,11 +266,10 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec consumer1.subscribe(List.of(topic)); consumer2.subscribe(List.of(topic)); - Duration pollTimeout = Duration.ofMillis(100); TestUtils.waitForCondition(() -> { - consumer0.poll(pollTimeout); - consumer1.poll(pollTimeout); - consumer2.poll(pollTimeout); + consumer0.poll(Duration.ofMillis(1000)); + consumer1.poll(Duration.ofMillis(1000)); + consumer2.poll(Duration.ofMillis(1000)); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().isEmpty() && consumer2.assignment().isEmpty(); @@ -285,9 +284,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec ); clusterInstance.waitTopicCreation(topic, 3); TestUtils.waitForCondition(() -> { - consumer0.poll(pollTimeout); - consumer1.poll(pollTimeout); - consumer2.poll(pollTimeout); + consumer0.poll(Duration.ofMillis(1000)); + consumer1.poll(Duration.ofMillis(1000)); + consumer2.poll(Duration.ofMillis(1000)); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) && consumer2.assignment().isEmpty(); @@ -302,9 +301,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec ); clusterInstance.waitTopicCreation(topic, 6); TestUtils.waitForCondition(() -> { - consumer0.poll(pollTimeout); - consumer1.poll(pollTimeout); - consumer2.poll(pollTimeout); + consumer0.poll(Duration.ofMillis(1000)); + consumer1.poll(Duration.ofMillis(1000)); + consumer2.poll(Duration.ofMillis(1000)); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) && consumer2.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4), new TopicPartition(topic, 5))); @@ -326,9 +325,9 @@ public void testRackAwareAssignment(ClusterInstance clusterInstance) throws Exec new TopicPartition(topic, 5), Optional.of(new NewPartitionReassignment(List.of(0))) )).all().get(); TestUtils.waitForCondition(() -> { - consumer0.poll(pollTimeout); - consumer1.poll(pollTimeout); - consumer2.poll(pollTimeout); + consumer0.poll(Duration.ofMillis(1000)); + consumer1.poll(Duration.ofMillis(1000)); + consumer2.poll(Duration.ofMillis(1000)); return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 5))) && consumer1.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4))) && consumer2.assignment().equals(Set.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(topic, 2))); From f9c03455dce53d099053c4a374c198453defca46 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 20:09:04 -0700 Subject: [PATCH 084/123] Minor correction in code comment --- .../clients/consumer/internals/ConsumerNetworkThread.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 4e86f40aedfca..67656cf327b5b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -195,9 +195,9 @@ private void processApplicationEvents() { if (event instanceof CompletableEvent) { applicationEventReaper.add((CompletableEvent) event); } - // Check if there are any metadata errors and fail the CompletableEvent if an error is present. - // This call is meant to handle "immediately completed events" which may not enter the awaiting state, - // so metadata errors need to be checked and handled right away. + // Check if there are any metadata errors and fail the event if an error is present. + // This call is meant to handle "immediately completed events" which may not enter the + // awaiting state, so metadata errors need to be checked and handled right away. if (event instanceof MetadataErrorNotifiableEvent) { if (maybeFailOnMetadataError(List.of(event))) continue; From 293f08b691556c23f6b09720385d13bc910c8c86 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 20:30:34 -0700 Subject: [PATCH 085/123] ConsumerIntegrationTest needs more time to resolve the assignment --- .../apache/kafka/clients/consumer/ConsumerIntegrationTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java index 220866c240f4a..c202bd222af22 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -123,7 +123,6 @@ public void onPartitionsAssigned(Collection partitions) { }); TestUtils.waitForCondition(() -> consumer.poll(Duration.ofSeconds(1)).count() == 1, - 5000, "failed to poll data"); } } From f26e8327703b6e4ed452fd42b0dc77c51f9b71fb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 20:31:25 -0700 Subject: [PATCH 086/123] Reworking logic in pollForFetches --- .../consumer/internals/AsyncKafkaConsumer.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 4f3f9e26d5850..7aa6a47417e8b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1827,10 +1827,18 @@ private Fetch pollForFetches(Timer timer) { return fetch; } - // 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 + // With the non-blocking poll design, it's possible that at this point the background thread is + // concurrently working to update positions. Therefore, a _copy_ of the current assignment is retrieved + // and iterated looking for any partitions with invalid positions. This is done to avoid being stuck + // in poll for an unnecessarily long amount of time if we are missing some positions since the offset + // lookup may be backing off after a failure. if (pollTimeout > retryBackoffMs) { - pollTimeout = retryBackoffMs; + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (!subscriptions.hasValidPosition(tp)) { + pollTimeout = retryBackoffMs; + break; + } + } } log.trace("Polling for fetches with timeout {}", pollTimeout); From ba0fdacf1aa340d1e78e431ca3a7b07101c3939d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 7 Oct 2025 20:34:35 -0700 Subject: [PATCH 087/123] Removed an outdated comment in ApplicationEventProcessor --- .../consumer/internals/events/ApplicationEventProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 0f57304f956b3..872144c748d1f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -744,7 +744,6 @@ private void process(final AsyncPollEvent event) { if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) return; - // Create a fetch request if there's no data in the FetchBuffer. requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> { if (maybeCompleteAsyncPollEventExceptionally(event, fetchError)) return; From c664ddf95970afedc7d06fb5ce053e957ec94bc2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 07:30:05 -0700 Subject: [PATCH 088/123] Revert change for onConsumerPoll() to limit to the async consumer and streams --- .../internals/events/ApplicationEventProcessor.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 872144c748d1f..4bae553e6cb62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -28,7 +28,6 @@ import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager; -import org.apache.kafka.clients.consumer.internals.ShareMembershipManager; import org.apache.kafka.clients.consumer.internals.StreamsMembershipManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; @@ -232,9 +231,7 @@ private void process(final SharePollEvent event) { requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager -> consumerMembershipManager.maybeReconcile(true)); requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> { - ShareMembershipManager membershipManager = hrm.membershipManager(); - maybeUpdatePatternSubscription(membershipManager::onSubscriptionUpdated); - membershipManager.onConsumerPoll(); + hrm.membershipManager().onConsumerPoll(); hrm.resetPollTimer(event.pollTimeMs()); }); } @@ -844,10 +841,9 @@ int metadataVersionSnapshot() { } /** - * Ideally the {@link AbstractMembershipManager#onSubscriptionUpdated()} API could be invoked directly for the - * three membership managers, but unfortunately {@link StreamsMembershipManager} doesn't extend from - * {@link AbstractMembershipManager}, so that method is not directly available. This functional interface acts - * as a shim to support all three membership managers. + * Ideally the {@link AbstractMembershipManager#onSubscriptionUpdated()} API could be invoked directly, but + * unfortunately {@link StreamsMembershipManager} doesn't extend from {@link AbstractMembershipManager}, so + * that method is not directly available. This functional interface acts as a shim to support both. */ private interface MembershipManagerShim { From 45af5713007a3cb72370e7d7d8133e5ae50ff3da Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 08:12:54 -0700 Subject: [PATCH 089/123] Revert some changes in KafkaConsumerTest --- .../clients/consumer/KafkaConsumerTest.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 40db20e7be045..499596ec0f387 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -935,6 +935,7 @@ public void verifyPollTimesOutDuringMetadataUpdate(GroupProtocol groupProtocol) @ParameterizedTest @EnumSource(GroupProtocol.class) + @SuppressWarnings("unchecked") public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -950,7 +951,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol g client.prepareResponse(listOffsetsResponse(Map.of(tp0, 50L))); client.prepareResponse(fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -1000,6 +1001,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit(GroupPr @ParameterizedTest @EnumSource(value = GroupProtocol.class) + @SuppressWarnings("unchecked") public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupProtocol) { // Verifies that we can make progress on one partition while we are awaiting // a reset on another partition. @@ -1044,7 +1046,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); assertEquals(1, records.nextOffsets().size()); @@ -1728,6 +1730,7 @@ private void initializeSubscriptionWithSingleTopic(KafkaConsumer consumer, @ParameterizedTest @EnumSource(GroupProtocol.class) + @SuppressWarnings("unchecked") public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1762,7 +1765,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(100)); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); @@ -1785,6 +1788,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP @ParameterizedTest @EnumSource(GroupProtocol.class) + @SuppressWarnings("unchecked") public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1821,7 +1825,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); - ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -2116,7 +2120,7 @@ public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol gro time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); - final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + final ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ZERO); assertFalse(records.isEmpty()); assertFalse(records.nextOffsets().isEmpty()); } @@ -2649,6 +2653,7 @@ public void testInvalidGroupMetadata(GroupProtocol groupProtocol) throws Interru @ParameterizedTest @EnumSource(GroupProtocol.class) + @SuppressWarnings("unchecked") public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedException { final ConsumerMetadata metadata = createMetadata(subscription); final MockClient client = new MockClient(time, metadata); @@ -2714,7 +2719,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept final FetchInfo fetchInfo = new FetchInfo(1L, 99L, 50L, 5); client.respondToRequest(fetchRequest, fetchResponse(Map.of(tp0, fetchInfo))); - final ConsumerRecords records = ConsumerPollTestUtils.waitForRecords(consumer); + final ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); assertEquals(1, records.nextOffsets().size()); @@ -3172,7 +3177,7 @@ private static class FetchInfo { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws InterruptedException { + public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); From 328cb135de24d1f27a9a6a5c602d21027c3e283f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 08:18:36 -0700 Subject: [PATCH 090/123] Minor tweaks to KafkaConsumerTest to minimize diff noise --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 499596ec0f387..b9be0cf00a4f9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1001,7 +1001,6 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit(GroupPr @ParameterizedTest @EnumSource(value = GroupProtocol.class) - @SuppressWarnings("unchecked") public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupProtocol) { // Verifies that we can make progress on one partition while we are awaiting // a reset on another partition. @@ -1046,6 +1045,7 @@ public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupPro }, fetchResponse(tp0, 50L, 5)); + @SuppressWarnings("unchecked") ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(Set.of(tp0), records.partitions()); @@ -1788,7 +1788,6 @@ public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupP @ParameterizedTest @EnumSource(GroupProtocol.class) - @SuppressWarnings("unchecked") public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol groupProtocol) { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1825,6 +1824,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol group client.prepareResponse(listOffsetsResponse(Map.of(tp0, 10L))); client.prepareResponse(fetchResponse(tp0, 10L, 1)); + @SuppressWarnings("unchecked") ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1)); assertEquals(1, records.count()); assertEquals(11L, consumer.position(tp0)); From 5109b825fdb94861060ebaeb200cfd9624098cac Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 11:38:07 -0700 Subject: [PATCH 091/123] Revert changes to PlaintextConsumerTest --- .../consumer/PlaintextConsumerTest.java | 59 ++++++++++++------- 1 file changed, 38 insertions(+), 21 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index 323be036e08eb..6be82a1e0f12a 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -66,6 +66,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.IntStream; import static org.apache.kafka.clients.ClientsTestUtils.BaseConsumerTestcase.BROKER_COUNT; @@ -809,7 +810,7 @@ private void testPerPartitionLeadMetricsCleanUpWithSubscribe( // Create a consumer and consumer some messages. var listener = new TestConsumerReassignmentListener(); consumer.subscribe(List.of(TOPIC, topic2), listener); - var records = ConsumerPollTestUtils.waitForRecords(consumer); + var records = awaitNonEmptyRecords(consumer, TP); assertEquals(1, listener.callsToAssigned, "should be assigned once"); // Verify the metric exist. @@ -876,7 +877,7 @@ private void testPerPartitionLagMetricsCleanUpWithSubscribe( // Create a consumer and consumer some messages. var listener = new TestConsumerReassignmentListener(); consumer.subscribe(List.of(TOPIC, topic2), listener); - var records = ConsumerPollTestUtils.waitForRecords(consumer); + var records = awaitNonEmptyRecords(consumer, TP); assertEquals(1, listener.callsToAssigned, "should be assigned once"); // Verify the metric exist. @@ -943,7 +944,7 @@ private void testPerPartitionLeadMetricsCleanUpWithAssign( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - var records = ConsumerPollTestUtils.waitForRecords(consumer); + var records = awaitNonEmptyRecords(consumer, TP); // Verify the metric exist. Map tags = Map.of( @@ -957,7 +958,7 @@ private void testPerPartitionLeadMetricsCleanUpWithAssign( assertEquals((double) records.count(), fetchLead.metricValue(), "The lead should be " + records.count()); consumer.assign(List.of(tp2)); - ConsumerPollTestUtils.waitForRecords(consumer); + awaitNonEmptyRecords(consumer, tp2); assertNull(consumer.metrics().get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags))); } } @@ -998,7 +999,7 @@ private void testPerPartitionLagMetricsCleanUpWithAssign( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - var records = ConsumerPollTestUtils.waitForRecords(consumer); + var records = awaitNonEmptyRecords(consumer, TP); // Verify the metric exist. Map tags = Map.of( @@ -1013,7 +1014,7 @@ private void testPerPartitionLagMetricsCleanUpWithAssign( var expectedLag = numMessages - records.count(); assertEquals(expectedLag, (double) fetchLag.metricValue(), EPSILON, "The lag should be " + expectedLag); consumer.assign(List.of(tp2)); - ConsumerPollTestUtils.waitForRecords(consumer); + awaitNonEmptyRecords(consumer, tp2); assertNull(consumer.metrics().get(new MetricName(TP + ".records-lag", "consumer-fetch-manager-metrics", "", tags))); assertNull(consumer.metrics().get(new MetricName("records-lag", "consumer-fetch-manager-metrics", "", tags))); } @@ -1057,7 +1058,7 @@ private void testPerPartitionLagMetricsWhenReadCommitted( sendRecords(producer, tp2, numMessages, System.currentTimeMillis()); consumer.assign(List.of(TP)); - ConsumerPollTestUtils.waitForRecords(consumer); + awaitNonEmptyRecords(consumer, TP); // Verify the metric exist. Map tags = Map.of( @@ -1202,21 +1203,12 @@ private void testConsumingWithNullGroupId(Map consumerConfig) th consumer3.assign(List.of(TP)); consumer3.seek(TP, 1); - TestUtils.waitForCondition( - () -> consumer1.poll(Duration.ofMillis(5000)).count() == 3, - "consumer1 did not consume from earliest offset" - ); + var numRecords1 = consumer1.poll(Duration.ofMillis(5000)).count(); assertThrows(InvalidGroupIdException.class, consumer1::commitSync); assertThrows(InvalidGroupIdException.class, () -> consumer2.committed(Set.of(TP))); - TestUtils.waitForCondition( - () -> consumer2.poll(Duration.ofMillis(5000)).count() == 0, - "Expected consumer2 to consume from latest offset" - ); - TestUtils.waitForCondition( - () -> consumer3.poll(Duration.ofMillis(5000)).count() == 2, - "Expected consumer3 to consume from offset 1" - ); + var numRecords2 = consumer2.poll(Duration.ofMillis(5000)).count(); + var numRecords3 = consumer3.poll(Duration.ofMillis(5000)).count(); consumer1.unsubscribe(); consumer2.unsubscribe(); @@ -1225,7 +1217,14 @@ private void testConsumingWithNullGroupId(Map consumerConfig) th assertTrue(consumer1.assignment().isEmpty()); assertTrue(consumer2.assignment().isEmpty()); assertTrue(consumer3.assignment().isEmpty()); - } + + consumer1.close(); + consumer2.close(); + consumer3.close(); + + assertEquals(3, numRecords1, "Expected consumer1 to consume from earliest offset"); + assertEquals(0, numRecords2, "Expected consumer2 to consume from latest offset"); + assertEquals(2, numRecords3, "Expected consumer3 to consume from offset 1"); } } @ClusterTest @@ -1654,7 +1653,7 @@ private void testStallBetweenPoll(GroupProtocol groupProtocol) throws Exception consumer.subscribe(List.of(testTopic)); // This is here to allow the consumer time to settle the group membership/assignment. - ConsumerPollTestUtils.waitForRecords(consumer); + awaitNonEmptyRecords(consumer, new TopicPartition(testTopic, 0)); // Keep track of the last time the poll is invoked to ensure the deltas between invocations don't // exceed the delay threshold defined above. @@ -1674,6 +1673,24 @@ private void testStallBetweenPoll(GroupProtocol groupProtocol) throws Exception } } + private ConsumerRecords awaitNonEmptyRecords( + Consumer consumer, + TopicPartition tp + ) throws Exception { + AtomicReference> result = new AtomicReference<>(); + + TestUtils.waitForCondition(() -> { + var polledRecords = consumer.poll(Duration.ofSeconds(1)); + boolean hasRecords = !polledRecords.isEmpty(); + if (hasRecords) { + result.set(polledRecords); + } + return hasRecords; + }, "Timed out waiting for non-empty records from topic " + tp.topic() + " partition " + tp.partition()); + + return result.get(); + } + public static class SerializerImpl implements Serializer { private final ByteArraySerializer serializer = new ByteArraySerializer(); From 0bde7902023703665a31f6b7307bf92c4829f931 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 11:38:37 -0700 Subject: [PATCH 092/123] Update PlaintextConsumerTest.java --- .../apache/kafka/clients/consumer/PlaintextConsumerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index 6be82a1e0f12a..d4375898ddb25 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -1224,7 +1224,8 @@ private void testConsumingWithNullGroupId(Map consumerConfig) th assertEquals(3, numRecords1, "Expected consumer1 to consume from earliest offset"); assertEquals(0, numRecords2, "Expected consumer2 to consume from latest offset"); - assertEquals(2, numRecords3, "Expected consumer3 to consume from offset 1"); } + assertEquals(2, numRecords3, "Expected consumer3 to consume from offset 1"); + } } @ClusterTest From 0e1180f8608f4f7b61cbd8726a403d656354860d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 11:45:06 -0700 Subject: [PATCH 093/123] Reverting changes to PlaintextConsumerCommitTest --- .../clients/consumer/PlaintextConsumerCommitTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java index b0519077d092e..5f2a2b15784a2 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java @@ -283,14 +283,11 @@ public void onPartitionsRevoked(Collection partitions) { // In both CLASSIC and CONSUMER protocols, interceptors are executed in poll and close. // However, in the CONSUMER protocol, the assignment may be changed outside a poll, so // we need to poll once to ensure the interceptor is called. - TestUtils.waitForCondition( - () -> { - consumer.poll(Duration.ZERO); - return MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance; - }, - "Consumer.poll() did not invoke onCommit() before timeout elapse" - ); + if (groupProtocol == GroupProtocol.CONSUMER) { + consumer.poll(Duration.ZERO); + } + assertTrue(MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance); // verify commits are intercepted on close var commitCountBeforeClose = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue(); consumer.close(); From 1a4fe5dc3ad42e86b22e549561c16ab1fad9b114 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 11:46:00 -0700 Subject: [PATCH 094/123] Update PlaintextConsumerCommitTest.java --- .../kafka/clients/consumer/PlaintextConsumerCommitTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java index 5f2a2b15784a2..c00d1ddab90a0 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java @@ -288,6 +288,7 @@ public void onPartitionsRevoked(Collection partitions) { } assertTrue(MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance); + // verify commits are intercepted on close var commitCountBeforeClose = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue(); consumer.close(); From 802d7d1a4bdd24b5314e4b4c29ec8f0a0e30f29f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 14:34:05 -0700 Subject: [PATCH 095/123] Revert more changes to AsyncKafkaConsumerTest --- .../internals/AsyncKafkaConsumerTest.java | 30 +++++-------------- 1 file changed, 8 insertions(+), 22 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 1cfe218519101..f9899b0e6ff26 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -112,7 +112,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Properties; import java.util.Set; @@ -508,11 +507,8 @@ public void onPartitionsAssigned(final Collection partitions) { completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList(topicName), listener); completeAsyncPollEventSuccessfully(); - ConsumerPollTestUtils.waitForCondition( - consumer, - callbackExecuted::get, - "Consumer.poll() did not execute callback within timeout" - ); + consumer.poll(Duration.ZERO); + assertTrue(callbackExecuted.get()); } @Test @@ -679,11 +675,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); completeAsyncPollEventSuccessfully(); - ConsumerPollTestUtils.waitForCondition( - consumer, - () -> callback.invoked == 1 && callback.exception == null, - "Consumer.poll() did not execute the callback once (without error) in allottec timeout" - ); + assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @Test @@ -1464,7 +1456,7 @@ public void testListenerCallbacksInvoke(List expectedExceptionOpt + Optional expectedException ) { consumer = newConsumer(); CounterConsumerRebalanceListener consumerRebalanceListener = new CounterConsumerRebalanceListener( @@ -1485,16 +1477,10 @@ public void testListenerCallbacksInvoke(List Objects.equals(t.getClass(), expectedException.getClass()) && - Objects.equals(t.getMessage(), expectedException.getMessage()) && - Objects.equals(t.getCause(), expectedException.getCause()), - "Consumer.poll() did not throw the expected exception " + expectedException - ); - } else { + if (expectedException.isPresent()) { + Exception exception = assertThrows(expectedException.get().getClass(), () -> consumer.poll(Duration.ZERO)); + assertEquals(expectedException.get().getMessage(), exception.getMessage()); + assertEquals(expectedException.get().getCause(), exception.getCause()); } else { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } From cd516e3dd308b8c26218de2feef5fc0905944034 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 15:00:29 -0700 Subject: [PATCH 096/123] Fixed code tweak --- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index f9899b0e6ff26..aecbd5747994d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1480,7 +1480,8 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); assertEquals(expectedException.get().getMessage(), exception.getMessage()); - assertEquals(expectedException.get().getCause(), exception.getCause()); } else { + assertEquals(expectedException.get().getCause(), exception.getCause()); + } else { when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } From 089bb24c14c4f6e6fc6a1d7342aa3afcd8a8cc05 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 17:18:26 -0700 Subject: [PATCH 097/123] Test clean up --- .../consumer/ConsumerPollTestUtils.java | 103 ------------------ .../clients/consumer/KafkaConsumerTest.java | 95 ++++++++-------- .../internals/AsyncKafkaConsumerTest.java | 24 ++-- .../events/ApplicationEventProcessorTest.java | 15 ++- 4 files changed, 69 insertions(+), 168 deletions(-) delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java deleted file mode 100644 index 82e479c040c80..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPollTestUtils.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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; - -import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; -import org.apache.kafka.common.errors.InterruptException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.test.TestUtils; - -import java.time.Duration; -import java.util.function.Function; -import java.util.function.Supplier; - -import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; - -/** - * This class provides utilities for tests to wait for a call to {@link Consumer#poll(Duration)} to produce a - * result (error, records, specific condition, etc.). This is mostly due to the subtle difference in behavior - * of the non-blocking {@link AsyncKafkaConsumer}. A single pass of {@link AsyncKafkaConsumer#poll(Duration)} - * may not be sufficient to provide an immediate result. - */ -public class ConsumerPollTestUtils { - - /** - * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} to return records from the given {@link Consumer}. - */ - public static ConsumerRecords waitForRecords(Consumer consumer) { - Timer timer = Time.SYSTEM.timer(DEFAULT_MAX_WAIT_MS); - - while (timer.notExpired()) { - @SuppressWarnings("unchecked") - ConsumerRecords records = (ConsumerRecords) consumer.poll(Duration.ofMillis(1000)); - - if (!records.isEmpty()) - return records; - - timer.update(); - } - - throw new TimeoutException("no records to return"); - } - - /** - * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} for the {@link Consumer} to produce the side effect - * that causes {@link Supplier condition} to evaluate to {@code true}. - */ - public static void waitForCondition(Consumer consumer, - Supplier testCondition, - String conditionDetails) { - try { - TestUtils.waitForCondition( - () -> { - consumer.poll(Duration.ZERO); - return testCondition.get(); - }, - conditionDetails - ); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } - - /** - * Wait up to {@link TestUtils#DEFAULT_MAX_WAIT_MS} for the {@link Consumer} to throw an exception that, - * when tested against the {@link Function condition}, will evaluate to {@code true}. - */ - public static void waitForException(Consumer consumer, - Function testCondition, - String conditionDetails) { - try { - TestUtils.waitForCondition( - () -> { - try { - consumer.poll(Duration.ZERO); - return false; - } catch (Throwable t) { - return testCondition.apply(t); - } - }, - conditionDetails - ); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index b9be0cf00a4f9..9d01c74d01933 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1065,7 +1065,7 @@ private void initMetadata(MockClient mockClient, Map partitionC @ParameterizedTest @EnumSource(value = GroupProtocol.class) - public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) { + public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) throws InterruptedException { SubscriptionState subscription = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE); ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -1081,14 +1081,15 @@ public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) { true, groupId, groupInstanceId, false); consumer.assign(List.of(tp0)); - // Consumer.poll(0) needs to wait for the offset fetch event added by a call to poll, to be processed - // by the background thread, so it can realize there are no committed offsets and then - // throw the NoOffsetForPartitionException. - ConsumerPollTestUtils.waitForException( - consumer, - NoOffsetForPartitionException.class::isInstance, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout" - ); + if (groupProtocol == GroupProtocol.CONSUMER) { + // New consumer poll(ZERO) needs to wait for the offset fetch event added by a call to poll, to be processed + // by the background thread, so it can realize there are no committed offsets and then + // throw the NoOffsetForPartitionException + assertPollEventuallyThrows(consumer, NoOffsetForPartitionException.class, + "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); + } else { + assertThrows(NoOffsetForPartitionException.class, () -> consumer.poll(Duration.ZERO)); + } } @ParameterizedTest @@ -2270,18 +2271,19 @@ public void testEndOffsetsAuthenticationFailure(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testPollAuthenticationFailure(GroupProtocol groupProtocol) { + public void testPollAuthenticationFailure(GroupProtocol groupProtocol) throws InterruptedException { final KafkaConsumer consumer = consumerWithPendingAuthenticationError(groupProtocol); consumer.subscribe(Set.of(topic)); - // Consumer.poll(0) needs to wait for the event added by a call to poll, to be processed - // by the background thread, so it can realize there is authentication fail and then - // throw the AuthenticationException. - ConsumerPollTestUtils.waitForException( - consumer, - AuthenticationException.class::isInstance, - "this consumer was not able to discover metadata errors during continuous polling." - ); + if (groupProtocol == GroupProtocol.CONSUMER) { + // New consumer poll(ZERO) needs to wait for the event added by a call to poll, to be processed + // by the background thread, so it can realize there is authentication fail and then + // throw the AuthenticationException + assertPollEventuallyThrows(consumer, AuthenticationException.class, + "this consumer was not able to discover metadata errors during continuous polling."); + } else { + assertThrows(AuthenticationException.class, () -> consumer.poll(Duration.ZERO)); + } } // TODO: this test triggers a bug with the CONSUMER group protocol implementation. @@ -2668,11 +2670,9 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept consumer.assign(Set.of(tp0)); // poll once to update with the current metadata - ConsumerPollTestUtils.waitForCondition( - consumer, - () -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), - "No metadata requests sent" - ); + consumer.poll(Duration.ofMillis(0)); + TestUtils.waitForCondition(() -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), + "No metadata requests sent"); client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, metadata.fetch().nodes().get(0))); // no error for no current position @@ -2685,11 +2685,11 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept } // poll once again, which should send the list-offset request consumer.seek(tp0, 50L); + consumer.poll(Duration.ofMillis(0)); // requests: list-offset - ConsumerPollTestUtils.waitForCondition( - consumer, + TestUtils.waitForCondition( () -> requestGenerated(client, ApiKeys.LIST_OFFSETS), - "No list-offset sent" + "No list-offset request sent" ); // no error for no end offset (so unknown lag) @@ -2699,11 +2699,11 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // and hence next call would return correct lag result ClientRequest listOffsetRequest = findRequest(client, ApiKeys.LIST_OFFSETS); client.respondToRequest(listOffsetRequest, listOffsetsResponse(Map.of(tp0, 90L))); + consumer.poll(Duration.ofMillis(0)); // requests: fetch - ConsumerPollTestUtils.waitForCondition( - consumer, + TestUtils.waitForCondition( () -> requestGenerated(client, ApiKeys.FETCH), - "No fetch sent" + "No fetch request sent" ); // For AsyncKafkaConsumer, subscription state is updated in background, so the result will eventually be updated. @@ -3177,7 +3177,7 @@ private static class FetchInfo { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) { + public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws InterruptedException { ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); @@ -3198,14 +3198,27 @@ public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) { KafkaConsumer consumer = newConsumer(groupProtocol, time, client, subscription, metadata, assignor, true, groupInstanceId); consumer.subscribe(Set.of(invalidTopicName), getConsumerRebalanceListener(consumer)); - // Consumer.poll(0) needs to wait for the event added by a call to poll, to be processed - // by the background thread, so it can realize there is invalid topics and then - // throw the InvalidTopicException. - ConsumerPollTestUtils.waitForException( - consumer, - InvalidTopicException.class::isInstance, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout" - ); + if (groupProtocol == GroupProtocol.CONSUMER) { + // New consumer poll(ZERO) needs to wait for the event added by a call to poll, to be processed + // by the background thread, so it can realize there is invalid topics and then + // throw the InvalidTopicException + assertPollEventuallyThrows(consumer, InvalidTopicException.class, + "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); + } else { + assertThrows(InvalidTopicException.class, () -> consumer.poll(Duration.ZERO)); + } + } + + private static void assertPollEventuallyThrows(KafkaConsumer consumer, + Class expectedException, String errMsg) throws InterruptedException { + TestUtils.waitForCondition(() -> { + try { + consumer.poll(Duration.ZERO); + return false; + } catch (Throwable exception) { + return expectedException.isInstance(exception); + } + }, errMsg); } @ParameterizedTest @@ -3645,11 +3658,7 @@ public void testPreventMultiThread(GroupProtocol groupProtocol) throws Interrupt service.execute(() -> consumer.poll(Duration.ofSeconds(5))); try { TimeUnit.SECONDS.sleep(1); - ConsumerPollTestUtils.waitForException( - consumer, - t -> t instanceof ConcurrentModificationException, - "Consumer did not throw ConcurrentModificationException within timeout" - ); + assertThrows(ConcurrentModificationException.class, () -> consumer.poll(Duration.ZERO)); client.wakeup(); consumer.wakeup(); } finally { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index aecbd5747994d..9df6deab359b0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.CloseOptions; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerPollTestUtils; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -1545,11 +1544,9 @@ public void testBackgroundError() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); completeAsyncPollEventSuccessfully(); - ConsumerPollTestUtils.waitForException( - consumer, - t -> t.getMessage().equals(expectedException.getMessage()), - "Consumer.poll() did not fail with expected exception " + expectedException + " within timeout" - ); + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException.getMessage(), exception.getMessage()); } @Test @@ -1566,11 +1563,9 @@ public void testMultipleBackgroundErrors() { completeAssignmentChangeEventSuccessfully(); consumer.assign(singletonList(new TopicPartition("topic", 0))); completeAsyncPollEventSuccessfully(); - ConsumerPollTestUtils.waitForException( - consumer, - t -> t.getMessage().equals(expectedException1.getMessage()), - "Consumer.poll() did not fail with expected exception " + expectedException1 + " within timeout" - ); + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException1.getMessage(), exception.getMessage()); assertTrue(backgroundEventQueue.isEmpty()); } @@ -1839,12 +1834,7 @@ void testReaperInvokedInPoll() { consumer.subscribe(Collections.singletonList("topic")); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAsyncPollEventSuccessfully(); - - ConsumerPollTestUtils.waitForCondition( - consumer, - () -> backgroundEventReaper.size() == 0, - "Consumer.poll() did not reap background events within timeout" - ); + consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index bcd42abe2ae90..fe294f030e690 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -88,6 +88,7 @@ public class ApplicationEventProcessorTest { private final ConsumerHeartbeatRequestManager heartbeatRequestManager = mock(ConsumerHeartbeatRequestManager.class); private final ConsumerMembershipManager membershipManager = mock(ConsumerMembershipManager.class); private final OffsetsRequestManager offsetsRequestManager = mock(OffsetsRequestManager.class); + private final FetchRequestManager fetchRequestManager = mock(FetchRequestManager.class); private SubscriptionState subscriptionState = mock(SubscriptionState.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock(StreamsGroupHeartbeatRequestManager.class); @@ -99,7 +100,7 @@ private void setupProcessor(boolean withGroupId) { new LogContext(), offsetsRequestManager, mock(TopicMetadataRequestManager.class), - mock(FetchRequestManager.class), + fetchRequestManager, withGroupId ? Optional.of(mock(CoordinatorRequestManager.class)) : Optional.empty(), withGroupId ? Optional.of(commitRequestManager) : Optional.empty(), withGroupId ? Optional.of(heartbeatRequestManager) : Optional.empty(), @@ -264,16 +265,20 @@ public void testSeekUnvalidatedEventWithException() { } @Test - public void testPollEvent() { + public void testAsyncPollEvent() { AsyncPollEvent event = new AsyncPollEvent(12346, 12345); setupProcessor(true); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); - when(offsetsRequestManager.updateFetchPositions(anyLong())).thenReturn(new CompletableFuture<>()); + when(offsetsRequestManager.updateFetchPositions(event.deadlineMs())).thenReturn(CompletableFuture.completedFuture(true)); + when(fetchRequestManager.createFetchRequests()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); - verify(commitRequestManager).updateTimerAndMaybeCommit(12345); + assertTrue(event.isComplete()); + verify(commitRequestManager).updateTimerAndMaybeCommit(event.pollTimeMs()); verify(membershipManager).onConsumerPoll(); - verify(heartbeatRequestManager).resetPollTimer(12345); + verify(heartbeatRequestManager).resetPollTimer(event.pollTimeMs()); + verify(offsetsRequestManager).updateFetchPositions(event.deadlineMs()); + verify(fetchRequestManager).createFetchRequests(); } @Test From 8e86cf465f816d8901e1fbab9ca09ad6a4f0caad Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 8 Oct 2025 17:20:44 -0700 Subject: [PATCH 098/123] Removed whitespace diffs --- .../kafka/clients/consumer/KafkaConsumerTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 9d01c74d01933..77fd8dd1eb289 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1086,7 +1086,7 @@ public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) throws I // by the background thread, so it can realize there are no committed offsets and then // throw the NoOffsetForPartitionException assertPollEventuallyThrows(consumer, NoOffsetForPartitionException.class, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); + "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); } else { assertThrows(NoOffsetForPartitionException.class, () -> consumer.poll(Duration.ZERO)); } @@ -2280,7 +2280,7 @@ public void testPollAuthenticationFailure(GroupProtocol groupProtocol) throws In // by the background thread, so it can realize there is authentication fail and then // throw the AuthenticationException assertPollEventuallyThrows(consumer, AuthenticationException.class, - "this consumer was not able to discover metadata errors during continuous polling."); + "this consumer was not able to discover metadata errors during continuous polling."); } else { assertThrows(AuthenticationException.class, () -> consumer.poll(Duration.ZERO)); } @@ -2672,7 +2672,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // poll once to update with the current metadata consumer.poll(Duration.ofMillis(0)); TestUtils.waitForCondition(() -> requestGenerated(client, ApiKeys.FIND_COORDINATOR), - "No metadata requests sent"); + "No metadata requests sent"); client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, metadata.fetch().nodes().get(0))); // no error for no current position @@ -3203,14 +3203,14 @@ public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) throws I // by the background thread, so it can realize there is invalid topics and then // throw the InvalidTopicException assertPollEventuallyThrows(consumer, InvalidTopicException.class, - "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); + "Consumer was not able to update fetch positions on continuous calls with 0 timeout"); } else { assertThrows(InvalidTopicException.class, () -> consumer.poll(Duration.ZERO)); } } private static void assertPollEventuallyThrows(KafkaConsumer consumer, - Class expectedException, String errMsg) throws InterruptedException { + Class expectedException, String errMsg) throws InterruptedException { TestUtils.waitForCondition(() -> { try { consumer.poll(Duration.ZERO); From ef07ee10072dfea8cf762332d35830d5e1612404 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 9 Oct 2025 22:28:36 -0700 Subject: [PATCH 099/123] Trigger build From 1bf56bf57f620bdc3843a3448b4de98f3136cd9f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 10 Oct 2025 11:30:05 -0700 Subject: [PATCH 100/123] Reverted NetworkClientDelegate authentication error check --- .../internals/NetworkClientDelegate.java | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index ca0492e39b569..31c402df2a6db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -210,30 +210,6 @@ boolean doSend(final UnsentRequest r, final long currentTimeMs) { } ClientRequest request = makeClientRequest(r, node, currentTimeMs); if (!client.ready(node, currentTimeMs)) { - AuthenticationException authenticationException = client.authenticationException(node); - - // The client may not be ready because it hit an unrecoverable authentication error. In that case, there's - // no benefit from retrying, so propagate the error here. - if (authenticationException != null) { - request.callback().onComplete( - new ClientResponse( - request.makeHeader( - request.requestBuilder().latestAllowedVersion() - ), - request.callback(), - request.destination(), - request.createdTimeMs(), - currentTimeMs, - true, - null, - authenticationException, - null - ) - ); - - return false; - } - // enqueue the request again if the node isn't ready yet. The request will be handled in the next iteration // of the event loop log.debug("Node is not ready, handle the request in the next event loop: node={}, request={}", node, r); From 7df60c7a92b1d1003e683e8c2fd151891dd8f4d4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 10 Oct 2025 16:01:03 -0700 Subject: [PATCH 101/123] Refactored completeExceptionally to set the error _first_, then set the flag --- .../kafka/clients/consumer/internals/events/AsyncPollEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index e63f85ecc33f3..fb710c0223103 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -80,8 +80,8 @@ public void completeSuccessfully() { } public void completeExceptionally(KafkaException e) { + error = e; isComplete = true; - this.error = e; } @Override From 390ac0422716305544b1e4e45b4c0082f7b7f705 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 10 Oct 2025 19:11:30 -0700 Subject: [PATCH 102/123] Update AsyncKafkaConsumer.java --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 7aa6a47417e8b..36effad608181 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -909,6 +909,8 @@ public void checkInflightPoll(Timer timer) { throw errorOpt.get(); } } else if (!newlySubmittedEvent) { + timer.update(); + if (timer.isExpired()) { // The inflight event is expired... log.trace("Inflight event {} expired without completing, clearing", inflightPoll); From 477267511a7837a26928cdea2bd34de437d43750 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 12 Oct 2025 19:45:26 -0400 Subject: [PATCH 103/123] Remove group.id configuration from testConsumeUsingAssignWithNoAccess --- .../integration/kafka/api/AuthorizerIntegrationTest.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 7e17b15374638..f7d8ab2c99c51 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -1356,9 +1356,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRecords(producer, 1, tp) removeAllClientAcls() - val consumer = createConsumer() + // Remove the group.id configuration since this self-assigning partitions. + val consumer = createConsumer(configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) consumer.assign(java.util.List.of(tp)) - assertThrows(classOf[AuthorizationException], () => consumeRecords(consumer)) + assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) From eb77ad8982d0b43bd8793f2968967888a5745aac Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 12 Oct 2025 22:05:01 -0400 Subject: [PATCH 104/123] Added whitespace to README.md to force rebuild --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index f367d1b9bf5c2..a7b0a6b593933 100644 --- a/README.md +++ b/README.md @@ -1,3 +1,4 @@ +

From 51a04b86ed2decdca6d9b310e5a505f175955136 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 13 Oct 2025 22:01:36 -0400 Subject: [PATCH 105/123] Removing whitespace to kick off build --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index a7b0a6b593933..f367d1b9bf5c2 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,3 @@ -

From 61bca4c8a14b30d6cbaa8357c668f6aceacf3bbf Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 15 Oct 2025 12:47:51 -0400 Subject: [PATCH 106/123] Restoring ConsumerIntegrationTest timeout and fixing pre-assignment timeout issue --- .../consumer/ConsumerIntegrationTest.java | 1 + .../consumer/internals/AsyncKafkaConsumer.java | 17 +++++++++++++---- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java index c202bd222af22..220866c240f4a 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -123,6 +123,7 @@ public void onPartitionsAssigned(Collection partitions) { }); TestUtils.waitForCondition(() -> consumer.poll(Duration.ofSeconds(1)).count() == 1, + 5000, "failed to poll data"); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 36effad608181..d5568b041cfb1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1835,10 +1835,19 @@ private Fetch pollForFetches(Timer timer) { // in poll for an unnecessarily long amount of time if we are missing some positions since the offset // lookup may be backing off after a failure. if (pollTimeout > retryBackoffMs) { - for (TopicPartition tp : subscriptions.assignedPartitions()) { - if (!subscriptions.hasValidPosition(tp)) { - pollTimeout = retryBackoffMs; - break; + Set partitions = subscriptions.assignedPartitions(); + + if (partitions.isEmpty()) { + // If there aren't any assigned partitions, this could mean that this consumer's group membership + // has not been established or assignments have been removed and not yet reassigned. In either case, + // reduce the poll time for the fetch buffer wait. + pollTimeout = retryBackoffMs; + } else { + for (TopicPartition tp : partitions) { + if (!subscriptions.hasValidPosition(tp)) { + pollTimeout = retryBackoffMs; + break; + } } } } From e3e35bcc99f695aab6e6f84d7538c082fed75fa2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 17 Oct 2025 00:17:32 -0400 Subject: [PATCH 107/123] Changed incorrect references to PollApplicationEvent to AsyncPollEvent --- .../consumer/internals/AbstractHeartbeatRequestManager.java | 3 ++- .../internals/StreamsGroupHeartbeatRequestManager.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java index 3998d672006a3..11077d88c93f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager; @@ -240,7 +241,7 @@ public PollResult pollOnClose(long currentTimeMs) { * are sent, so blocking for longer than the heartbeat interval might mean the application thread is not * responsive to changes. * - *

Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure + *

Similarly, we may have to unblock the application thread to send a {@link AsyncPollEvent} to make sure * our poll timer will not expire while we are polling. * *

In the event that heartbeats are currently being skipped, this still returns the next heartbeat diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java index ceeeb6c191607..55b87ea5286bd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager; @@ -425,7 +426,7 @@ public StreamsMembershipManager membershipManager() { * are sent, so blocking for longer than the heartbeat interval might mean the application thread is not * responsive to changes. * - *

Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure + *

Similarly, we may have to unblock the application thread to send a {@link AsyncPollEvent} to make sure * our poll timer will not expire while we are polling. * *

In the event that heartbeats are currently being skipped, this still returns the next heartbeat From 8e370615b8a2c1f8aead1dafe1766818971ab679 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 17 Oct 2025 00:20:31 -0400 Subject: [PATCH 108/123] Reverted change in PlaintextConsumerTest --- .../apache/kafka/clients/consumer/PlaintextConsumerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index d4375898ddb25..bd92f0c56851e 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -1681,7 +1681,7 @@ private ConsumerRecords awaitNonEmptyRecords( AtomicReference> result = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - var polledRecords = consumer.poll(Duration.ofSeconds(1)); + var polledRecords = consumer.poll(Duration.ofSeconds(10)); boolean hasRecords = !polledRecords.isEmpty(); if (hasRecords) { result.set(polledRecords); From ae8907500ea6ae78e72602806aedb520e1418fc6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 17 Oct 2025 09:16:15 -0400 Subject: [PATCH 109/123] Removed unnecessary import --- .../consumer/internals/events/ApplicationEventProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 8900a306a69e0..43e84ecb646e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -23,7 +23,6 @@ import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMembershipManager; -import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; From 9629d52c2bd24daf9a3c369565a87309de67f442 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 18 Oct 2025 11:09:08 -0400 Subject: [PATCH 110/123] Update KafkaConsumerTest.testCurrentLag() to move the poll() to inside the waitForCondition loop --- .../clients/consumer/KafkaConsumerTest.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 77fd8dd1eb289..4207421b011b5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2685,12 +2685,13 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept } // poll once again, which should send the list-offset request consumer.seek(tp0, 50L); - consumer.poll(Duration.ofMillis(0)); - // requests: list-offset - TestUtils.waitForCondition( - () -> requestGenerated(client, ApiKeys.LIST_OFFSETS), - "No list-offset request sent" - ); + // requests: list-offset, fetch + TestUtils.waitForCondition(() -> { + consumer.poll(Duration.ofMillis(0)); + boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); + boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); + return hasListOffsetRequest && hasFetchRequest; + }, () -> "No list-offset & fetch request sent"); // no error for no end offset (so unknown lag) assertEquals(OptionalLong.empty(), consumer.currentLag(tp0)); @@ -2699,12 +2700,6 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // and hence next call would return correct lag result ClientRequest listOffsetRequest = findRequest(client, ApiKeys.LIST_OFFSETS); client.respondToRequest(listOffsetRequest, listOffsetsResponse(Map.of(tp0, 90L))); - consumer.poll(Duration.ofMillis(0)); - // requests: fetch - TestUtils.waitForCondition( - () -> requestGenerated(client, ApiKeys.FETCH), - "No fetch request sent" - ); // For AsyncKafkaConsumer, subscription state is updated in background, so the result will eventually be updated. TestUtils.waitForCondition(() -> { From ed5b1634052ba8e1fdba18725896abb9fea81c57 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 18 Oct 2025 11:22:59 -0400 Subject: [PATCH 111/123] Re-added call to poll() in testCurrentLag() --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 4207421b011b5..0ef68807599c5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2700,6 +2700,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept // and hence next call would return correct lag result ClientRequest listOffsetRequest = findRequest(client, ApiKeys.LIST_OFFSETS); client.respondToRequest(listOffsetRequest, listOffsetsResponse(Map.of(tp0, 90L))); + consumer.poll(Duration.ofMillis(0)); // For AsyncKafkaConsumer, subscription state is updated in background, so the result will eventually be updated. TestUtils.waitForCondition(() -> { From ac1b65132af4ab692873498851c55a93d0fb8d81 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 18 Oct 2025 11:24:41 -0400 Subject: [PATCH 112/123] Update KafkaConsumerTest.java --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 0ef68807599c5..f73e22ed69f08 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2691,7 +2691,7 @@ public void testCurrentLag(GroupProtocol groupProtocol) throws InterruptedExcept boolean hasListOffsetRequest = requestGenerated(client, ApiKeys.LIST_OFFSETS); boolean hasFetchRequest = requestGenerated(client, ApiKeys.FETCH); return hasListOffsetRequest && hasFetchRequest; - }, () -> "No list-offset & fetch request sent"); + }, "No list-offset & fetch request sent"); // no error for no end offset (so unknown lag) assertEquals(OptionalLong.empty(), consumer.currentLag(tp0)); From 387002335229ef162f75551428a84015cae66175 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 22 Oct 2025 10:35:51 -0700 Subject: [PATCH 113/123] Removing unused size() method call from BackgroundEventHandler and OffsetCommitCallbackInvoker --- .../internals/OffsetCommitCallbackInvoker.java | 10 ---------- .../internals/events/BackgroundEventHandler.java | 10 ---------- 2 files changed, 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java index 77b8f6f81fe72..3c1ebc6dec3a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java @@ -50,16 +50,6 @@ public void enqueueInterceptorInvocation(final Mappause - * itself to return to the application thread for processing. - * - * @return Current size of queue - */ - public int size() { - return callbackQueue.size(); - } - public void enqueueUserCallbackInvocation(final OffsetCommitCallback callback, final Map offsets, final Exception exception) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 935220bf70d62..3e83908f3df42 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -45,16 +45,6 @@ public BackgroundEventHandler(final BlockingQueue backgroundEve this.asyncConsumerMetrics = asyncConsumerMetrics; } - /** - * Returns the current size of the queue. Used by the background thread to determine if it needs to pause - * itself to return to the application thread for processing. - * - * @return Current size of queue - */ - public int size() { - return backgroundEventQueue.size(); - } - /** * Add a {@link BackgroundEvent} to the handler. * From 965570bf1077077f601d49e08f5f8ecd2b87ffbe Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 23 Oct 2025 16:43:07 -0700 Subject: [PATCH 114/123] Handle previous inflight poll events in AsyncKafkaConsumer Adds logic to distinguish the first pass of the poll loop and process any inflight poll events from previous invocations. Ensures completed or expired inflight events are cleared and errors are thrown immediately, improving event handling and error propagation. --- .../internals/AsyncKafkaConsumer.java | 39 ++++++++++++++++--- 1 file changed, 33 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index bff953815cdff..18e8b8f6db8e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -834,6 +834,10 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } + // This distinguishes the first pass of the inner do/while loop from subsequent passes for the + // inflight poll event logic. + boolean firstPass = true; + do { // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling @@ -841,7 +845,8 @@ public ConsumerRecords poll(final Duration timeout) { // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); - checkInflightPoll(timer); + checkInflightPoll(timer, firstPass); + firstPass = false; final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { // before returning the fetched records, we can send off the next round of fetches @@ -875,7 +880,28 @@ public ConsumerRecords poll(final Duration timeout) { * is made during each invocation to see if the inflight event has completed. If it has, it will be * processed accordingly. */ - public void checkInflightPoll(Timer timer) { + public void checkInflightPoll(Timer timer, boolean firstPass) { + // Handle the case where there's an inflight poll from the *previous* invocation of AsyncKafkaConsumer.poll(). + if (firstPass && inflightPoll != null) { + if (inflightPoll.isComplete()) { + // If the previous inflight event is complete, check if it resulted in an error. + log.trace("Previous inflight event {} completed, clearing", inflightPoll); + Optional errorOpt = inflightPoll.error(); + inflightPoll = null; + + // If there is an error, throw it without delay. If it completed without error, then proceed as usual + // and a new event will be enqueued below. + if (errorOpt.isPresent()) { + throw errorOpt.get(); + } + } else if (inflightPoll.deadlineMs() < time.milliseconds()) { + // The previous inflight is not complete, but it has expired. There's no result to check, so just + // clear out the event reference and a new event will be enqueued below. + log.trace("Previous inflight event {} incomplete and expired, clearing", inflightPoll); + inflightPoll = null; + } + } + boolean newlySubmittedEvent = false; if (inflightPoll == null) { @@ -884,7 +910,7 @@ public void checkInflightPoll(Timer timer) { if (log.isTraceEnabled()) { log.trace( - "Submitting new inflight event {} with {} remaining on timer", + "Inflight event {} submitted with {} remaining on timer", inflightPoll, timer.remainingMs() ); @@ -928,9 +954,10 @@ public void checkInflightPoll(Timer timer) { } } } catch (Throwable t) { - // If an exception is hit, bubble it up to the user but make sure to clear out the inflight request - // because the error effectively renders it complete. - log.debug("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); + // If an exception is hit in the offset commit callbacks, the background events, or the event result, + // bubble it up to the user but make sure to clear out the inflight request because the error effectively + // renders it complete. + log.trace("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); inflightPoll = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); } From cdaa914db0a2b70c388d4ec6a20997c6088d521d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 24 Oct 2025 21:47:43 -0700 Subject: [PATCH 115/123] Fix race condition in async poll event position validation Introduces isValidatePositionsComplete flag to AsyncPollEvent and ensures the application thread waits for background thread to complete position validation before collecting fetch data. Updates event handling logic in AsyncKafkaConsumer and ApplicationEventProcessor to prevent concurrent updates to SubscriptionState.position(), addressing a potential race condition. --- .../internals/AsyncKafkaConsumer.java | 44 ++++++++++++++----- .../events/ApplicationEventProcessor.java | 1 + .../internals/events/AsyncPollEvent.java | 12 ++++- 3 files changed, 46 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 18e8b8f6db8e1..c149054f22807 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -884,19 +884,33 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { // Handle the case where there's an inflight poll from the *previous* invocation of AsyncKafkaConsumer.poll(). if (firstPass && inflightPoll != null) { if (inflightPoll.isComplete()) { - // If the previous inflight event is complete, check if it resulted in an error. - log.trace("Previous inflight event {} completed, clearing", inflightPoll); Optional errorOpt = inflightPoll.error(); - inflightPoll = null; - // If there is an error, throw it without delay. If it completed without error, then proceed as usual - // and a new event will be enqueued below. if (errorOpt.isPresent()) { + // If the previous inflight event is complete, check if it resulted in an error. If there was + // an error, throw it without delay. + log.trace("Previous inflight event {} completed with an error, clearing", inflightPoll); + inflightPoll = null; throw errorOpt.get(); + } else { + if (fetchBuffer.isEmpty()) { + // If it completed without error, but without populating the fetch buffer, clear the event + // so that a new event will be enqueued below. + log.trace("Previous inflight event {} completed without filling the buffer, clearing", inflightPoll); + inflightPoll = null; + } else { + // However, if the event completed, and it populated the buffer, *don't* create a new event. + // This is to prevent an edge case of starvation when poll() is called with a timeout of 0. + // If a new event was created on *every* poll, each time the event would have to complete the + // validate positions stage before the data in the fetch buffer is used. Because there is + // no blocking, and effectively a 0 wait, the data in the fetch buffer is continuously ignored + // leading to no data ever being returned from poll(). + log.trace("Previous inflight event {} completed and filled the buffer, not clearing", inflightPoll); + } } - } else if (inflightPoll.deadlineMs() < time.milliseconds()) { - // The previous inflight is not complete, but it has expired. There's no result to check, so just - // clear out the event reference and a new event will be enqueued below. + } else if (time.milliseconds() > inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { + // Although the previous event passed the validate positions stage, it has expired. Since there's no + // result to check, clear out the event so that a new one will be enqueued below. log.trace("Previous inflight event {} incomplete and expired, clearing", inflightPoll); inflightPoll = null; } @@ -939,8 +953,8 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { } else if (!newlySubmittedEvent) { timer.update(); - if (timer.isExpired()) { - // The inflight event is expired... + if (timer.isExpired() && inflightPoll.isValidatePositionsComplete()) { + // The inflight event inflight validated positions, but it has expired. log.trace("Inflight event {} expired without completing, clearing", inflightPoll); inflightPoll = null; } else { @@ -1909,6 +1923,16 @@ private Fetch pollForFetches(Timer timer) { * for returning. */ private Fetch collectFetch() { + // With the non-blocking async poll, it's critical that the application thread wait until the background + // thread has completed the stage of validating positions. This prevents a race condition where both + // threads may attempt to update the SubscriptionState.position() for a given partition. So if the background + // thread has not completed that stage for the inflight event, don't attempt to collect data from the fetch + // buffer. If the inflight event was nulled out by checkInflightPoll(), that implies that it is safe to + // attempt to collect data from the fetch buffer. + if (inflightPoll != null && !inflightPoll.isValidatePositionsComplete()) { + return Fetch.empty(); + } + return fetchCollector.collectFetch(fetchBuffer); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 43e84ecb646e2..41e88ccec3d58 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -736,6 +736,7 @@ private void process(final AsyncPollEvent event) { } CompletableFuture updatePositionsFuture = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); + event.markValidatePositionsComplete(); updatePositionsFuture.whenComplete((__, updatePositionsError) -> { if (maybeCompleteAsyncPollEventExceptionally(event, updatePositionsError)) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index fb710c0223103..0065a57a39e09 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -45,6 +45,7 @@ public class AsyncPollEvent extends ApplicationEvent implements MetadataErrorNot private final long pollTimeMs; private volatile KafkaException error; private volatile boolean isComplete; + private volatile boolean isValidatePositionsComplete; /** * Creates a new event to signify a multi-stage processing of {@link Consumer#poll(Duration)} logic. @@ -71,6 +72,14 @@ public Optional error() { return Optional.ofNullable(error); } + public boolean isValidatePositionsComplete() { + return isValidatePositionsComplete; + } + + public void markValidatePositionsComplete() { + this.isValidatePositionsComplete = true; + } + public boolean isComplete() { return isComplete; } @@ -95,6 +104,7 @@ protected String toStringBase() { ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", error=" + error + - ", isComplete=" + isComplete; + ", isComplete=" + isComplete + + ", isValidatePositionsComplete=" + isValidatePositionsComplete; } } From 3ec3dd13e7f1b9350a84044f52cec793713bc2fe Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 25 Oct 2025 05:59:46 -0700 Subject: [PATCH 116/123] Suppress CyclomaticComplexity in checkInflightPoll :( Added @SuppressWarnings annotation for 'CyclomaticComplexity' to the checkInflightPoll method to suppress static analysis warnings about method complexity. --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index c149054f22807..4c58ab6aa7874 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -880,6 +880,7 @@ public ConsumerRecords poll(final Duration timeout) { * is made during each invocation to see if the inflight event has completed. If it has, it will be * processed accordingly. */ + @SuppressWarnings({"CyclomaticComplexity"}) public void checkInflightPoll(Timer timer, boolean firstPass) { // Handle the case where there's an inflight poll from the *previous* invocation of AsyncKafkaConsumer.poll(). if (firstPass && inflightPoll != null) { From a549a765b5b5ed0c4e63b4f2cb2e287fe38c7dbd Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 27 Oct 2025 10:20:26 -0700 Subject: [PATCH 117/123] Wake up FetchBuffer when no fetch requests are sent to avoid unnecessary waiting --- .../clients/consumer/internals/FetchRequestManager.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index c52b5453e21d9..e7139657d5abf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -145,6 +145,14 @@ private PollResult pollInternal(FetchRequestPreparer fetchRequestPreparer, try { Map fetchRequests = fetchRequestPreparer.prepare(); + if (fetchRequests.isEmpty()) { + // If there's nothing to fetch, wake up the FetchBuffer so it doesn't needlessly wait for a wakeup + // that won't come until the data in the fetch buffer is consumed. + fetchBuffer.wakeup(); + pendingFetchRequestFuture.complete(null); + return PollResult.EMPTY; + } + List requests = fetchRequests.entrySet().stream().map(entry -> { final Node fetchTarget = entry.getKey(); final FetchSessionHandler.FetchRequestData data = entry.getValue(); From ddfbca3a64bdb1f86995373bbd5b0abf1093b890 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 28 Oct 2025 14:14:43 -0700 Subject: [PATCH 118/123] Updated expiration logic for inflight poll events --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 4c58ab6aa7874..17f652eae0c9a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -909,7 +909,7 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { log.trace("Previous inflight event {} completed and filled the buffer, not clearing", inflightPoll); } } - } else if (time.milliseconds() > inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { + } else if (time.milliseconds() >= inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { // Although the previous event passed the validate positions stage, it has expired. Since there's no // result to check, clear out the event so that a new one will be enqueued below. log.trace("Previous inflight event {} incomplete and expired, clearing", inflightPoll); @@ -952,9 +952,7 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { throw errorOpt.get(); } } else if (!newlySubmittedEvent) { - timer.update(); - - if (timer.isExpired() && inflightPoll.isValidatePositionsComplete()) { + if (time.milliseconds() >= inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { // The inflight event inflight validated positions, but it has expired. log.trace("Inflight event {} expired without completing, clearing", inflightPoll); inflightPoll = null; From 5946e7be0a2c030351bfa36b8de0695d71e10665 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 31 Oct 2025 11:32:04 -0700 Subject: [PATCH 119/123] Refactor checkInflightPoll to be less complex --- .../internals/AsyncKafkaConsumer.java | 135 +++++++++--------- .../internals/events/AsyncPollEvent.java | 5 + 2 files changed, 71 insertions(+), 69 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 17f652eae0c9a..4b2e7d1e97113 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -880,41 +880,11 @@ public ConsumerRecords poll(final Duration timeout) { * is made during each invocation to see if the inflight event has completed. If it has, it will be * processed accordingly. */ - @SuppressWarnings({"CyclomaticComplexity"}) - public void checkInflightPoll(Timer timer, boolean firstPass) { - // Handle the case where there's an inflight poll from the *previous* invocation of AsyncKafkaConsumer.poll(). + private void checkInflightPoll(Timer timer, boolean firstPass) { if (firstPass && inflightPoll != null) { - if (inflightPoll.isComplete()) { - Optional errorOpt = inflightPoll.error(); - - if (errorOpt.isPresent()) { - // If the previous inflight event is complete, check if it resulted in an error. If there was - // an error, throw it without delay. - log.trace("Previous inflight event {} completed with an error, clearing", inflightPoll); - inflightPoll = null; - throw errorOpt.get(); - } else { - if (fetchBuffer.isEmpty()) { - // If it completed without error, but without populating the fetch buffer, clear the event - // so that a new event will be enqueued below. - log.trace("Previous inflight event {} completed without filling the buffer, clearing", inflightPoll); - inflightPoll = null; - } else { - // However, if the event completed, and it populated the buffer, *don't* create a new event. - // This is to prevent an edge case of starvation when poll() is called with a timeout of 0. - // If a new event was created on *every* poll, each time the event would have to complete the - // validate positions stage before the data in the fetch buffer is used. Because there is - // no blocking, and effectively a 0 wait, the data in the fetch buffer is continuously ignored - // leading to no data ever being returned from poll(). - log.trace("Previous inflight event {} completed and filled the buffer, not clearing", inflightPoll); - } - } - } else if (time.milliseconds() >= inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { - // Although the previous event passed the validate positions stage, it has expired. Since there's no - // result to check, clear out the event so that a new one will be enqueued below. - log.trace("Previous inflight event {} incomplete and expired, clearing", inflightPoll); - inflightPoll = null; - } + // Handle the case where there's a remaining inflight poll from the *previous* invocation + // of AsyncKafkaConsumer.poll(). + maybeClearPreviousInflightPoll(); } boolean newlySubmittedEvent = false; @@ -922,15 +892,7 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { if (inflightPoll == null) { inflightPoll = new AsyncPollEvent(calculateDeadlineMs(timer), time.milliseconds()); newlySubmittedEvent = true; - - if (log.isTraceEnabled()) { - log.trace( - "Inflight event {} submitted with {} remaining on timer", - inflightPoll, - timer.remainingMs() - ); - } - + log.trace("Inflight event {} submitted", inflightPoll); applicationEventHandler.add(inflightPoll); } @@ -939,40 +901,75 @@ public void checkInflightPoll(Timer timer, boolean firstPass) { // the inflight event is cleared. offsetCommitCallbackInvoker.executeCallbacks(); processBackgroundEvents(); + } catch (Throwable t) { + // If an exception was thrown during execution of offset commit callbacks or background events, + // bubble it up to the user but make sure to clear out the inflight request because the error effectively + // renders it complete. + log.trace("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); + inflightPoll = null; + throw ConsumerUtils.maybeWrapAsKafkaException(t); + } - if (inflightPoll.isComplete()) { - Optional errorOpt = inflightPoll.error(); + if (inflightPoll != null) { + maybeClearCurrentInflightPoll(newlySubmittedEvent); + } + } - // The async poll event has completed, either successfully or not. In either case, clear out the - // inflight request. - log.trace("Inflight event {} completed, clearing", inflightPoll); - inflightPoll = null; + private void maybeClearPreviousInflightPoll() { + if (inflightPoll.isComplete()) { + Optional errorOpt = inflightPoll.error(); - if (errorOpt.isPresent()) { - throw errorOpt.get(); - } - } else if (!newlySubmittedEvent) { - if (time.milliseconds() >= inflightPoll.deadlineMs() && inflightPoll.isValidatePositionsComplete()) { - // The inflight event inflight validated positions, but it has expired. - log.trace("Inflight event {} expired without completing, clearing", inflightPoll); + if (errorOpt.isPresent()) { + // If the previous inflight event is complete, check if it resulted in an error. If there was + // an error, throw it without delay. + KafkaException error = errorOpt.get(); + log.trace("Previous inflight event {} completed with an error ({}), clearing", inflightPoll, error); + inflightPoll = null; + throw error; + } else { + // Successful case... + if (fetchBuffer.isEmpty()) { + // If it completed without error, but without populating the fetch buffer, clear the event + // so that a new event will be enqueued below. + log.trace("Previous inflight event {} completed without filling the buffer, clearing", inflightPoll); inflightPoll = null; } else { - if (log.isTraceEnabled()) { - log.trace( - "Inflight event {} is incomplete with {} remaining on timer", - inflightPoll, - timer.remainingMs() - ); - } + // However, if the event completed, and it populated the buffer, *don't* create a new event. + // This is to prevent an edge case of starvation when poll() is called with a timeout of 0. + // If a new event was created on *every* poll, each time the event would have to complete the + // validate positions stage before the data in the fetch buffer is used. Because there is + // no blocking, and effectively a 0 wait, the data in the fetch buffer is continuously ignored + // leading to no data ever being returned from poll(). + log.trace("Previous inflight event {} completed and filled the buffer, not clearing", inflightPoll); } } - } catch (Throwable t) { - // If an exception is hit in the offset commit callbacks, the background events, or the event result, - // bubble it up to the user but make sure to clear out the inflight request because the error effectively - // renders it complete. - log.trace("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); + } else if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) { + // The inflight event inflight validated positions, but it has expired. + log.trace("Previous inflight event {} expired without completing, clearing", inflightPoll); inflightPoll = null; - throw ConsumerUtils.maybeWrapAsKafkaException(t); + } + } + + private void maybeClearCurrentInflightPoll(boolean newlySubmittedEvent) { + if (inflightPoll.isComplete()) { + Optional errorOpt = inflightPoll.error(); + + if (errorOpt.isPresent()) { + // If the inflight event completed with an error, throw it without delay. + KafkaException error = errorOpt.get(); + log.trace("Inflight event {} completed with an error ({}), clearing", inflightPoll, error); + inflightPoll = null; + throw error; + } else { + log.trace("Inflight event {} completed without error, clearing", inflightPoll); + inflightPoll = null; + } + } else if (!newlySubmittedEvent) { + if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) { + // The inflight event inflight validated positions, but it has expired. + log.trace("Inflight event {} expired without completing, clearing", inflightPoll); + inflightPoll = null; + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java index 0065a57a39e09..068193ca498d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncPollEvent.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.Time; import java.time.Duration; import java.util.Optional; @@ -72,6 +73,10 @@ public Optional error() { return Optional.ofNullable(error); } + public boolean isExpired(Time time) { + return time.milliseconds() >= deadlineMs(); + } + public boolean isValidatePositionsComplete() { return isValidatePositionsComplete; } From 09070137b698e39403f01953312448b08366641f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 31 Oct 2025 11:32:42 -0700 Subject: [PATCH 120/123] Update tests and migrate some from AsyncKafkaConsumerTest to ApplicationEventProcessorTest --- .../internals/AsyncKafkaConsumerTest.java | 58 ---------------- .../events/ApplicationEventProcessorTest.java | 67 +++++++++++++++++++ 2 files changed, 67 insertions(+), 58 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 727facf28f362..6517e25ff6baa 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; import org.apache.kafka.clients.consumer.internals.events.CommitEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; @@ -56,7 +55,6 @@ import org.apache.kafka.clients.consumer.internals.events.TopicRe2JPatternSubscriptionChangeEvent; import org.apache.kafka.clients.consumer.internals.events.TopicSubscriptionChangeEvent; import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent; -import org.apache.kafka.clients.consumer.internals.events.UpdatePatternSubscriptionEvent; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -417,7 +415,6 @@ public void testWakeupBeforeCallingPoll() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -439,7 +436,6 @@ public void testWakeupAfterEmptyFetch() { consumer.wakeup(); return Fetch.empty(); }).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -463,7 +459,6 @@ public void testWakeupAfterNonEmptyFetch() { consumer.wakeup(); return Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), "")); }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -482,7 +477,6 @@ public void testCommitInRebalanceCallback() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); @@ -522,7 +516,6 @@ public void testClearWakeupTriggerAfterPoll() { ); doReturn(Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), ""))) .when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -668,7 +661,6 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { MockCommitCallback callback = new MockCommitCallback(); completeCommitAsyncApplicationEventSuccessfully(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAssignmentChangeEventSuccessfully(); consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); @@ -1195,19 +1187,6 @@ public void testNoInterceptorCommitAsyncFailed() { assertEquals(0, MockConsumerInterceptor.ON_COMMIT_COUNT.get()); } - @Test - public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { - consumer = newConsumer(); - testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); - } - - @Test - public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { - // Create consumer without group id so committed offsets are not used for updating positions - consumer = newConsumerWithoutGroupId(); - testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); - } - @Test public void testSubscribeGeneratesEvent() { consumer = newConsumer(); @@ -1482,7 +1461,6 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); } @@ -1642,7 +1620,6 @@ public void testEnsurePollEventSentOnConsumerPoll() { doAnswer(invocation -> Fetch.forPartition(tp, records, true, new OffsetAndMetadata(3, Optional.of(0), ""))) .when(fetchCollector) .collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); @@ -1657,17 +1634,6 @@ private Properties requiredConsumerConfigAndGroupId(final String groupId) { return props; } - private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { - completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); - - completeAssignmentChangeEventSuccessfully(); - consumer.assign(singleton(new TopicPartition("t1", 1))); - completeAsyncPollEventSuccessfully(); - consumer.poll(Duration.ZERO); - } - @Test public void testLongPollWaitIsLimited() { consumer = newConsumer(); @@ -1696,7 +1662,6 @@ public void testLongPollWaitIsLimited() { }).doAnswer(invocation -> Fetch.forPartition(tp, records, true, nextOffsetAndMetadata) ).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAsyncPollEventSuccessfully(); // And then poll for up to 10000ms, which should return 2 records without timing out @@ -1794,7 +1759,6 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -1833,7 +1797,6 @@ void testReaperInvokedInPoll() { doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList("topic")); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAsyncPollEventSuccessfully(); consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); @@ -1886,27 +1849,6 @@ public void testSeekToEnd() { assertEquals(AutoOffsetResetStrategy.LATEST, resetOffsetEvent.offsetResetStrategy()); } - @Test - public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { - consumer = newConsumer(); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - completeAssignmentChangeEventSuccessfully(); - completeTopicPatternSubscriptionChangeEventSuccessfully(); - completeUnsubscribeApplicationEventSuccessfully(); - - consumer.assign(singleton(new TopicPartition("topic1", 0))); - completeAsyncPollEventSuccessfully(); - consumer.poll(Duration.ZERO); - verify(applicationEventHandler, never()).addAndGet(any(UpdatePatternSubscriptionEvent.class)); - - consumer.unsubscribe(); - - consumer.subscribe(Pattern.compile("t*")); - consumer.poll(Duration.ZERO); - } - @Test public void testSubscribeToRe2JPatternValidation() { consumer = newConsumer(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index fe294f030e690..e85409b391e93 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -49,6 +49,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.verification.VerificationMode; import java.util.Collection; import java.util.Collections; @@ -65,6 +66,7 @@ import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -660,6 +662,71 @@ public void testStreamsOnAllTasksLostCallbackCompletedEventWithoutStreamsMembers } } + @Test + public void testUpdatePatternSubscriptionInvokedWhenMetadataUpdated() { + when(subscriptionState.hasPatternSubscription()).thenReturn(true); + when(subscriptionState.matchesSubscribedPattern(any(String.class))).thenReturn(true); + when(metadata.updateVersion()).thenReturn(1, 2); + testUpdatePatternSubscription(times(1)); + } + + @Test + public void testUpdatePatternSubscriptionNotInvokedWhenNotUsingPatternSubscription() { + when(subscriptionState.hasPatternSubscription()).thenReturn(false); + when(metadata.updateVersion()).thenReturn(1, 2); + testUpdatePatternSubscription(never()); + } + + @Test + public void testUpdatePatternSubscriptionNotInvokedWhenMetadataNotUpdated() { + when(subscriptionState.hasPatternSubscription()).thenReturn(true); + when(subscriptionState.matchesSubscribedPattern(any(String.class))).thenReturn(true); + when(metadata.updateVersion()).thenReturn(1, 1); + testUpdatePatternSubscription(never()); + } + + private void testUpdatePatternSubscription(VerificationMode verificationMode) { + String topic = "test-topic"; + Cluster cluster = mock(Cluster.class); + + when(metadata.fetch()).thenReturn(cluster); + when(cluster.topics()).thenReturn(Set.of(topic)); + + when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); + when(offsetsRequestManager.updateFetchPositions(anyLong())).thenReturn(CompletableFuture.completedFuture(true)); + + setupProcessor(true); + processor.process(new AsyncPollEvent(110, 100)); + verify(membershipManager, verificationMode).onSubscriptionUpdated(); + } + + @Test + public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { + setupProcessor(true); + testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); + } + + @Test + public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { + // Create consumer without group id so committed offsets are not used for updating positions + setupProcessor(false); + testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); + } + + private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { + when(offsetsRequestManager.updateFetchPositions(anyLong())).thenReturn( + CompletableFuture.failedFuture(new Throwable("Intentional failure")) + ); + when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); + + // Verify that the poll completes even when the update fetch positions throws an error. + AsyncPollEvent event = new AsyncPollEvent(110, 100); + processor.process(event); + verify(offsetsRequestManager).updateFetchPositions(anyLong()); + assertTrue(event.isComplete()); + assertFalse(event.error().isEmpty()); + } + private List mockCommitResults() { return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class)); } From a2aa8b4b1263b3b78126af37473e2468ae85190e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 3 Nov 2025 10:09:03 -0800 Subject: [PATCH 121/123] Fixed comments --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 4b2e7d1e97113..ea1a5e490dd6b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -944,7 +944,7 @@ private void maybeClearPreviousInflightPoll() { } } } else if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) { - // The inflight event inflight validated positions, but it has expired. + // The inflight event validated positions, but it has expired. log.trace("Previous inflight event {} expired without completing, clearing", inflightPoll); inflightPoll = null; } @@ -966,7 +966,7 @@ private void maybeClearCurrentInflightPoll(boolean newlySubmittedEvent) { } } else if (!newlySubmittedEvent) { if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) { - // The inflight event inflight validated positions, but it has expired. + // The inflight event validated positions, but it has expired. log.trace("Inflight event {} expired without completing, clearing", inflightPoll); inflightPoll = null; } From ca3ccdb7d3c14e469b3234aa68344efc9cad7a69 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 3 Nov 2025 10:09:42 -0800 Subject: [PATCH 122/123] Verify when matchesSubscriptionPattern is called --- .../consumer/internals/events/ApplicationEventProcessorTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index e85409b391e93..61b53f9c19d57 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -697,6 +697,7 @@ private void testUpdatePatternSubscription(VerificationMode verificationMode) { setupProcessor(true); processor.process(new AsyncPollEvent(110, 100)); + verify(subscriptionState, verificationMode).matchesSubscribedPattern(topic); verify(membershipManager, verificationMode).onSubscriptionUpdated(); } From 7ef75a1f36f270b9524e2dbb32a841b0256c1f15 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 3 Nov 2025 11:49:22 -0800 Subject: [PATCH 123/123] Updating the Timer after running any callbacks and/or background events --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index ea1a5e490dd6b..e806b21c46558 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -908,6 +908,8 @@ private void checkInflightPoll(Timer timer, boolean firstPass) { log.trace("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t)); inflightPoll = null; throw ConsumerUtils.maybeWrapAsKafkaException(t); + } finally { + timer.update(); } if (inflightPoll != null) {