From 34932e22224f7c108959da155c7069d141c56972 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 9 Sep 2025 21:58:16 -0700 Subject: [PATCH 01/37] [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 02/37] [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 03/37] 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 04/37] 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 05/37] 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 06/37] 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 07/37] 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 08/37] 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 09/37] 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 10/37] 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 11/37] 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 12/37] 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 13/37] 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 14/37] 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 15/37] 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 16/37] 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 17/37] 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 18/37] 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 19/37] 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 20/37] 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 21/37] 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 22/37] 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 23/37] 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 24/37] 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 25/37] 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 26/37] 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 27/37] 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 28/37] 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 29/37] 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 30/37] 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 31/37] 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 32/37] 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 33/37] 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 34/37] 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 35/37] 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 36/37] 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 37/37] 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)) } /**