From 19f29cfd00416954d54cb4b3ecc7e4e6f441ed01 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 6 Aug 2025 14:03:51 -0700 Subject: [PATCH 01/31] WIP: Attempting to short-circuit event call in updateFetchPositions() --- .../consumer/internals/AsyncKafkaConsumer.java | 17 +++++++++++------ .../internals/OffsetsRequestManager.java | 7 +++++++ .../consumer/internals/RequestManagers.java | 5 ++++- .../internals/AsyncKafkaConsumerTest.java | 4 +++- .../internals/OffsetsRequestManagerTest.java | 4 ++++ .../consumer/internals/RequestManagersTest.java | 7 +++++-- 6 files changed, 34 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 1ca42dbc75c6c..582cac5964177 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 @@ -131,6 +131,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -361,7 +362,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { private Optional clientTelemetryReporter = Optional.empty(); // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates - private boolean cachedSubscriptionHasAllFetchPositions; + private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -492,7 +493,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metrics, offsetCommitCallbackInvoker, memberStateListener, - streamsRebalanceData + streamsRebalanceData, + cachedSubscriptionHasAllFetchPositions ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -678,7 +680,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metrics, offsetCommitCallbackInvoker, memberStateListener, - Optional.empty() + Optional.empty(), + cachedSubscriptionHasAllFetchPositions ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -1792,7 +1795,7 @@ private Fetch pollForFetches(Timer timer) { // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call // updateAssignmentMetadataIfNeeded before this method. - if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > retryBackoffMs) { + if (!cachedSubscriptionHasAllFetchPositions.get() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } @@ -1847,11 +1850,13 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { - cachedSubscriptionHasAllFetchPositions = false; + if (cachedSubscriptionHasAllFetchPositions.get()) + return true; + try { CheckAndUpdatePositionsEvent checkAndUpdatePositionsEvent = new CheckAndUpdatePositionsEvent(calculateDeadlineMs(timer)); wakeupTrigger.setActiveTask(checkAndUpdatePositionsEvent.future()); - cachedSubscriptionHasAllFetchPositions = applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent); + applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent); } catch (TimeoutException e) { return false; } finally { 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..a88b3a98f89c2 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 @@ -53,6 +53,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; @@ -92,6 +93,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; + private final AtomicBoolean cachedSubscriptionHasAllFetchPositions; private final long defaultApiTimeoutMs; /** @@ -119,6 +121,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final CommitRequestManager commitRequestManager, + final AtomicBoolean cachedSubscriptionHasAllFetchPositions, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -146,6 +149,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); this.commitRequestManager = commitRequestManager; + this.cachedSubscriptionHasAllFetchPositions = cachedSubscriptionHasAllFetchPositions; } private static class PendingFetchCommittedRequest { @@ -243,8 +247,11 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (subscriptionState.hasAllFetchPositions()) { // All positions are already available + cachedSubscriptionHasAllFetchPositions.set(true); result.complete(true); return result; + } else { + cachedSubscriptionHasAllFetchPositions.set(false); } // Some positions are missing, so trigger requests to fetch offsets and update them. 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..27da44c58af55 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 @@ -35,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import static java.util.Objects.requireNonNull; @@ -167,7 +168,8 @@ public static Supplier supplier(final Time time, final Metrics metrics, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, - final Optional streamsRebalanceData + final Optional streamsRebalanceData, + final AtomicBoolean cachedSubscriptionHasAllFetchPositions ) { return new CachedSupplier<>() { @Override @@ -292,6 +294,7 @@ protected RequestManagers create() { apiVersions, networkClientDelegate, commitRequestManager, + cachedSubscriptionHasAllFetchPositions, logContext); return new RequestManagers( 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 16e4c9d4b3fde..3e17676a10e50 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 @@ -1355,6 +1355,7 @@ private MemberStateListener captureGroupMetadataUpdateListener(final MockedStati any(), any(), applicationThreadMemberStateListener.capture(), + any(), any() )); return applicationThreadMemberStateListener.getValue(); @@ -1426,7 +1427,8 @@ private Optional captureStreamRebalanceData(final MockedSt any(), any(), any(), - streamRebalanceData.capture() + streamRebalanceData.capture(), + any() )); return streamRebalanceData.getValue(); } 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..27a55b0c28488 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 @@ -62,6 +62,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -91,6 +92,7 @@ public class OffsetsRequestManagerTest { private final Time time = mock(Time.class); private ApiVersions apiVersions; private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); + private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); private static final String TEST_TOPIC = "t1"; private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1); private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2); @@ -118,6 +120,7 @@ public void setup() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, + cachedSubscriptionHasAllFetchPositions, logContext ); } @@ -807,6 +810,7 @@ public void testRemoteListOffsetsRequestTimeoutMs() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, + cachedSubscriptionHasAllFetchPositions, new LogContext() ); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 67628c513406a..4c1653167e5ed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -30,6 +30,7 @@ import java.util.Optional; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -66,7 +67,8 @@ public void testMemberStateListenerRegistered() { new Metrics(), mock(OffsetCommitCallbackInvoker.class), listener, - Optional.empty() + Optional.empty(), + new AtomicBoolean() ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -106,7 +108,8 @@ public void testStreamMemberStateListenerRegistered() { new Metrics(), mock(OffsetCommitCallbackInvoker.class), listener, - Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())) + Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), + new AtomicBoolean() ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From 2691b226cc585365518c39fa915287220313baff Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Aug 2025 11:25:37 -0700 Subject: [PATCH 02/31] Experiment: skip the call to updateFetchPositions by accessing shared, thread-safe state --- .../internals/AsyncKafkaConsumer.java | 32 +++++-- .../internals/OffsetsRequestManager.java | 50 ++++------- .../consumer/internals/RequestManagers.java | 6 +- .../internals/SharedOffsetsState.java | 88 +++++++++++++++++++ .../internals/AsyncKafkaConsumerTest.java | 6 +- .../internals/OffsetsRequestManagerTest.java | 8 +- .../internals/RequestManagersTest.java | 37 +++++--- 7 files changed, 164 insertions(+), 63 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.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 582cac5964177..4bd24c0e85b25 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 @@ -131,7 +131,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -361,8 +360,7 @@ 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 final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); + private final SharedOffsetsState sharedOffsetsState; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -478,6 +476,14 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); + this.sharedOffsetsState = new SharedOffsetsState( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + apiVersions + ); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -494,7 +500,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - cachedSubscriptionHasAllFetchPositions + sharedOffsetsState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -565,7 +571,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int requestTimeoutMs, int defaultApiTimeoutMs, String groupId, - boolean autoCommitEnabled) { + boolean autoCommitEnabled, + SharedOffsetsState sharedOffsetsState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -595,6 +602,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, kafkaConsumerMetrics ); + this.sharedOffsetsState = sharedOffsetsState; } AsyncKafkaConsumer(LogContext logContext, @@ -664,6 +672,14 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.sharedOffsetsState = new SharedOffsetsState( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + apiVersions + ); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, @@ -681,7 +697,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - cachedSubscriptionHasAllFetchPositions + sharedOffsetsState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -1795,7 +1811,7 @@ private Fetch pollForFetches(Timer timer) { // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call // updateAssignmentMetadataIfNeeded before this method. - if (!cachedSubscriptionHasAllFetchPositions.get() && pollTimeout > retryBackoffMs) { + if (!sharedOffsetsState.subscriptionHasAllFetchPositions() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } @@ -1850,7 +1866,7 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { - if (cachedSubscriptionHasAllFetchPositions.get()) + if (sharedOffsetsState.canSkipUpdateFetchPositions()) return true; try { 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 a88b3a98f89c2..dcb97c7cdf11f 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 @@ -53,9 +53,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -83,7 +81,6 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final ConsumerMetadata metadata; private final IsolationLevel isolationLevel; private final Logger log; - private final OffsetFetcherUtils offsetFetcherUtils; private final SubscriptionState subscriptionState; private final Set requestsToRetry; @@ -93,14 +90,8 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; - private final AtomicBoolean cachedSubscriptionHasAllFetchPositions; 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<>(); + private final SharedOffsetsState sharedOffsetsState; /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update @@ -115,13 +106,12 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final ConsumerMetadata metadata, final IsolationLevel isolationLevel, final Time time, - final long retryBackoffMs, final int requestTimeoutMs, final long defaultApiTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final CommitRequestManager commitRequestManager, - final AtomicBoolean cachedSubscriptionHasAllFetchPositions, + final SharedOffsetsState sharedOffsetsState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -142,14 +132,12 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; - this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState, - time, retryBackoffMs, apiVersions); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); this.commitRequestManager = commitRequestManager; - this.cachedSubscriptionHasAllFetchPositions = cachedSubscriptionHasAllFetchPositions; + this.sharedOffsetsState = sharedOffsetsState; } private static class PendingFetchCommittedRequest { @@ -198,7 +186,7 @@ public CompletableFuture> fetchO ListOffsetsRequestState listOffsetsRequestState = new ListOffsetsRequestState( timestampsToSearch, requireTimestamps, - offsetFetcherUtils, + sharedOffsetsState.offsetFetcherUtils(), isolationLevel); listOffsetsRequestState.globalResult.whenComplete((result, error) -> { metadata.clearTransientTopics(); @@ -247,11 +235,11 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (subscriptionState.hasAllFetchPositions()) { // All positions are already available - cachedSubscriptionHasAllFetchPositions.set(true); + sharedOffsetsState.setSubscriptionHasAllFetchPositions(true); result.complete(true); return result; } else { - cachedSubscriptionHasAllFetchPositions.set(false); + sharedOffsetsState.setSubscriptionHasAllFetchPositions(false); } // Some positions are missing, so trigger requests to fetch offsets and update them. @@ -270,9 +258,9 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { } private boolean maybeCompleteWithPreviousException(CompletableFuture result) { - Throwable cachedException = cachedUpdatePositionsException.getAndSet(null); - if (cachedException != null) { - result.completeExceptionally(cachedException); + Optional cachedException = sharedOffsetsState.getAndClearCachedUpdatePositionsException(); + if (cachedException.isPresent()) { + result.completeExceptionally(cachedException.get()); return true; } return false; @@ -325,7 +313,7 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - cachedUpdatePositionsException.set(error); + sharedOffsetsState.setCachedUpdatePositionsException(error); } }); } @@ -482,7 +470,7 @@ CompletableFuture resetPositionsIfNeeded() { Map partitionAutoOffsetResetStrategyMap; try { - partitionAutoOffsetResetStrategyMap = offsetFetcherUtils.getOffsetResetStrategyForPartitions(); + partitionAutoOffsetResetStrategyMap = sharedOffsetsState.offsetFetcherUtils().getOffsetResetStrategyForPartitions(); } catch (Exception e) { CompletableFuture result = new CompletableFuture<>(); result.completeExceptionally(e); @@ -508,7 +496,7 @@ CompletableFuture resetPositionsIfNeeded() { * next call to this function. */ void validatePositionsIfNeeded() { - Map partitionsToValidate = offsetFetcherUtils.getPartitionsToValidate(); + Map partitionsToValidate = sharedOffsetsState.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return; } @@ -575,7 +563,7 @@ private List buildListOffsetsRequests( if (error == null) { listOffsetsRequestState.fetchedOffsets.putAll(multiNodeResult.fetchedOffsets); listOffsetsRequestState.addPartitionsToRetry(multiNodeResult.partitionsToRetry); - offsetFetcherUtils.updateSubscriptionState(multiNodeResult.fetchedOffsets, + sharedOffsetsState.offsetFetcherUtils().updateSubscriptionState(multiNodeResult.fetchedOffsets, isolationLevel); if (listOffsetsRequestState.remainingToSearch.isEmpty()) { @@ -645,7 +633,7 @@ private CompletableFuture buildListOffsetRequestToNode( ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); try { - ListOffsetResult listOffsetResult = offsetFetcherUtils.handleListOffsetResponse(lor); + ListOffsetResult listOffsetResult = sharedOffsetsState.offsetFetcherUtils().handleListOffsetResponse(lor); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -687,7 +675,7 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( partialResult.whenComplete((result, error) -> { if (error == null) { - offsetFetcherUtils.onSuccessfulResponseForResettingPositions(result, + sharedOffsetsState.offsetFetcherUtils().onSuccessfulResponseForResettingPositions(result, partitionAutoOffsetResetStrategyMap); } else { RuntimeException e; @@ -697,7 +685,7 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( e = new RuntimeException("Unexpected failure in ListOffsets request for " + "resetting positions", error); } - offsetFetcherUtils.onFailedResponseForResettingPositions(resetTimestamps, e); + sharedOffsetsState.offsetFetcherUtils().onFailedResponseForResettingPositions(resetTimestamps, e); } if (expectedResponses.decrementAndGet() == 0) { globalResult.complete(null); @@ -763,7 +751,7 @@ private void sendOffsetsForLeaderEpochRequestsAndValidatePositions( partialResult.whenComplete((offsetsResult, error) -> { if (error == null) { - offsetFetcherUtils.onSuccessfulResponseForValidatingPositions(fetchPositions, + sharedOffsetsState.offsetFetcherUtils().onSuccessfulResponseForValidatingPositions(fetchPositions, offsetsResult); } else { RuntimeException e; @@ -773,7 +761,7 @@ private void sendOffsetsForLeaderEpochRequestsAndValidatePositions( e = new RuntimeException("Unexpected failure in OffsetsForLeaderEpoch " + "request for validating positions", error); } - offsetFetcherUtils.onFailedResponseForValidatingPositions(fetchPositions, e); + sharedOffsetsState.offsetFetcherUtils().onFailedResponseForValidatingPositions(fetchPositions, e); } }); }); @@ -916,7 +904,7 @@ private Map supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final AtomicBoolean cachedSubscriptionHasAllFetchPositions + final SharedOffsetsState sharedOffsetsState ) { return new CachedSupplier<>() { @Override @@ -288,13 +287,12 @@ protected RequestManagers create() { metadata, fetchConfig.isolationLevel, time, - retryBackoffMs, requestTimeoutMs, defaultApiTimeoutMs, apiVersions, networkClientDelegate, commitRequestManager, - cachedSubscriptionHasAllFetchPositions, + sharedOffsetsState, logContext); return new RequestManagers( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java new file mode 100644 index 0000000000000..ba486e7d9f1c0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java @@ -0,0 +1,88 @@ +/* + * 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.ApiVersions; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +class SharedOffsetsState { + + // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates + private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); + /** + * 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<>(); + private final OffsetFetcherUtils offsetFetcherUtils; + + SharedOffsetsState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs, + ApiVersions apiVersions) { + this.offsetFetcherUtils = new OffsetFetcherUtils( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + apiVersions + ); + } + + Optional getAndClearCachedUpdatePositionsException() { + return Optional.ofNullable(cachedUpdatePositionsException.getAndSet(null)); + } + + void setCachedUpdatePositionsException(Throwable t) { + cachedUpdatePositionsException.set(t); + } + + boolean subscriptionHasAllFetchPositions() { + return cachedSubscriptionHasAllFetchPositions.get(); + } + + void setSubscriptionHasAllFetchPositions(boolean value) { + cachedSubscriptionHasAllFetchPositions.set(value); + } + + Map getPartitionsToValidate() { + return offsetFetcherUtils.getPartitionsToValidate(); + } + + boolean canSkipUpdateFetchPositions() { + Optional error = getAndClearCachedUpdatePositionsException(); + + if (error.isPresent()) + throw ConsumerUtils.maybeWrapAsKafkaException(error.get()); + + return getPartitionsToValidate().isEmpty() && subscriptionHasAllFetchPositions(); + } + + OffsetFetcherUtils offsetFetcherUtils() { + return offsetFetcherUtils; + } +} 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 3e17676a10e50..e37a9505b1d6d 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NodeApiVersions; @@ -276,7 +277,8 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, groupId, - autoCommitEnabled); + autoCommitEnabled, + new SharedOffsetsState(new LogContext(), metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); } @Test @@ -670,6 +672,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { MockCommitCallback callback = new MockCommitCallback(); completeCommitAsyncApplicationEventSuccessfully(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + when(metadata.updateVersion()).thenReturn(-1); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAssignmentChangeEventSuccessfully(); @@ -1682,6 +1685,7 @@ private Properties requiredConsumerConfigAndGroupId(final String groupId) { private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + when(metadata.updateVersion()).thenReturn(-1); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); completeAssignmentChangeEventSuccessfully(); 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 27a55b0c28488..21d2804dbd807 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 @@ -62,7 +62,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -92,7 +91,6 @@ public class OffsetsRequestManagerTest { private final Time time = mock(Time.class); private ApiVersions apiVersions; private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); - private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); private static final String TEST_TOPIC = "t1"; private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1); private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2); @@ -114,13 +112,12 @@ public void setup() { metadata, DEFAULT_ISOLATION_LEVEL, time, - RETRY_BACKOFF_MS, REQUEST_TIMEOUT_MS, DEFAULT_API_TIMEOUT_MS, apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - cachedSubscriptionHasAllFetchPositions, + new SharedOffsetsState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), logContext ); } @@ -804,13 +801,12 @@ public void testRemoteListOffsetsRequestTimeoutMs() { metadata, DEFAULT_ISOLATION_LEVEL, time, - RETRY_BACKOFF_MS, requestTimeoutMs, defaultApiTimeoutMs, apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - cachedSubscriptionHasAllFetchPositions, + new SharedOffsetsState(new LogContext(), metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), new LogContext() ); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 4c1653167e5ed..9af81dfbb846c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -30,7 +30,6 @@ import java.util.Optional; import java.util.Properties; import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -51,16 +50,22 @@ public void testMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); + LogContext logContext = new LogContext(); + MockTime time = new MockTime(); + ConsumerMetadata metadata = mock(ConsumerMetadata.class); + SubscriptionState subscriptions = mock(SubscriptionState.class); + ApiVersions apiVersions = mock(ApiVersions.class); + long retryBackoffMs = 1000L; final RequestManagers requestManagers = RequestManagers.supplier( - new MockTime(), - new LogContext(), + time, + logContext, mock(BackgroundEventHandler.class), - mock(ConsumerMetadata.class), - mock(SubscriptionState.class), + metadata, + subscriptions, mock(FetchBuffer.class), config, groupRebalanceConfig, - mock(ApiVersions.class), + apiVersions, mock(FetchMetricsManager.class), () -> mock(NetworkClientDelegate.class), Optional.empty(), @@ -68,7 +73,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - new AtomicBoolean() + new SharedOffsetsState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -92,16 +97,22 @@ public void testStreamMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); + LogContext logContext = new LogContext(); + MockTime time = new MockTime(); + ConsumerMetadata metadata = mock(ConsumerMetadata.class); + SubscriptionState subscriptions = mock(SubscriptionState.class); + ApiVersions apiVersions = mock(ApiVersions.class); + long retryBackoffMs = 1000L; final RequestManagers requestManagers = RequestManagers.supplier( - new MockTime(), - new LogContext(), + time, + logContext, mock(BackgroundEventHandler.class), - mock(ConsumerMetadata.class), - mock(SubscriptionState.class), + metadata, + subscriptions, mock(FetchBuffer.class), config, groupRebalanceConfig, - mock(ApiVersions.class), + apiVersions, mock(FetchMetricsManager.class), () -> mock(NetworkClientDelegate.class), Optional.empty(), @@ -109,7 +120,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - new AtomicBoolean() + new SharedOffsetsState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From f62bf8a46ac4135a1bb5f4e27dbadd80cd9fc435 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Aug 2025 16:27:32 -0700 Subject: [PATCH 03/31] Setting cachedSubscriptionHasAllFetchPositions to false when assignments change --- .../clients/consumer/internals/RequestManagers.java | 1 + .../consumer/internals/SharedOffsetsState.java | 13 ++++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) 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 01b80712766db..8cbc147498fe0 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 @@ -269,6 +269,7 @@ protected RequestManagers create() { .updateMetricsLabels(Map.of(ClientTelemetryProvider.GROUP_MEMBER_ID, membershipManager.memberId())); } + membershipManager.registerStateListener(sharedOffsetsState); membershipManager.registerStateListener(commitRequestManager); membershipManager.registerStateListener(applicationThreadMemberStateListener); heartbeatRequestManager = new ConsumerHeartbeatRequestManager( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java index ba486e7d9f1c0..211056cacc6f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java @@ -23,10 +23,11 @@ import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -class SharedOffsetsState { +class SharedOffsetsState implements MemberStateListener { // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); @@ -85,4 +86,14 @@ boolean canSkipUpdateFetchPositions() { OffsetFetcherUtils offsetFetcherUtils() { return offsetFetcherUtils; } + + @Override + public void onMemberEpochUpdated(Optional memberEpoch, String memberId) { + // Ignore... + } + + @Override + public void onGroupAssignmentUpdated(Set partitions) { + setSubscriptionHasAllFetchPositions(false); + } } From 386fafb3aaf3fa6de8e960ed60f84c597e2f2477 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Aug 2025 16:01:22 -0700 Subject: [PATCH 04/31] Reverting use of sharedOffsetsState.offsetFetcherUtils() and retrieving the OffsetFetcherUtils once, in the constructor --- .../internals/OffsetsRequestManager.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 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 dcb97c7cdf11f..6c6422735fb9b 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 @@ -81,6 +81,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final ConsumerMetadata metadata; private final IsolationLevel isolationLevel; private final Logger log; + private final OffsetFetcherUtils offsetFetcherUtils; private final SubscriptionState subscriptionState; private final Set requestsToRetry; @@ -132,6 +133,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; + this.offsetFetcherUtils = sharedOffsetsState.offsetFetcherUtils(); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are // initialized and the network thread started. @@ -186,7 +188,7 @@ public CompletableFuture> fetchO ListOffsetsRequestState listOffsetsRequestState = new ListOffsetsRequestState( timestampsToSearch, requireTimestamps, - sharedOffsetsState.offsetFetcherUtils(), + offsetFetcherUtils, isolationLevel); listOffsetsRequestState.globalResult.whenComplete((result, error) -> { metadata.clearTransientTopics(); @@ -470,7 +472,7 @@ CompletableFuture resetPositionsIfNeeded() { Map partitionAutoOffsetResetStrategyMap; try { - partitionAutoOffsetResetStrategyMap = sharedOffsetsState.offsetFetcherUtils().getOffsetResetStrategyForPartitions(); + partitionAutoOffsetResetStrategyMap = offsetFetcherUtils.getOffsetResetStrategyForPartitions(); } catch (Exception e) { CompletableFuture result = new CompletableFuture<>(); result.completeExceptionally(e); @@ -563,7 +565,7 @@ private List buildListOffsetsRequests( if (error == null) { listOffsetsRequestState.fetchedOffsets.putAll(multiNodeResult.fetchedOffsets); listOffsetsRequestState.addPartitionsToRetry(multiNodeResult.partitionsToRetry); - sharedOffsetsState.offsetFetcherUtils().updateSubscriptionState(multiNodeResult.fetchedOffsets, + offsetFetcherUtils.updateSubscriptionState(multiNodeResult.fetchedOffsets, isolationLevel); if (listOffsetsRequestState.remainingToSearch.isEmpty()) { @@ -633,7 +635,7 @@ private CompletableFuture buildListOffsetRequestToNode( ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); try { - ListOffsetResult listOffsetResult = sharedOffsetsState.offsetFetcherUtils().handleListOffsetResponse(lor); + ListOffsetResult listOffsetResult = offsetFetcherUtils.handleListOffsetResponse(lor); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -675,7 +677,7 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( partialResult.whenComplete((result, error) -> { if (error == null) { - sharedOffsetsState.offsetFetcherUtils().onSuccessfulResponseForResettingPositions(result, + offsetFetcherUtils.onSuccessfulResponseForResettingPositions(result, partitionAutoOffsetResetStrategyMap); } else { RuntimeException e; @@ -685,7 +687,7 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( e = new RuntimeException("Unexpected failure in ListOffsets request for " + "resetting positions", error); } - sharedOffsetsState.offsetFetcherUtils().onFailedResponseForResettingPositions(resetTimestamps, e); + offsetFetcherUtils.onFailedResponseForResettingPositions(resetTimestamps, e); } if (expectedResponses.decrementAndGet() == 0) { globalResult.complete(null); @@ -751,7 +753,7 @@ private void sendOffsetsForLeaderEpochRequestsAndValidatePositions( partialResult.whenComplete((offsetsResult, error) -> { if (error == null) { - sharedOffsetsState.offsetFetcherUtils().onSuccessfulResponseForValidatingPositions(fetchPositions, + offsetFetcherUtils.onSuccessfulResponseForValidatingPositions(fetchPositions, offsetsResult); } else { RuntimeException e; @@ -761,7 +763,7 @@ private void sendOffsetsForLeaderEpochRequestsAndValidatePositions( e = new RuntimeException("Unexpected failure in OffsetsForLeaderEpoch " + "request for validating positions", error); } - sharedOffsetsState.offsetFetcherUtils().onFailedResponseForValidatingPositions(fetchPositions, e); + offsetFetcherUtils.onFailedResponseForValidatingPositions(fetchPositions, e); } }); }); @@ -904,7 +906,7 @@ private Map Date: Fri, 8 Aug 2025 16:12:08 -0700 Subject: [PATCH 05/31] Renaming to CommitOffsetsSharedState --- .../internals/AsyncKafkaConsumer.java | 18 +++++----- ...ate.java => CommitOffsetsSharedState.java} | 36 +++++++++++-------- .../internals/OffsetsRequestManager.java | 22 ++++++------ .../consumer/internals/RequestManagers.java | 6 ++-- .../internals/AsyncKafkaConsumerTest.java | 2 +- .../internals/OffsetsRequestManagerTest.java | 4 +-- .../internals/RequestManagersTest.java | 4 +-- 7 files changed, 49 insertions(+), 43 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedOffsetsState.java => CommitOffsetsSharedState.java} (74%) 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 4bd24c0e85b25..9f5143d447274 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 @@ -360,7 +360,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - private final SharedOffsetsState sharedOffsetsState; + private final CommitOffsetsSharedState commitOffsetsSharedState; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -476,7 +476,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); - this.sharedOffsetsState = new SharedOffsetsState( + this.commitOffsetsSharedState = new CommitOffsetsSharedState( logContext, metadata, subscriptions, @@ -500,7 +500,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - sharedOffsetsState + commitOffsetsSharedState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -572,7 +572,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int defaultApiTimeoutMs, String groupId, boolean autoCommitEnabled, - SharedOffsetsState sharedOffsetsState) { + CommitOffsetsSharedState commitOffsetsSharedState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -602,7 +602,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, kafkaConsumerMetrics ); - this.sharedOffsetsState = sharedOffsetsState; + this.commitOffsetsSharedState = commitOffsetsSharedState; } AsyncKafkaConsumer(LogContext logContext, @@ -672,7 +672,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.sharedOffsetsState = new SharedOffsetsState( + this.commitOffsetsSharedState = new CommitOffsetsSharedState( logContext, metadata, subscriptions, @@ -697,7 +697,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - sharedOffsetsState + commitOffsetsSharedState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -1811,7 +1811,7 @@ private Fetch pollForFetches(Timer timer) { // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call // updateAssignmentMetadataIfNeeded before this method. - if (!sharedOffsetsState.subscriptionHasAllFetchPositions() && pollTimeout > retryBackoffMs) { + if (!commitOffsetsSharedState.subscriptionHasAllFetchPositions() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } @@ -1866,7 +1866,7 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { - if (sharedOffsetsState.canSkipUpdateFetchPositions()) + if (commitOffsetsSharedState.canSkipUpdateFetchPositions()) return true; try { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java similarity index 74% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java index 211056cacc6f3..2026b55c6e773 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedOffsetsState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java @@ -27,10 +27,16 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -class SharedOffsetsState implements MemberStateListener { +/** + * + */ +public class CommitOffsetsSharedState implements MemberStateListener { - // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates + /** + * To keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates. + */ private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); + /** * 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. @@ -38,12 +44,12 @@ class SharedOffsetsState implements MemberStateListener { private final AtomicReference cachedUpdatePositionsException = new AtomicReference<>(); private final OffsetFetcherUtils offsetFetcherUtils; - SharedOffsetsState(LogContext logContext, - ConsumerMetadata metadata, - SubscriptionState subscriptions, - Time time, - long retryBackoffMs, - ApiVersions apiVersions) { + CommitOffsetsSharedState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs, + ApiVersions apiVersions) { this.offsetFetcherUtils = new OffsetFetcherUtils( logContext, metadata, @@ -54,12 +60,12 @@ class SharedOffsetsState implements MemberStateListener { ); } - Optional getAndClearCachedUpdatePositionsException() { - return Optional.ofNullable(cachedUpdatePositionsException.getAndSet(null)); + Throwable getAndClearCachedUpdatePositionsException() { + return cachedUpdatePositionsException.getAndSet(null); } - void setCachedUpdatePositionsException(Throwable t) { - cachedUpdatePositionsException.set(t); + void setCachedUpdatePositionsException(Throwable exception) { + cachedUpdatePositionsException.set(exception); } boolean subscriptionHasAllFetchPositions() { @@ -75,10 +81,10 @@ Map getPartitionsToValidate() { } boolean canSkipUpdateFetchPositions() { - Optional error = getAndClearCachedUpdatePositionsException(); + Throwable exception = getAndClearCachedUpdatePositionsException(); - if (error.isPresent()) - throw ConsumerUtils.maybeWrapAsKafkaException(error.get()); + if (exception != null) + throw ConsumerUtils.maybeWrapAsKafkaException(exception); return getPartitionsToValidate().isEmpty() && subscriptionHasAllFetchPositions(); } 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 6c6422735fb9b..0ae922c932533 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 @@ -92,7 +92,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; - private final SharedOffsetsState sharedOffsetsState; + private final CommitOffsetsSharedState commitOffsetsSharedState; /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update @@ -112,7 +112,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final CommitRequestManager commitRequestManager, - final SharedOffsetsState sharedOffsetsState, + final CommitOffsetsSharedState commitOffsetsSharedState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -133,13 +133,13 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; - this.offsetFetcherUtils = sharedOffsetsState.offsetFetcherUtils(); + this.offsetFetcherUtils = commitOffsetsSharedState.offsetFetcherUtils(); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); this.commitRequestManager = commitRequestManager; - this.sharedOffsetsState = sharedOffsetsState; + this.commitOffsetsSharedState = commitOffsetsSharedState; } private static class PendingFetchCommittedRequest { @@ -237,11 +237,11 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (subscriptionState.hasAllFetchPositions()) { // All positions are already available - sharedOffsetsState.setSubscriptionHasAllFetchPositions(true); + commitOffsetsSharedState.setSubscriptionHasAllFetchPositions(true); result.complete(true); return result; } else { - sharedOffsetsState.setSubscriptionHasAllFetchPositions(false); + commitOffsetsSharedState.setSubscriptionHasAllFetchPositions(false); } // Some positions are missing, so trigger requests to fetch offsets and update them. @@ -260,9 +260,9 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { } private boolean maybeCompleteWithPreviousException(CompletableFuture result) { - Optional cachedException = sharedOffsetsState.getAndClearCachedUpdatePositionsException(); - if (cachedException.isPresent()) { - result.completeExceptionally(cachedException.get()); + Throwable cachedException = commitOffsetsSharedState.getAndClearCachedUpdatePositionsException(); + if (cachedException != null) { + result.completeExceptionally(cachedException); return true; } return false; @@ -315,7 +315,7 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - sharedOffsetsState.setCachedUpdatePositionsException(error); + commitOffsetsSharedState.setCachedUpdatePositionsException(error); } }); } @@ -498,7 +498,7 @@ CompletableFuture resetPositionsIfNeeded() { * next call to this function. */ void validatePositionsIfNeeded() { - Map partitionsToValidate = sharedOffsetsState.getPartitionsToValidate(); + Map partitionsToValidate = commitOffsetsSharedState.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return; } 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 8cbc147498fe0..4401d82de8094 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 @@ -168,7 +168,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final SharedOffsetsState sharedOffsetsState + final CommitOffsetsSharedState commitOffsetsSharedState ) { return new CachedSupplier<>() { @Override @@ -269,7 +269,7 @@ protected RequestManagers create() { .updateMetricsLabels(Map.of(ClientTelemetryProvider.GROUP_MEMBER_ID, membershipManager.memberId())); } - membershipManager.registerStateListener(sharedOffsetsState); + membershipManager.registerStateListener(commitOffsetsSharedState); membershipManager.registerStateListener(commitRequestManager); membershipManager.registerStateListener(applicationThreadMemberStateListener); heartbeatRequestManager = new ConsumerHeartbeatRequestManager( @@ -293,7 +293,7 @@ protected RequestManagers create() { apiVersions, networkClientDelegate, commitRequestManager, - sharedOffsetsState, + commitOffsetsSharedState, logContext); return new RequestManagers( 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 e37a9505b1d6d..0e3a6e79089d3 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 @@ -278,7 +278,7 @@ private AsyncKafkaConsumer newConsumer( defaultApiTimeoutMs, groupId, autoCommitEnabled, - new SharedOffsetsState(new LogContext(), metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); + new CommitOffsetsSharedState(new LogContext(), metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); } @Test 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 21d2804dbd807..b23f75ebdb8b3 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 @@ -117,7 +117,7 @@ public void setup() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new SharedOffsetsState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + new CommitOffsetsSharedState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), logContext ); } @@ -806,7 +806,7 @@ public void testRemoteListOffsetsRequestTimeoutMs() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new SharedOffsetsState(new LogContext(), metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + new CommitOffsetsSharedState(new LogContext(), metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), new LogContext() ); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 9af81dfbb846c..a3d41eb4fd4c6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -73,7 +73,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - new SharedOffsetsState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new CommitOffsetsSharedState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -120,7 +120,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - new SharedOffsetsState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new CommitOffsetsSharedState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From cc35591a43e2ab41967454d39d8413991b89be6a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 10 Aug 2025 16:13:12 -0700 Subject: [PATCH 06/31] KAFKA-19588: Reduce number of events generated in AsyncKafkaConsumer.poll() MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We create—and wait on—PollEvent in Consumer.poll() to ensure we wait for reconciliation and/or auto-commit. However, reconciliation is relatively rare, and auto-commit only happens every N seconds, so the remainder of the time, we should try to avoid sending poll events. --- .../internals/AbstractMembershipManager.java | 30 ++++++++++--------- .../internals/AsyncKafkaConsumer.java | 30 ++++++++++++------- .../internals/CommitRequestManager.java | 24 ++++++++------- .../internals/ConsumerMembershipManager.java | 13 +++++--- .../consumer/internals/RequestManagers.java | 11 +++++-- .../internals/ShareMembershipManager.java | 4 ++- .../internals/AsyncKafkaConsumerTest.java | 6 +++- .../internals/CommitRequestManagerTest.java | 9 ++++-- .../ConsumerMembershipManagerTest.java | 7 +++-- .../internals/RequestManagersTest.java | 9 ++++-- 10 files changed, 92 insertions(+), 51 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 74ecf7f9bb8d4..683b1f94c3a85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -135,13 +135,6 @@ public abstract class AbstractMembershipManager impl */ private LocalAssignment currentTargetAssignment; - /** - * If there is a reconciliation running (triggering commit, callbacks) for the - * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered - * after receiving a heartbeat response, or a metadata update. - */ - private boolean reconciliationInProgress; - /** * True if a reconciliation is in progress and the member rejoined the group since the start * of the reconciliation. Used to know that the reconciliation in progress should be @@ -178,6 +171,13 @@ public abstract class AbstractMembershipManager impl private final Time time; + /** + * If there is a reconciliation running (triggering commit, callbacks) for the + * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered + * after receiving a heartbeat response, or a metadata update. + */ + private final AtomicBoolean reconciliationInProgress; + /** * AtomicBoolean to track whether the subscription is updated. * If it's true and subscription state is UNSUBSCRIBED, the next {@link #onConsumerPoll()} will change member state to JOINING. @@ -208,7 +208,8 @@ public abstract class AbstractMembershipManager impl Logger log, Time time, RebalanceMetricsManager metricsManager, - boolean autoCommitEnabled) { + boolean autoCommitEnabled, + AtomicBoolean reconciliationInProgress) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.subscriptions = subscriptions; @@ -221,6 +222,7 @@ public abstract class AbstractMembershipManager impl this.time = time; this.metricsManager = metricsManager; this.autoCommitEnabled = autoCommitEnabled; + this.reconciliationInProgress = reconciliationInProgress; } /** @@ -530,7 +532,7 @@ public void transitionToJoining() { "the member is in FATAL state"); return; } - if (reconciliationInProgress) { + if (reconciliationInProgress.get()) { rejoinedWhileReconciliationInProgress = true; } resetEpoch(); @@ -830,7 +832,7 @@ public void maybeReconcile(boolean canCommit) { "current assignment."); return; } - if (reconciliationInProgress) { + if (reconciliationInProgress.get()) { log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. " + "Assignment {} will be handled in the next reconciliation loop.", currentTargetAssignment); return; @@ -963,7 +965,7 @@ private void revokeAndAssign(LocalAssignment resolvedAssignment, log.error("Reconciliation failed.", error); markReconciliationCompleted(); } else { - if (reconciliationInProgress && !maybeAbortReconciliation()) { + if (reconciliationInProgress.get() && !maybeAbortReconciliation()) { currentAssignment = resolvedAssignment; signalReconciliationCompleting(); @@ -1034,7 +1036,7 @@ protected CompletableFuture signalPartitionsLost(Set parti * Visible for testing. */ void markReconciliationInProgress() { - reconciliationInProgress = true; + reconciliationInProgress.set(true); rejoinedWhileReconciliationInProgress = false; } @@ -1042,7 +1044,7 @@ void markReconciliationInProgress() { * Visible for testing. */ void markReconciliationCompleted() { - reconciliationInProgress = false; + reconciliationInProgress.set(false); rejoinedWhileReconciliationInProgress = false; } @@ -1372,7 +1374,7 @@ Map> topicPartitionsAwaitingReconciliation() { * by a call to {@link #maybeReconcile(boolean)}. Visible for testing. */ boolean reconciliationInProgress() { - return reconciliationInProgress; + return reconciliationInProgress.get(); } /** 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 1ca42dbc75c6c..4b868701889fc 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 @@ -131,6 +131,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -356,6 +357,8 @@ private StreamsRebalanceListener streamsRebalanceListener() { private final int requestTimeoutMs; private final Duration defaultApiTimeoutMs; private final boolean autoCommitEnabled; + private final AtomicBoolean cachedIsReconciliationInProgress = new AtomicBoolean(); + private final AtomicBoolean cachedHasInflightCommit = new AtomicBoolean(); private volatile boolean closed = false; // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); @@ -492,7 +495,9 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metrics, offsetCommitCallbackInvoker, memberStateListener, - streamsRebalanceData + streamsRebalanceData, + cachedHasInflightCommit, + cachedIsReconciliationInProgress ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -678,7 +683,9 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metrics, offsetCommitCallbackInvoker, memberStateListener, - Optional.empty() + Optional.empty(), + cachedHasInflightCommit, + cachedIsReconciliationInProgress ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -860,14 +867,17 @@ 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()); + if (cachedIsReconciliationInProgress.get() || cachedHasInflightCommit.get()) { + 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 diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index b6a6a9f14cf84..86c38cbb59bff 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -65,6 +65,7 @@ import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -115,7 +116,8 @@ public CommitRequestManager( final String groupId, final Optional groupInstanceId, final Metrics metrics, - final ConsumerMetadata metadata) { + final ConsumerMetadata metadata, + final AtomicBoolean cachedHasInflightCommit) { this(time, logContext, subscriptions, @@ -128,7 +130,8 @@ public CommitRequestManager( config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG), OptionalDouble.empty(), metrics, - metadata); + metadata, + cachedHasInflightCommit); } // Visible for testing @@ -145,7 +148,8 @@ public CommitRequestManager( final long retryBackoffMaxMs, final OptionalDouble jitter, final Metrics metrics, - final ConsumerMetadata metadata) { + final ConsumerMetadata metadata, + final AtomicBoolean cachedHasInflightCommit) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.time = time; this.logContext = logContext; @@ -154,7 +158,7 @@ public CommitRequestManager( if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long autoCommitInterval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval, logContext)); + this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval, logContext, cachedHasInflightCommit)); } else { this.autoCommitState = Optional.empty(); } @@ -1307,17 +1311,17 @@ private void maybeFailOnCoordinatorFatalError() { private static class AutoCommitState { private final Timer timer; private final long autoCommitInterval; - private boolean hasInflightCommit; - + private final AtomicBoolean cachedHasInflightCommit; private final Logger log; public AutoCommitState( final Time time, final long autoCommitInterval, - final LogContext logContext) { + final LogContext logContext, + final AtomicBoolean cachedHasInflightCommit) { this.autoCommitInterval = autoCommitInterval; this.timer = time.timer(autoCommitInterval); - this.hasInflightCommit = false; + this.cachedHasInflightCommit = cachedHasInflightCommit; this.log = logContext.logger(getClass()); } @@ -1325,7 +1329,7 @@ public boolean shouldAutoCommit() { if (!this.timer.isExpired()) { return false; } - if (this.hasInflightCommit) { + if (this.cachedHasInflightCommit.get()) { log.trace("Skipping auto-commit on the interval because a previous one is still in-flight."); return false; } @@ -1350,7 +1354,7 @@ public void updateTimer(final long currentTimeMs) { } public void setInflightCommitStatus(final boolean inflightCommitStatus) { - this.hasInflightCommit = inflightCommitStatus; + this.cachedHasInflightCommit.set(inflightCommitStatus); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java index e07424a63938e..eed2b32cd8cc9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java @@ -44,6 +44,7 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.DEFAULT; import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.LEAVE_GROUP; @@ -151,7 +152,8 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, Metrics metrics, - boolean autoCommitEnabled) { + boolean autoCommitEnabled, + AtomicBoolean reconciliationInProgress) { this(groupId, groupInstanceId, rackId, @@ -164,7 +166,8 @@ public ConsumerMembershipManager(String groupId, backgroundEventHandler, time, new ConsumerRebalanceMetricsManager(metrics), - autoCommitEnabled); + autoCommitEnabled, + reconciliationInProgress); } // Visible for testing @@ -180,14 +183,16 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, RebalanceMetricsManager metricsManager, - boolean autoCommitEnabled) { + boolean autoCommitEnabled, + AtomicBoolean reconciliationInProgress) { super(groupId, subscriptions, metadata, logContext.logger(ConsumerMembershipManager.class), time, metricsManager, - autoCommitEnabled); + autoCommitEnabled, + reconciliationInProgress); this.groupInstanceId = groupInstanceId; this.rackId = rackId; this.rebalanceTimeoutMs = rebalanceTimeoutMs; 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..aa604529d14e6 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 @@ -35,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import static java.util.Objects.requireNonNull; @@ -167,7 +168,9 @@ public static Supplier supplier(final Time time, final Metrics metrics, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, - final Optional streamsRebalanceData + final Optional streamsRebalanceData, + final AtomicBoolean cachedHasInflightCommit, + final AtomicBoolean cachedReconciliationInProgress ) { return new CachedSupplier<>() { @Override @@ -216,7 +219,8 @@ protected RequestManagers create() { groupRebalanceConfig.groupId, groupRebalanceConfig.groupInstanceId, metrics, - metadata); + metadata, + cachedHasInflightCommit); if (streamsRebalanceData.isPresent()) { streamsMembershipManager = new StreamsMembershipManager( groupRebalanceConfig.groupId, @@ -258,7 +262,8 @@ protected RequestManagers create() { backgroundEventHandler, time, metrics, - config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), + cachedReconciliationInProgress); // Update the group member ID label in the client telemetry reporter. // According to KIP-1082, the consumer will generate the member ID as the incarnation ID of the process. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java index 47ab87edb358d..ba15c2c74e486 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicBoolean; /** * Group manager for a single consumer that has a group id defined in the config @@ -107,7 +108,8 @@ public ShareMembershipManager(LogContext logContext, logContext.logger(ShareMembershipManager.class), time, metricsManager, - false); + false, + new AtomicBoolean()); this.rackId = rackId; } 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 16e4c9d4b3fde..7eb5e631b1e5b 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 @@ -1355,6 +1355,8 @@ private MemberStateListener captureGroupMetadataUpdateListener(final MockedStati any(), any(), applicationThreadMemberStateListener.capture(), + any(), + any(), any() )); return applicationThreadMemberStateListener.getValue(); @@ -1426,7 +1428,9 @@ private Optional captureStreamRebalanceData(final MockedSt any(), any(), any(), - streamRebalanceData.capture() + streamRebalanceData.capture(), + any(), + any() )); return streamRebalanceData.getValue(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index d4ceeedde56b1..e2bb28d4a59ff 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -71,6 +71,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -117,7 +118,7 @@ public class CommitRequestManagerTest { private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final Metrics metrics = new Metrics(); private Properties props; - + private final AtomicBoolean cachedHasInflightCommit = new AtomicBoolean(); private final int defaultApiTimeoutMs = 60000; @@ -152,7 +153,8 @@ public void testOffsetFetchRequestStateToStringBase() { retryBackoffMaxMs, OptionalDouble.of(0), metrics, - metadata); + metadata, + cachedHasInflightCommit); commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString()); Set requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1)); @@ -1578,7 +1580,8 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long retryBackoffMaxMs, OptionalDouble.of(0), metrics, - metadata)); + metadata, + cachedHasInflightCommit)); } private ClientResponse buildOffsetFetchClientResponse( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index aa8c7bdb1dcae..efba723399f3d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -66,6 +66,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -144,7 +145,7 @@ private ConsumerMembershipManager createMembershipManager(String groupInstanceId ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, true)); + backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean())); assertMemberIdIsGenerated(manager.memberId()); return manager; } @@ -157,7 +158,7 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(rackId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, true)); + metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean())); assertMemberIdIsGenerated(manager.memberId()); manager.transitionToJoining(); return manager; @@ -245,7 +246,7 @@ public void testTransitionToFailedWhenTryingToJoin() { ConsumerMembershipManager membershipManager = new ConsumerMembershipManager( GROUP_ID, Optional.empty(), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, true); + backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean()); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 67628c513406a..678775000ecfa 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -30,6 +30,7 @@ import java.util.Optional; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -66,7 +67,9 @@ public void testMemberStateListenerRegistered() { new Metrics(), mock(OffsetCommitCallbackInvoker.class), listener, - Optional.empty() + Optional.empty(), + new AtomicBoolean(), + new AtomicBoolean() ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -106,7 +109,9 @@ public void testStreamMemberStateListenerRegistered() { new Metrics(), mock(OffsetCommitCallbackInvoker.class), listener, - Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())) + Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), + new AtomicBoolean(), + new AtomicBoolean() ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From 0c4dff62d0a7f4554160f504cd7d4eb621499b3c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 10 Aug 2025 16:22:54 -0700 Subject: [PATCH 07/31] Updates --- .../internals/AsyncKafkaConsumer.java | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 4b868701889fc..b11fa62b8892b 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 @@ -866,19 +866,19 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - do { - if (cachedIsReconciliationInProgress.get() || cachedHasInflightCommit.get()) { - 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()); - } + 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); + + if (cachedIsReconciliationInProgress.get() || cachedHasInflightCommit.get()) { + // 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()); + } + 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 @@ -1908,7 +1908,7 @@ private boolean isCommittedOffsetsManagementEnabled() { */ private void sendFetches(Timer timer) { try { - applicationEventHandler.addAndGet(new CreateFetchRequestsEvent(calculateDeadlineMs(timer))); + applicationEventHandler.add(new CreateFetchRequestsEvent(calculateDeadlineMs(timer))); } catch (TimeoutException swallow) { // Can be ignored, per above comments. } From 7550adbb4a55c65397454229b47faf5d3afd7864 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 15 Aug 2025 09:18:08 -0700 Subject: [PATCH 08/31] Update to check the SubscriptionState instead of relying solely on the cached value --- .../internals/CommitOffsetsSharedState.java | 69 +++++++++++++++++-- .../internals/OffsetsRequestManager.java | 2 +- .../internals/RequestManagersTest.java | 2 +- 3 files changed, 65 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java index 2026b55c6e773..94b4cd027a0de 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java @@ -17,18 +17,29 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import java.util.Map; +import java.time.Duration; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; /** + * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the + * background thread ({@link OffsetsRequestManager}) to determine if a costly call to check offsets can be skipped + * inside {@link Consumer#poll(Duration)}. * + *

+ * + * This class compromises on the ideal of keeping the state only in the background thread. However, this class only + * relies on the {@link SubscriptionState} and {@link ConsumerMetadata} which are, unfortunately, already used + * sparingly in both the application and background threads. Both of those classes are heavily synchronized given + * their use by the {@link ClassicKafkaConsumer}, so their use in a multithreaded fashion is already established. */ public class CommitOffsetsSharedState implements MemberStateListener { @@ -43,6 +54,7 @@ public class CommitOffsetsSharedState implements MemberStateListener { */ private final AtomicReference cachedUpdatePositionsException = new AtomicReference<>(); private final OffsetFetcherUtils offsetFetcherUtils; + private final SubscriptionState subscriptions; CommitOffsetsSharedState(LogContext logContext, ConsumerMetadata metadata, @@ -58,6 +70,7 @@ public class CommitOffsetsSharedState implements MemberStateListener { retryBackoffMs, apiVersions ); + this.subscriptions = subscriptions; } Throwable getAndClearCachedUpdatePositionsException() { @@ -76,17 +89,61 @@ void setSubscriptionHasAllFetchPositions(boolean value) { cachedSubscriptionHasAllFetchPositions.set(value); } - Map getPartitionsToValidate() { - return offsetFetcherUtils.getPartitionsToValidate(); - } - + /** + * This method is used by {@code AsyncKafkaConsumer#updateFetchPositions()} to determine if it can skip + * the step of sending (and waiting for) a {@link CheckAndUpdatePositionsEvent}. {@code updateFetchPositions()} + * is in the critical path for the {@link Consumer#poll(Duration)}, and if the application thread can determine + * that it doesn't need to perform the {@link OffsetsRequestManager#updateFetchPositions(long)} call (via the + * {@link CheckAndUpdatePositionsEvent}), that is a big performance savings. + * + *

+ * + * This method performs similar checks to the start of {@link OffsetsRequestManager#updateFetchPositions(long)}: + * + *

    + *
  1. + * Checks for previous exceptions during update positions + * ({@code OffsetsRequestManager#cacheExceptionIfEventExpired()}) + *
  2. + *
  3. + * Checks that the previously cached version of {@link #cachedSubscriptionHasAllFetchPositions} is still + * {@code true}. This covers a couple additional cases (like before first assignment), so although it's + * not completely optimal, the fallback is we force the check which will block the application thread + * while the background thread performs its checks which may result in the value being set back to + * {@code true} before the application thread resumes. + *
  4. + *
  5. + * Checks that there are no positions in the {@link SubscriptionState.FetchStates#AWAIT_VALIDATION} + * state ({@link OffsetFetcherUtils#getPartitionsToValidate()}) + *
  6. + *
  7. + * Checks that all positions are in the {@link SubscriptionState.FetchStates#FETCHING} state + * ({@link SubscriptionState#hasAllFetchPositions()}) + *
  8. + *
+ * + * If the first check fails, an exception will be thrown. If any of the second, third, or fourth checks fail, this + * method will return {@code false}. Otherwise, this method will return {@code true}, which signals to the + * application thread that the {@link CheckAndUpdatePositionsEvent} can be skipped. + * + * @return true if all checks pass, false if either of the latter two checks fail + */ boolean canSkipUpdateFetchPositions() { Throwable exception = getAndClearCachedUpdatePositionsException(); if (exception != null) throw ConsumerUtils.maybeWrapAsKafkaException(exception); - return getPartitionsToValidate().isEmpty() && subscriptionHasAllFetchPositions(); + // If the cached value is set and there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or + // INITIALIZING states, it's ok to skip. + if (cachedSubscriptionHasAllFetchPositions.get() && offsetFetcherUtils.getPartitionsToValidate().isEmpty() && subscriptions.hasAllFetchPositions()) + return true; + + // However, if even one partition is not set to FETCHING, set the cached value to false and signal the + // application thread that it should call the background thread to perform the necessary logic to resolve the + // state of the partitions. + setSubscriptionHasAllFetchPositions(false); + return false; } OffsetFetcherUtils offsetFetcherUtils() { 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 0ae922c932533..a4cfcc6c8023f 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 @@ -498,7 +498,7 @@ CompletableFuture resetPositionsIfNeeded() { * next call to this function. */ void validatePositionsIfNeeded() { - Map partitionsToValidate = commitOffsetsSharedState.getPartitionsToValidate(); + Map partitionsToValidate = offsetFetcherUtils.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index a3d41eb4fd4c6..cc4339ec42ac6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -79,7 +79,7 @@ public void testMemberStateListenerRegistered() { assertTrue(requestManagers.streamsMembershipManager.isEmpty()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isEmpty()); - assertEquals(2, requestManagers.consumerMembershipManager.get().stateListeners().size()); + assertEquals(3, requestManagers.consumerMembershipManager.get().stateListeners().size()); assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().stream() .anyMatch(m -> m instanceof CommitRequestManager)); assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().contains(listener)); From d45011cd3f1cb7626d8397fa0c2ec1f8bb974889 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 15 Aug 2025 11:00:56 -0700 Subject: [PATCH 09/31] Updated naming --- .../consumer/internals/AsyncKafkaConsumer.java | 16 ++++++++-------- .../internals/CommitRequestManager.java | 18 +++++++++--------- 2 files changed, 17 insertions(+), 17 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 b11fa62b8892b..5df146ac16424 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 @@ -357,8 +357,8 @@ private StreamsRebalanceListener streamsRebalanceListener() { private final int requestTimeoutMs; private final Duration defaultApiTimeoutMs; private final boolean autoCommitEnabled; - private final AtomicBoolean cachedIsReconciliationInProgress = new AtomicBoolean(); - private final AtomicBoolean cachedHasInflightCommit = new AtomicBoolean(); + private final AtomicBoolean reconciliationInProgress = new AtomicBoolean(); + private final AtomicBoolean hasInflightCommit = new AtomicBoolean(); private volatile boolean closed = false; // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); @@ -496,8 +496,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - cachedHasInflightCommit, - cachedIsReconciliationInProgress + hasInflightCommit, + reconciliationInProgress ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -684,8 +684,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - cachedHasInflightCommit, - cachedIsReconciliationInProgress + hasInflightCommit, + reconciliationInProgress ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -872,7 +872,7 @@ public ConsumerRecords poll(final Duration timeout) { // the interval time or reconciling new assignments applicationEventHandler.add(event); - if (cachedIsReconciliationInProgress.get() || cachedHasInflightCommit.get()) { + if (reconciliationInProgress.get() || hasInflightCommit.get()) { // 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()); @@ -1908,7 +1908,7 @@ private boolean isCommittedOffsetsManagementEnabled() { */ private void sendFetches(Timer timer) { try { - applicationEventHandler.add(new CreateFetchRequestsEvent(calculateDeadlineMs(timer))); + applicationEventHandler.addAndGet(new CreateFetchRequestsEvent(calculateDeadlineMs(timer))); } catch (TimeoutException swallow) { // Can be ignored, per above comments. } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 86c38cbb59bff..9f722e3e903e3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -117,7 +117,7 @@ public CommitRequestManager( final Optional groupInstanceId, final Metrics metrics, final ConsumerMetadata metadata, - final AtomicBoolean cachedHasInflightCommit) { + final AtomicBoolean hasInflightCommit) { this(time, logContext, subscriptions, @@ -131,7 +131,7 @@ public CommitRequestManager( OptionalDouble.empty(), metrics, metadata, - cachedHasInflightCommit); + hasInflightCommit); } // Visible for testing @@ -149,7 +149,7 @@ public CommitRequestManager( final OptionalDouble jitter, final Metrics metrics, final ConsumerMetadata metadata, - final AtomicBoolean cachedHasInflightCommit) { + final AtomicBoolean hasInflightCommit) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.time = time; this.logContext = logContext; @@ -158,7 +158,7 @@ public CommitRequestManager( if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long autoCommitInterval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval, logContext, cachedHasInflightCommit)); + this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval, logContext, hasInflightCommit)); } else { this.autoCommitState = Optional.empty(); } @@ -1311,17 +1311,17 @@ private void maybeFailOnCoordinatorFatalError() { private static class AutoCommitState { private final Timer timer; private final long autoCommitInterval; - private final AtomicBoolean cachedHasInflightCommit; + private final AtomicBoolean hasInflightCommit; private final Logger log; public AutoCommitState( final Time time, final long autoCommitInterval, final LogContext logContext, - final AtomicBoolean cachedHasInflightCommit) { + final AtomicBoolean hasInflightCommit) { this.autoCommitInterval = autoCommitInterval; this.timer = time.timer(autoCommitInterval); - this.cachedHasInflightCommit = cachedHasInflightCommit; + this.hasInflightCommit = hasInflightCommit; this.log = logContext.logger(getClass()); } @@ -1329,7 +1329,7 @@ public boolean shouldAutoCommit() { if (!this.timer.isExpired()) { return false; } - if (this.cachedHasInflightCommit.get()) { + if (this.hasInflightCommit.get()) { log.trace("Skipping auto-commit on the interval because a previous one is still in-flight."); return false; } @@ -1354,7 +1354,7 @@ public void updateTimer(final long currentTimeMs) { } public void setInflightCommitStatus(final boolean inflightCommitStatus) { - this.cachedHasInflightCommit.set(inflightCommitStatus); + this.hasInflightCommit.set(inflightCommitStatus); } } From 9d1a93445f02efacaa5b36b4fe6411261208dc84 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 16 Aug 2025 17:31:18 -0700 Subject: [PATCH 10/31] Made AutoCommitState a top-level class and refactoring to support --- .../internals/AsyncKafkaConsumer.java | 21 +- .../consumer/internals/AutoCommitState.java | 194 ++++++++++++++++++ .../internals/CommitRequestManager.java | 95 ++------- .../consumer/internals/RequestManagers.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 5 +- .../internals/CommitRequestManagerTest.java | 13 +- .../internals/RequestManagersTest.java | 4 +- 7 files changed, 236 insertions(+), 100 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.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 5df146ac16424..f893992215c8a 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 @@ -356,9 +356,8 @@ private StreamsRebalanceListener streamsRebalanceListener() { private final long retryBackoffMs; private final int requestTimeoutMs; private final Duration defaultApiTimeoutMs; - private final boolean autoCommitEnabled; private final AtomicBoolean reconciliationInProgress = new AtomicBoolean(); - private final AtomicBoolean hasInflightCommit = new AtomicBoolean(); + private final AutoCommitState autoCommitState; private volatile boolean closed = false; // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); @@ -425,7 +424,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, GroupRebalanceConfig.ProtocolType.CONSUMER ); this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); - this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); LogContext logContext = createLogContext(config, groupRebalanceConfig); this.backgroundEventQueue = backgroundEventQueue; this.log = logContext.logger(getClass()); @@ -480,6 +478,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); + this.autoCommitState = AutoCommitState.newInstance(logContext, config, time); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -496,8 +495,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - hasInflightCommit, - reconciliationInProgress + autoCommitState, + reconciliationInProgress ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -568,7 +567,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int requestTimeoutMs, int defaultApiTimeoutMs, String groupId, - boolean autoCommitEnabled) { + AutoCommitState autoCommitState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -591,7 +590,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.applicationEventHandler = applicationEventHandler; this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics); this.clientTelemetryReporter = Optional.empty(); - this.autoCommitEnabled = autoCommitEnabled; + this.autoCommitState = autoCommitState; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, @@ -611,7 +610,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); - this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); this.fetchBuffer = new FetchBuffer(logContext); this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; this.time = time; @@ -667,6 +665,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.autoCommitState = AutoCommitState.newInstance(logContext, config, time); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, @@ -684,7 +683,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - hasInflightCommit, + autoCommitState, reconciliationInProgress ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( @@ -872,7 +871,7 @@ public ConsumerRecords poll(final Duration timeout) { // the interval time or reconciling new assignments applicationEventHandler.add(event); - if (reconciliationInProgress.get() || hasInflightCommit.get()) { + if (reconciliationInProgress.get() || autoCommitState.shouldAutoCommit()) { // 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()); @@ -1524,7 +1523,7 @@ private void autoCommitOnClose(final Timer timer) { if (groupMetadata.get().isEmpty()) return; - if (autoCommitEnabled) + if (autoCommitState.isAutoCommitEnabled()) commitSyncAllConsumed(timer); applicationEventHandler.add(new CommitOnCloseEvent()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java new file mode 100644 index 0000000000000..58c6b7b39dcb8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java @@ -0,0 +1,194 @@ +/* + * 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.ConsumerConfig; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; + +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Encapsulates the state of auto-committing and manages the auto-commit timer. + */ +public interface AutoCommitState { + + /** + * @return {@code true} if auto-commit is enabled as defined in the configuration + * {@link ConsumerConfig#ENABLE_AUTO_COMMIT_CONFIG}. + */ + boolean isAutoCommitEnabled(); + + boolean shouldAutoCommit(); + + boolean isExpired(); + + /** + * Reset the auto-commit timer to the auto-commit interval, so that the next auto-commit is + * sent out on the interval starting from now. If auto-commit is disabled this will + * perform no action. + */ + void resetTimer(); + + /** + * Reset the auto-commit timer to the provided time (backoff), so that the next auto-commit is + * sent out then. If auto-commit is disabled this will perform no action. + */ + void resetTimer(long retryBackoffMs); + + long remainingMs(final long currentTimeMs); + + void updateTimer(final long currentTimeMs); + + void setInflightCommitStatus(final boolean inflightCommitStatus); + + static AutoCommitState whenEnabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + return new AutoCommitStateEnabled(logContext, time, autoCommitInterval); + } + + static AutoCommitState whenDisabled() { + return new AutoCommitStateDisabled(); + } + + static AutoCommitState newInstance(final LogContext logContext, + final ConsumerConfig config, + final Time time) { + if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { + final long interval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + return whenEnabled(logContext, time, interval); + } else { + return whenDisabled(); + } + } + + class AutoCommitStateEnabled implements AutoCommitState { + + private final Logger log; + private final Timer timer; + private final long autoCommitInterval; + private final AtomicBoolean hasInflightCommit; + + private AutoCommitStateEnabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + this.log = logContext.logger(AutoCommitState.class); + this.timer = time.timer(autoCommitInterval); + this.autoCommitInterval = autoCommitInterval; + this.hasInflightCommit = new AtomicBoolean(); + } + + @Override + public boolean isAutoCommitEnabled() { + return true; + } + + @Override + public synchronized boolean shouldAutoCommit() { + if (!timer.isExpired()) { + return false; + } + + if (hasInflightCommit.get()) { + log.trace("Skipping auto-commit on the interval because a previous one is still in-flight."); + return false; + } + + return true; + } + + @Override + public synchronized boolean isExpired() { + return timer.isExpired(); + } + + @Override + public synchronized void resetTimer() { + timer.reset(autoCommitInterval); + } + + @Override + public synchronized void resetTimer(long retryBackoffMs) { + timer.reset(retryBackoffMs); + } + + @Override + public synchronized long remainingMs(final long currentTimeMs) { + timer.update(currentTimeMs); + return timer.remainingMs(); + } + + @Override + public synchronized void updateTimer(final long currentTimeMs) { + timer.update(currentTimeMs); + } + + @Override + public synchronized void setInflightCommitStatus(final boolean inflightCommitStatus) { + hasInflightCommit.set(inflightCommitStatus); + } + } + + class AutoCommitStateDisabled implements AutoCommitState { + + private AutoCommitStateDisabled() { + } + + @Override + public boolean isAutoCommitEnabled() { + return false; + } + + @Override + public boolean shouldAutoCommit() { + return false; + } + + @Override + public boolean isExpired() { + return false; + } + + @Override + public void resetTimer() { + // No op + } + + @Override + public void resetTimer(long retryBackoffMs) { + // No op + } + + @Override + public long remainingMs(final long currentTimeMs) { + return Long.MAX_VALUE; + } + + @Override + public void updateTimer(final long currentTimeMs) { + // No op + } + + @Override + public void setInflightCommitStatus(final boolean inflightCommitStatus) { + // No op + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 9f722e3e903e3..7ddfc52a41210 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -65,7 +65,6 @@ import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -79,7 +78,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener private final ConsumerMetadata metadata; private final LogContext logContext; private final Logger log; - private final Optional autoCommitState; + private final AutoCommitState autoCommitState; private final CoordinatorRequestManager coordinatorRequestManager; private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final OffsetCommitMetricsManager metricsManager; @@ -117,7 +116,7 @@ public CommitRequestManager( final Optional groupInstanceId, final Metrics metrics, final ConsumerMetadata metadata, - final AtomicBoolean hasInflightCommit) { + final AutoCommitState autoCommitState) { this(time, logContext, subscriptions, @@ -131,7 +130,7 @@ public CommitRequestManager( OptionalDouble.empty(), metrics, metadata, - hasInflightCommit); + autoCommitState); } // Visible for testing @@ -149,20 +148,14 @@ public CommitRequestManager( final OptionalDouble jitter, final Metrics metrics, final ConsumerMetadata metadata, - final AtomicBoolean hasInflightCommit) { + final AutoCommitState autoCommitState) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.time = time; this.logContext = logContext; this.log = logContext.logger(getClass()); this.pendingRequests = new PendingRequests(); - if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { - final long autoCommitInterval = - Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval, logContext, hasInflightCommit)); - } else { - this.autoCommitState = Optional.empty(); - } this.coordinatorRequestManager = coordinatorRequestManager; + this.autoCommitState = autoCommitState; this.groupId = groupId; this.groupInstanceId = groupInstanceId; this.subscriptions = subscriptions; @@ -182,6 +175,8 @@ public CommitRequestManager( */ @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { + autoCommitState.updateTimer(currentTimeMs); + // poll when the coordinator node is known and fatal error is not present if (coordinatorRequestManager.coordinator().isEmpty()) { pendingRequests.maybeFailOnCoordinatorFatalError(); @@ -216,7 +211,7 @@ public void signalClose() { */ @Override public long maximumTimeToWait(long currentTimeMs) { - return autoCommitState.map(ac -> ac.remainingMs(currentTimeMs)).orElse(Long.MAX_VALUE); + return autoCommitState.remainingMs(currentTimeMs); } private static long findMinTime(final Collection requests, final long currentTimeMs) { @@ -243,12 +238,11 @@ private KafkaException maybeWrapAsTimeoutException(Throwable t) { * failed. */ private CompletableFuture> requestAutoCommit(final OffsetCommitRequestState requestState) { - AutoCommitState autocommit = autoCommitState.get(); CompletableFuture> result; if (requestState.offsets.isEmpty()) { result = CompletableFuture.completedFuture(Collections.emptyMap()); } else { - autocommit.setInflightCommitStatus(true); + autoCommitState.setInflightCommitStatus(true); OffsetCommitRequestState request = pendingRequests.addOffsetCommitRequest(requestState); result = request.future; result.whenComplete(autoCommitCallback(request.offsets)); @@ -270,7 +264,7 @@ private CompletableFuture> requestAutoCom * response for the in-flight is received. */ private void maybeAutoCommitAsync() { - if (autoCommitEnabled() && autoCommitState.get().shouldAutoCommit()) { + if (autoCommitState.shouldAutoCommit()) { OffsetCommitRequestState requestState = createOffsetCommitRequest( subscriptions.allConsumed(), Long.MAX_VALUE); @@ -278,7 +272,7 @@ private void maybeAutoCommitAsync() { // Reset timer to the interval (even if no request was generated), but ensure that if // the request completes with a retriable error, the timer is reset to send the next // auto-commit after the backoff expires. - resetAutoCommitTimer(); + autoCommitState.resetTimer(); maybeResetTimerWithBackoff(result); } } @@ -291,7 +285,7 @@ private void maybeResetTimerWithBackoff(final CompletableFuture maybeAutoCommitSyncBeforeRebalance(final long deadlineMs) { - if (!autoCommitEnabled()) { + if (!autoCommitState.isAutoCommitEnabled()) { return CompletableFuture.completedFuture(null); } @@ -370,7 +364,7 @@ private void autoCommitSyncBeforeRebalanceWithRetries(OffsetCommitRequestState r */ private BiConsumer, ? super Throwable> autoCommitCallback(final Map allConsumedOffsets) { return (response, throwable) -> { - autoCommitState.ifPresent(autoCommitState -> autoCommitState.setInflightCommitStatus(false)); + autoCommitState.setInflightCommitStatus(false); if (throwable == null) { offsetCommitCallbackInvoker.enqueueInterceptorInvocation(allConsumedOffsets); log.debug("Completed auto-commit of offsets {}", allConsumedOffsets); @@ -570,7 +564,7 @@ private boolean isStaleEpochErrorAndValidEpochAvailable(Throwable error) { } private void updateAutoCommitTimer(final long currentTimeMs) { - this.autoCommitState.ifPresent(t -> t.updateTimer(currentTimeMs)); + this.autoCommitState.updateTimer(currentTimeMs); } // Visible for testing @@ -604,7 +598,7 @@ public void onMemberEpochUpdated(Optional memberEpoch, String memberId) * @return True if auto-commit is enabled as defined in the config {@link ConsumerConfig#ENABLE_AUTO_COMMIT_CONFIG} */ public boolean autoCommitEnabled() { - return autoCommitState.isPresent(); + return autoCommitState.isAutoCommitEnabled(); } /** @@ -613,7 +607,7 @@ public boolean autoCommitEnabled() { * perform no action. */ public void resetAutoCommitTimer() { - autoCommitState.ifPresent(AutoCommitState::resetTimer); + autoCommitState.resetTimer(); } /** @@ -621,7 +615,7 @@ public void resetAutoCommitTimer() { * sent out then. If auto-commit is not enabled this will perform no action. */ public void resetAutoCommitTimer(long retryBackoffMs) { - autoCommitState.ifPresent(s -> s.resetTimer(retryBackoffMs)); + autoCommitState.resetTimer(retryBackoffMs); } /** @@ -1305,59 +1299,6 @@ private void maybeFailOnCoordinatorFatalError() { } } - /** - * Encapsulates the state of auto-committing and manages the auto-commit timer. - */ - private static class AutoCommitState { - private final Timer timer; - private final long autoCommitInterval; - private final AtomicBoolean hasInflightCommit; - private final Logger log; - - public AutoCommitState( - final Time time, - final long autoCommitInterval, - final LogContext logContext, - final AtomicBoolean hasInflightCommit) { - this.autoCommitInterval = autoCommitInterval; - this.timer = time.timer(autoCommitInterval); - this.hasInflightCommit = hasInflightCommit; - this.log = logContext.logger(getClass()); - } - - public boolean shouldAutoCommit() { - if (!this.timer.isExpired()) { - return false; - } - if (this.hasInflightCommit.get()) { - log.trace("Skipping auto-commit on the interval because a previous one is still in-flight."); - return false; - } - return true; - } - - public void resetTimer() { - this.timer.reset(autoCommitInterval); - } - - public void resetTimer(long retryBackoffMs) { - this.timer.reset(retryBackoffMs); - } - - public long remainingMs(final long currentTimeMs) { - this.timer.update(currentTimeMs); - return this.timer.remainingMs(); - } - - public void updateTimer(final long currentTimeMs) { - this.timer.update(currentTimeMs); - } - - public void setInflightCommitStatus(final boolean inflightCommitStatus) { - this.hasInflightCommit.set(inflightCommitStatus); - } - } - static class MemberInfo { String memberId = ""; Optional memberEpoch = Optional.empty(); 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 aa604529d14e6..f61dacb4257b9 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 @@ -169,7 +169,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final AtomicBoolean cachedHasInflightCommit, + final AutoCommitState autoCommitState, final AtomicBoolean cachedReconciliationInProgress ) { return new CachedSupplier<>() { @@ -220,7 +220,7 @@ protected RequestManagers create() { groupRebalanceConfig.groupInstanceId, metrics, metadata, - cachedHasInflightCommit); + autoCommitState); if (streamsRebalanceData.isPresent()) { streamsMembershipManager = new StreamsMembershipManager( groupRebalanceConfig.groupId, 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 7eb5e631b1e5b..0e03e3cc3f9a5 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 @@ -257,8 +257,9 @@ private AsyncKafkaConsumer newConsumer( long retryBackoffMs = 100L; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; + LogContext logContext = new LogContext(); return new AsyncKafkaConsumer<>( - new LogContext(), + logContext, clientId, new Deserializers<>(new StringDeserializer(), new StringDeserializer(), metrics), fetchBuffer, @@ -276,7 +277,7 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, groupId, - autoCommitEnabled); + autoCommitEnabled ? AutoCommitState.whenEnabled(logContext, time, 5000) : AutoCommitState.whenDisabled()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index e2bb28d4a59ff..f133843087a40 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -71,7 +71,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -118,7 +117,6 @@ public class CommitRequestManagerTest { private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final Metrics metrics = new Metrics(); private Properties props; - private final AtomicBoolean cachedHasInflightCommit = new AtomicBoolean(); private final int defaultApiTimeoutMs = 60000; @@ -139,7 +137,7 @@ public void setup() { @Test public void testOffsetFetchRequestStateToStringBase() { ConsumerConfig config = mock(ConsumerConfig.class); - + AutoCommitState autoCommitState = AutoCommitState.newInstance(logContext, config, time); CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, @@ -154,7 +152,7 @@ public void testOffsetFetchRequestStateToStringBase() { OptionalDouble.of(0), metrics, metadata, - cachedHasInflightCommit); + autoCommitState); commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString()); Set requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1)); @@ -1567,11 +1565,14 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long if (autoCommitEnabled) props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); + ConsumerConfig config = new ConsumerConfig(props); + AutoCommitState autoCommitState = AutoCommitState.newInstance(logContext, config, time); + return spy(new CommitRequestManager( this.time, this.logContext, this.subscriptionState, - new ConsumerConfig(props), + config, this.coordinatorRequestManager, this.offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, @@ -1581,7 +1582,7 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long OptionalDouble.of(0), metrics, metadata, - cachedHasInflightCommit)); + autoCommitState)); } private ClientResponse buildOffsetFetchClientResponse( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 678775000ecfa..90a231e687f82 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -68,7 +68,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - new AtomicBoolean(), + AutoCommitState.whenDisabled(), new AtomicBoolean() ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); @@ -110,7 +110,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - new AtomicBoolean(), + AutoCommitState.whenDisabled(), new AtomicBoolean() ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); From 42421273ce568de093ea18d93380de3e4209cbbf Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 16 Aug 2025 17:42:08 -0700 Subject: [PATCH 11/31] [WIP] Clean up --- .../clients/consumer/ConsumerConfig.java | 3 ++- .../internals/AbstractMembershipManager.java | 22 +++++++++---------- .../consumer/internals/AutoCommitState.java | 18 ++++++++++----- .../internals/ConsumerMembershipManager.java | 8 +++---- .../consumer/internals/RequestManagers.java | 2 +- .../internals/ShareMembershipManager.java | 2 +- .../internals/AsyncKafkaConsumerTest.java | 2 +- .../ConsumerMembershipManagerTest.java | 6 ++--- .../internals/RequestManagersTest.java | 4 ++-- 9 files changed, 37 insertions(+), 30 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 3fcdf20953cba..81ef26d2a15ec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -145,6 +145,7 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true."; + public static final int DEFAULT_AUTO_COMMIT_INTERVAL_MS = 5000; /** * partition.assignment.strategy @@ -462,7 +463,7 @@ public class ConsumerConfig extends AbstractConfig { ENABLE_AUTO_COMMIT_DOC) .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.INT, - 5000, + DEFAULT_AUTO_COMMIT_INTERVAL_MS, atLeast(0), Importance.LOW, AUTO_COMMIT_INTERVAL_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 683b1f94c3a85..9f01e76f63391 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -135,6 +135,13 @@ public abstract class AbstractMembershipManager impl */ private LocalAssignment currentTargetAssignment; + /** + * If there is a reconciliation running (triggering commit, callbacks) for the + * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered + * after receiving a heartbeat response, or a metadata update. + */ + private final AtomicBoolean reconciliationInProgress; + /** * True if a reconciliation is in progress and the member rejoined the group since the start * of the reconciliation. Used to know that the reconciliation in progress should be @@ -171,13 +178,6 @@ public abstract class AbstractMembershipManager impl private final Time time; - /** - * If there is a reconciliation running (triggering commit, callbacks) for the - * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered - * after receiving a heartbeat response, or a metadata update. - */ - private final AtomicBoolean reconciliationInProgress; - /** * AtomicBoolean to track whether the subscription is updated. * If it's true and subscription state is UNSUBSCRIBED, the next {@link #onConsumerPoll()} will change member state to JOINING. @@ -192,7 +192,7 @@ public abstract class AbstractMembershipManager impl */ private boolean isPollTimerExpired; - private final boolean autoCommitEnabled; + private final AutoCommitState autoCommitState; /** * Indicate the operation on consumer group membership that the consumer will perform when leaving the group. @@ -208,7 +208,7 @@ public abstract class AbstractMembershipManager impl Logger log, Time time, RebalanceMetricsManager metricsManager, - boolean autoCommitEnabled, + AutoCommitState autoCommitState, AtomicBoolean reconciliationInProgress) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; @@ -221,7 +221,7 @@ public abstract class AbstractMembershipManager impl this.stateUpdatesListeners = new ArrayList<>(); this.time = time; this.metricsManager = metricsManager; - this.autoCommitEnabled = autoCommitEnabled; + this.autoCommitState = autoCommitState; this.reconciliationInProgress = reconciliationInProgress; } @@ -852,7 +852,7 @@ public void maybeReconcile(boolean canCommit) { return; } - if (autoCommitEnabled && !canCommit) return; + if (autoCommitState.isAutoCommitEnabled() && !canCommit) return; markReconciliationInProgress(); // Keep copy of assigned TopicPartitions created from the TopicIdPartitions that are diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java index 58c6b7b39dcb8..e281b721b75ef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java @@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.kafka.clients.consumer.ConsumerConfig.DEFAULT_AUTO_COMMIT_INTERVAL_MS; + /** * Encapsulates the state of auto-committing and manages the auto-commit timer. */ @@ -58,13 +60,17 @@ public interface AutoCommitState { void setInflightCommitStatus(final boolean inflightCommitStatus); - static AutoCommitState whenEnabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { + static AutoCommitState enabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { return new AutoCommitStateEnabled(logContext, time, autoCommitInterval); } - static AutoCommitState whenDisabled() { + static AutoCommitState enabled(final LogContext logContext, final Time time) { + return enabled(logContext, time, DEFAULT_AUTO_COMMIT_INTERVAL_MS); + } + + static AutoCommitState disabled() { return new AutoCommitStateDisabled(); } @@ -73,9 +79,9 @@ static AutoCommitState newInstance(final LogContext logContext, final Time time) { if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long interval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - return whenEnabled(logContext, time, interval); + return enabled(logContext, time, interval); } else { - return whenDisabled(); + return disabled(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java index eed2b32cd8cc9..520088decfa8d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java @@ -152,7 +152,7 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, Metrics metrics, - boolean autoCommitEnabled, + AutoCommitState autoCommitState, AtomicBoolean reconciliationInProgress) { this(groupId, groupInstanceId, @@ -166,7 +166,7 @@ public ConsumerMembershipManager(String groupId, backgroundEventHandler, time, new ConsumerRebalanceMetricsManager(metrics), - autoCommitEnabled, + autoCommitState, reconciliationInProgress); } @@ -183,7 +183,7 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, RebalanceMetricsManager metricsManager, - boolean autoCommitEnabled, + AutoCommitState autoCommitState, AtomicBoolean reconciliationInProgress) { super(groupId, subscriptions, @@ -191,7 +191,7 @@ public ConsumerMembershipManager(String groupId, logContext.logger(ConsumerMembershipManager.class), time, metricsManager, - autoCommitEnabled, + autoCommitState, reconciliationInProgress); this.groupInstanceId = groupInstanceId; this.rackId = rackId; 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 f61dacb4257b9..efab511324725 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 @@ -262,7 +262,7 @@ protected RequestManagers create() { backgroundEventHandler, time, metrics, - config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), + autoCommitState, cachedReconciliationInProgress); // Update the group member ID label in the client telemetry reporter. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java index ba15c2c74e486..75f4cc5e6fc9b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java @@ -108,7 +108,7 @@ public ShareMembershipManager(LogContext logContext, logContext.logger(ShareMembershipManager.class), time, metricsManager, - false, + AutoCommitState.disabled(), new AtomicBoolean()); this.rackId = rackId; } 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 0e03e3cc3f9a5..646b8182c34bd 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 @@ -277,7 +277,7 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, groupId, - autoCommitEnabled ? AutoCommitState.whenEnabled(logContext, time, 5000) : AutoCommitState.whenDisabled()); + autoCommitEnabled ? AutoCommitState.enabled(logContext, time) : AutoCommitState.disabled()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index efba723399f3d..57c9d6d0ea1fc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -145,7 +145,7 @@ private ConsumerMembershipManager createMembershipManager(String groupInstanceId ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean())); + backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean())); assertMemberIdIsGenerated(manager.memberId()); return manager; } @@ -158,7 +158,7 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(rackId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean())); + metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean())); assertMemberIdIsGenerated(manager.memberId()); manager.transitionToJoining(); return manager; @@ -246,7 +246,7 @@ public void testTransitionToFailedWhenTryingToJoin() { ConsumerMembershipManager membershipManager = new ConsumerMembershipManager( GROUP_ID, Optional.empty(), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, true, new AtomicBoolean()); + backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean()); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 90a231e687f82..9b2ce81801972 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -68,7 +68,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - AutoCommitState.whenDisabled(), + AutoCommitState.disabled(), new AtomicBoolean() ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); @@ -110,7 +110,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - AutoCommitState.whenDisabled(), + AutoCommitState.disabled(), new AtomicBoolean() ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); From e26b71c8240ff73d5d8592f3d7618ade29195fda Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sat, 16 Aug 2025 17:59:49 -0700 Subject: [PATCH 12/31] Revert changes to CommitRequestManager to fix failures in CommitRequestManagerTest --- .../clients/consumer/internals/CommitRequestManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 7ddfc52a41210..7dde1ea90a6c5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -154,8 +154,8 @@ public CommitRequestManager( this.logContext = logContext; this.log = logContext.logger(getClass()); this.pendingRequests = new PendingRequests(); - this.coordinatorRequestManager = coordinatorRequestManager; this.autoCommitState = autoCommitState; + this.coordinatorRequestManager = coordinatorRequestManager; this.groupId = groupId; this.groupInstanceId = groupInstanceId; this.subscriptions = subscriptions; @@ -272,7 +272,7 @@ private void maybeAutoCommitAsync() { // Reset timer to the interval (even if no request was generated), but ensure that if // the request completes with a retriable error, the timer is reset to send the next // auto-commit after the backoff expires. - autoCommitState.resetTimer(); + resetAutoCommitTimer(); maybeResetTimerWithBackoff(result); } } @@ -285,7 +285,7 @@ private void maybeResetTimerWithBackoff(final CompletableFuture Date: Fri, 22 Aug 2025 17:04:05 -0700 Subject: [PATCH 13/31] Remove unused state listener logic from CommitOffsetsSharedState Eliminates the MemberStateListener implementation and related state caching from CommitOffsetsSharedState, simplifying its logic. Updates OffsetsRequestManager, RequestManagers, and associated tests to reflect the removal of this listener and its state management. --- .../internals/CommitOffsetsSharedState.java | 54 +++++-------------- .../internals/OffsetsRequestManager.java | 3 -- .../consumer/internals/RequestManagers.java | 1 - .../internals/RequestManagersTest.java | 2 +- 4 files changed, 13 insertions(+), 47 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java index 94b4cd027a0de..aa7693cdf2f74 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java @@ -19,14 +19,11 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import java.time.Duration; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicReference; /** @@ -41,12 +38,7 @@ * sparingly in both the application and background threads. Both of those classes are heavily synchronized given * their use by the {@link ClassicKafkaConsumer}, so their use in a multithreaded fashion is already established. */ -public class CommitOffsetsSharedState implements MemberStateListener { - - /** - * To keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates. - */ - private final AtomicBoolean cachedSubscriptionHasAllFetchPositions = new AtomicBoolean(); +public class CommitOffsetsSharedState { /** * Exception that occurred while updating positions after the triggering event had already @@ -82,11 +74,7 @@ void setCachedUpdatePositionsException(Throwable exception) { } boolean subscriptionHasAllFetchPositions() { - return cachedSubscriptionHasAllFetchPositions.get(); - } - - void setSubscriptionHasAllFetchPositions(boolean value) { - cachedSubscriptionHasAllFetchPositions.set(value); + return subscriptions.hasAllFetchPositions(); } /** @@ -106,13 +94,6 @@ void setSubscriptionHasAllFetchPositions(boolean value) { * ({@code OffsetsRequestManager#cacheExceptionIfEventExpired()}) * *
  • - * Checks that the previously cached version of {@link #cachedSubscriptionHasAllFetchPositions} is still - * {@code true}. This covers a couple additional cases (like before first assignment), so although it's - * not completely optimal, the fallback is we force the check which will block the application thread - * while the background thread performs its checks which may result in the value being set back to - * {@code true} before the application thread resumes. - *
  • - *
  • * Checks that there are no positions in the {@link SubscriptionState.FetchStates#AWAIT_VALIDATION} * state ({@link OffsetFetcherUtils#getPartitionsToValidate()}) *
  • @@ -129,34 +110,23 @@ void setSubscriptionHasAllFetchPositions(boolean value) { * @return true if all checks pass, false if either of the latter two checks fail */ boolean canSkipUpdateFetchPositions() { - Throwable exception = getAndClearCachedUpdatePositionsException(); + Throwable exception = cachedUpdatePositionsException.get(); + + if (exception != null) { + // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions + // from the call to Future.get(). + if (exception instanceof CompletionException) + exception = exception.getCause(); - if (exception != null) throw ConsumerUtils.maybeWrapAsKafkaException(exception); + } // If the cached value is set and there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or // INITIALIZING states, it's ok to skip. - if (cachedSubscriptionHasAllFetchPositions.get() && offsetFetcherUtils.getPartitionsToValidate().isEmpty() && subscriptions.hasAllFetchPositions()) - return true; - - // However, if even one partition is not set to FETCHING, set the cached value to false and signal the - // application thread that it should call the background thread to perform the necessary logic to resolve the - // state of the partitions. - setSubscriptionHasAllFetchPositions(false); - return false; + return offsetFetcherUtils.getPartitionsToValidate().isEmpty() && subscriptions.hasAllFetchPositions(); } OffsetFetcherUtils offsetFetcherUtils() { return offsetFetcherUtils; } - - @Override - public void onMemberEpochUpdated(Optional memberEpoch, String memberId) { - // Ignore... - } - - @Override - public void onGroupAssignmentUpdated(Set partitions) { - setSubscriptionHasAllFetchPositions(false); - } } 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 a4cfcc6c8023f..b6ed925fc41fe 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 @@ -237,11 +237,8 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (subscriptionState.hasAllFetchPositions()) { // All positions are already available - commitOffsetsSharedState.setSubscriptionHasAllFetchPositions(true); result.complete(true); return result; - } else { - commitOffsetsSharedState.setSubscriptionHasAllFetchPositions(false); } // Some positions are missing, so trigger requests to fetch offsets and update them. 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 4401d82de8094..3a2d5c7a093f0 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 @@ -269,7 +269,6 @@ protected RequestManagers create() { .updateMetricsLabels(Map.of(ClientTelemetryProvider.GROUP_MEMBER_ID, membershipManager.memberId())); } - membershipManager.registerStateListener(commitOffsetsSharedState); membershipManager.registerStateListener(commitRequestManager); membershipManager.registerStateListener(applicationThreadMemberStateListener); heartbeatRequestManager = new ConsumerHeartbeatRequestManager( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index cc4339ec42ac6..a3d41eb4fd4c6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -79,7 +79,7 @@ public void testMemberStateListenerRegistered() { assertTrue(requestManagers.streamsMembershipManager.isEmpty()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isEmpty()); - assertEquals(3, requestManagers.consumerMembershipManager.get().stateListeners().size()); + assertEquals(2, requestManagers.consumerMembershipManager.get().stateListeners().size()); assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().stream() .anyMatch(m -> m instanceof CommitRequestManager)); assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().contains(listener)); From 336213a1a1c0f1220c9c67c4c1ec1fa717965a0c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 25 Aug 2025 17:49:41 -0700 Subject: [PATCH 14/31] Refactored so that NetworkClientDelegate's metadata error was available to the shared consumer state object --- .../internals/AsyncKafkaConsumer.java | 64 ++++++----- .../internals/ConsumerNetworkThread.java | 9 +- .../internals/NetworkClientDelegate.java | 45 ++++---- .../internals/OffsetsRequestManager.java | 21 +--- .../consumer/internals/RequestManagers.java | 4 +- .../consumer/internals/ShareConsumerImpl.java | 29 ++++- ...redState.java => SharedConsumerState.java} | 107 +++++++++++------- .../internals/SharedErrorReference.java | 44 +++++++ .../consumer/internals/SharedReference.java | 73 ++++++++++++ .../events/ApplicationEventHandler.java | 8 +- .../ApplicationEventHandlerTest.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 10 +- .../internals/ConsumerNetworkThreadTest.java | 14 ++- .../internals/FetchRequestManagerTest.java | 4 +- .../internals/NetworkClientDelegateTest.java | 13 ++- .../internals/OffsetsRequestManagerTest.java | 7 +- .../internals/RequestManagersTest.java | 4 +- .../ShareConsumeRequestManagerTest.java | 4 +- .../internals/ShareConsumerImplTest.java | 2 +- 19 files changed, 322 insertions(+), 144 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{CommitOffsetsSharedState.java => SharedConsumerState.java} (60%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.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 9f5143d447274..bd54977477abb 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 @@ -360,7 +360,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - private final CommitOffsetsSharedState commitOffsetsSharedState; + private final SharedConsumerState sharedConsumerState; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -462,6 +462,14 @@ public AsyncKafkaConsumer(final ConsumerConfig config, // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); + this.sharedConsumerState = new SharedConsumerState( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + apiVersions + ); final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier(time, logContext, metadata, @@ -472,18 +480,11 @@ public AsyncKafkaConsumer(final ConsumerConfig config, clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, false, - kafkaConsumerMetrics + kafkaConsumerMetrics, + sharedConsumerState ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); - this.commitOffsetsSharedState = new CommitOffsetsSharedState( - logContext, - metadata, - subscriptions, - time, - retryBackoffMs, - apiVersions - ); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -500,7 +501,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - commitOffsetsSharedState + sharedConsumerState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -514,7 +515,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, - kafkaConsumerMetrics + kafkaConsumerMetrics, + sharedConsumerState ); this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( logContext, @@ -572,7 +574,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int defaultApiTimeoutMs, String groupId, boolean autoCommitEnabled, - CommitOffsetsSharedState commitOffsetsSharedState) { + SharedConsumerState sharedConsumerState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -602,7 +604,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, kafkaConsumerMetrics ); - this.commitOffsetsSharedState = commitOffsetsSharedState; + this.sharedConsumerState = sharedConsumerState; } AsyncKafkaConsumer(LogContext logContext, @@ -661,6 +663,14 @@ public AsyncKafkaConsumer(final ConsumerConfig config, new RebalanceCallbackMetricsManager(metrics) ); ApiVersions apiVersions = new ApiVersions(); + this.sharedConsumerState = new SharedConsumerState( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + apiVersions + ); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( time, config, @@ -669,17 +679,10 @@ public AsyncKafkaConsumer(final ConsumerConfig config, metadata, backgroundEventHandler, false, - kafkaConsumerMetrics + kafkaConsumerMetrics, + sharedConsumerState ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.commitOffsetsSharedState = new CommitOffsetsSharedState( - logContext, - metadata, - subscriptions, - time, - retryBackoffMs, - apiVersions - ); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, @@ -697,7 +700,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - commitOffsetsSharedState + sharedConsumerState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -712,7 +715,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, - kafkaConsumerMetrics); + kafkaConsumerMetrics, + sharedConsumerState); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = new CompletableEventReaper(logContext); } @@ -728,7 +732,8 @@ ApplicationEventHandler build( final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, - final AsyncConsumerMetrics asyncConsumerMetrics + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState ); } @@ -1808,10 +1813,7 @@ private Fetch pollForFetches(Timer timer) { // We do not want to be stuck blocking in poll if we are missing some positions // since the offset lookup may be backing off after a failure - - // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call - // updateAssignmentMetadataIfNeeded before this method. - if (!commitOffsetsSharedState.subscriptionHasAllFetchPositions() && pollTimeout > retryBackoffMs) { + if (!subscriptions.hasAllFetchPositions() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } @@ -1866,7 +1868,7 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { - if (commitOffsetsSharedState.canSkipUpdateFetchPositions()) + if (sharedConsumerState.canSkipUpdateFetchPositions()) return true; try { 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..3a1413ff5efff 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 @@ -63,6 +63,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; private final AsyncConsumerMetrics asyncConsumerMetrics; + private final SharedErrorReference metadataError; private ApplicationEventProcessor applicationEventProcessor; private NetworkClientDelegate networkClientDelegate; private RequestManagers requestManagers; @@ -79,7 +80,8 @@ public ConsumerNetworkThread(LogContext logContext, Supplier applicationEventProcessorSupplier, Supplier networkClientDelegateSupplier, Supplier requestManagersSupplier, - AsyncConsumerMetrics asyncConsumerMetrics) { + AsyncConsumerMetrics asyncConsumerMetrics, + SharedConsumerState sharedConsumerState) { super(BACKGROUND_THREAD_NAME, true); this.time = time; this.log = logContext.logger(getClass()); @@ -90,6 +92,7 @@ public ConsumerNetworkThread(LogContext logContext, this.requestManagersSupplier = requestManagersSupplier; this.running = true; this.asyncConsumerMetrics = asyncConsumerMetrics; + this.metadataError = sharedConsumerState.metadataError(); } @Override @@ -378,8 +381,8 @@ private void maybeFailOnMetadataError(List> events) { if (subscriptionMetadataEvent.isEmpty()) return; - networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError -> - subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) + metadataError.getClearAndRun(e -> + subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(e)) ); } } 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..9fc936f124960 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,19 +70,19 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private Optional metadataError; + private final SharedErrorReference metadataError; private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; - public NetworkClientDelegate( - final Time time, - final ConsumerConfig config, - final LogContext logContext, - final KafkaClient client, - final Metadata metadata, - final BackgroundEventHandler backgroundEventHandler, - final boolean notifyMetadataErrorsViaErrorQueue, - final AsyncConsumerMetrics asyncConsumerMetrics) { + public NetworkClientDelegate(final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client, + final Metadata metadata, + final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState) { this.time = time; this.client = client; this.metadata = metadata; @@ -91,7 +91,7 @@ public NetworkClientDelegate( this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - this.metadataError = Optional.empty(); + this.metadataError = sharedConsumerState.metadataError(); this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -163,7 +163,7 @@ private void maybePropagateMetadataError() { if (notifyMetadataErrorsViaErrorQueue) { backgroundEventHandler.add(new ErrorEvent(e)); } else { - metadataError = Optional.of(e); + metadataError.set(e); } } } @@ -247,12 +247,6 @@ 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(); @@ -453,7 +447,8 @@ public static Supplier supplier(final Time time, final ClientTelemetrySender clientTelemetrySender, final BackgroundEventHandler backgroundEventHandler, final boolean notifyMetadataErrorsViaErrorQueue, - final AsyncConsumerMetrics asyncConsumerMetrics) { + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState) { return new CachedSupplier<>() { @Override protected NetworkClientDelegate create() { @@ -467,7 +462,17 @@ 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, + notifyMetadataErrorsViaErrorQueue, + asyncConsumerMetrics, + sharedConsumerState + ); } }; } 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 b6ed925fc41fe..1394ecc664b71 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 @@ -92,7 +92,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; - private final CommitOffsetsSharedState commitOffsetsSharedState; + private final SharedErrorReference updatePositionsError; /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update @@ -112,7 +112,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final CommitRequestManager commitRequestManager, - final CommitOffsetsSharedState commitOffsetsSharedState, + final SharedConsumerState sharedConsumerState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -133,13 +133,13 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; - this.offsetFetcherUtils = commitOffsetsSharedState.offsetFetcherUtils(); + this.offsetFetcherUtils = sharedConsumerState.offsetFetcherUtils(); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); this.commitRequestManager = commitRequestManager; - this.commitOffsetsSharedState = commitOffsetsSharedState; + this.updatePositionsError = sharedConsumerState.updatePositionsError(); } private static class PendingFetchCommittedRequest { @@ -229,7 +229,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { CompletableFuture result = new CompletableFuture<>(); try { - if (maybeCompleteWithPreviousException(result)) { + if (updatePositionsError.getClearAndRun(result::completeExceptionally)) { return result; } @@ -256,15 +256,6 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { return result; } - private boolean maybeCompleteWithPreviousException(CompletableFuture result) { - Throwable cachedException = commitOffsetsSharedState.getAndClearCachedUpdatePositionsException(); - 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 @@ -312,7 +303,7 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - commitOffsetsSharedState.setCachedUpdatePositionsException(error); + updatePositionsError.set(error); } }); } 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 3a2d5c7a093f0..f6d6c8606e514 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 @@ -168,7 +168,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final CommitOffsetsSharedState commitOffsetsSharedState + final SharedConsumerState sharedConsumerState ) { return new CachedSupplier<>() { @Override @@ -292,7 +292,7 @@ protected RequestManagers create() { apiVersions, networkClientDelegate, commitRequestManager, - commitOffsetsSharedState, + sharedConsumerState, logContext); return new RequestManagers( 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 32663249e7a1e..57d04479b9551 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 @@ -268,6 +268,14 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new ShareFetchBuffer(logContext); + SharedConsumerState sharedConsumerState = new SharedConsumerState( + logContext, + metadata, + subscriptions, + time, + groupRebalanceConfig.retryBackoffMs, + apiVersions + ); final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier( time, logContext, @@ -279,7 +287,8 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null), backgroundEventHandler, true, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState ); this.completedAcknowledgements = new LinkedList<>(); @@ -311,7 +320,8 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, - asyncConsumerMetrics); + asyncConsumerMetrics, + sharedConsumerState); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); @@ -382,8 +392,15 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, time, asyncConsumerMetrics); + SharedConsumerState sharedConsumerState = new SharedConsumerState( + logContext, + metadata, + subscriptions, + time, + config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG) + ); final Supplier networkClientDelegateSupplier = - () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics); + () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics, sharedConsumerState); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, @@ -417,7 +434,8 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, - asyncConsumerMetrics); + asyncConsumerMetrics, + sharedConsumerState); this.backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventProcessor = new BackgroundEventProcessor(); @@ -481,7 +499,8 @@ ApplicationEventHandler build( final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, - final AsyncConsumerMetrics asyncConsumerMetrics + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState ); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java similarity index 60% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java index aa7693cdf2f74..7891b60763ea6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitOffsetsSharedState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java @@ -23,37 +23,69 @@ import org.apache.kafka.common.utils.Time; import java.time.Duration; -import java.util.concurrent.CompletionException; -import java.util.concurrent.atomic.AtomicReference; + +import static java.util.Objects.requireNonNull; /** * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the - * background thread ({@link OffsetsRequestManager}) to determine if a costly call to check offsets can be skipped - * inside {@link Consumer#poll(Duration)}. + * background thread ({@link OffsetsRequestManager}) to avoid costly inter-thread communication, where possible. + * This class compromises on the ideal of keeping state only in the background thread. However, this class only + * relies on classes which are designed to be multithread-safe, thus they can be used in both the application and + * background threads. * *

    * - * This class compromises on the ideal of keeping the state only in the background thread. However, this class only - * relies on the {@link SubscriptionState} and {@link ConsumerMetadata} which are, unfortunately, already used - * sparingly in both the application and background threads. Both of those classes are heavily synchronized given - * their use by the {@link ClassicKafkaConsumer}, so their use in a multithreaded fashion is already established. + * The following thread-safe classes are used by this class: + * + *

      + *
    • {@link ApiVersions}
    • + *
    • {@link ConsumerMetadata}
    • + *
    • {@link OffsetFetcherUtils}
    • + *
    • {@link SharedErrorReference}
    • + *
    • {@link SubscriptionState}
    • + *
    • {@link Time}
    • + *
    + * + *

    + * + * In general, callers from the application thread should not mutate any of the state contained within this class. + * It should be considered as read-only, and only the network thread should mutate the state. */ -public class CommitOffsetsSharedState { +public class SharedConsumerState { - /** - * 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<>(); - private final OffsetFetcherUtils offsetFetcherUtils; private final SubscriptionState subscriptions; + private final OffsetFetcherUtils offsetFetcherUtils; + private final SharedErrorReference updatePositionsError; + private final SharedErrorReference metadataError; + + public SharedConsumerState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs) { + this( + logContext, + metadata, + subscriptions, + time, + retryBackoffMs, + new ApiVersions() + ); + } + + public SharedConsumerState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs, + ApiVersions apiVersions) { + requireNonNull(logContext); + requireNonNull(metadata); + requireNonNull(subscriptions); + requireNonNull(time); + requireNonNull(apiVersions); - CommitOffsetsSharedState(LogContext logContext, - ConsumerMetadata metadata, - SubscriptionState subscriptions, - Time time, - long retryBackoffMs, - ApiVersions apiVersions) { + this.subscriptions = subscriptions; this.offsetFetcherUtils = new OffsetFetcherUtils( logContext, metadata, @@ -62,19 +94,20 @@ public class CommitOffsetsSharedState { retryBackoffMs, apiVersions ); - this.subscriptions = subscriptions; + this.updatePositionsError = new SharedErrorReference(); + this.metadataError = new SharedErrorReference(); } - Throwable getAndClearCachedUpdatePositionsException() { - return cachedUpdatePositionsException.getAndSet(null); + OffsetFetcherUtils offsetFetcherUtils() { + return offsetFetcherUtils; } - void setCachedUpdatePositionsException(Throwable exception) { - cachedUpdatePositionsException.set(exception); + public SharedErrorReference updatePositionsError() { + return updatePositionsError; } - boolean subscriptionHasAllFetchPositions() { - return subscriptions.hasAllFetchPositions(); + public SharedErrorReference metadataError() { + return metadataError; } /** @@ -109,24 +142,12 @@ boolean subscriptionHasAllFetchPositions() { * * @return true if all checks pass, false if either of the latter two checks fail */ - boolean canSkipUpdateFetchPositions() { - Throwable exception = cachedUpdatePositionsException.get(); - - if (exception != null) { - // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions - // from the call to Future.get(). - if (exception instanceof CompletionException) - exception = exception.getCause(); - - throw ConsumerUtils.maybeWrapAsKafkaException(exception); - } + public boolean canSkipUpdateFetchPositions() { + updatePositionsError.maybeThrowException(); + metadataError.maybeThrowException(); // If the cached value is set and there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or // INITIALIZING states, it's ok to skip. return offsetFetcherUtils.getPartitionsToValidate().isEmpty() && subscriptions.hasAllFetchPositions(); } - - OffsetFetcherUtils offsetFetcherUtils() { - return offsetFetcherUtils; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java new file mode 100644 index 0000000000000..9d16e8836e058 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java @@ -0,0 +1,44 @@ +/* + * 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 java.util.concurrent.CompletionException; + +/** + * {@code SharedErrorReference} builds on top of {@link SharedReference} both to be more explicit about the contents + * and to provide utility methods. + */ +public class SharedErrorReference extends SharedReference { + + /** + * If the underlying error is present, this will throw the error. Note: if the exception is wrapped in a + * {@link CompletionException}, it will be unwrapped. However, if the underlying error is not a subclass + * of {@link KafkaException}, it will be wrapped as such so that it is an unchecked exception. + */ + public void maybeThrowException() { + ifPresent(exception -> { + // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions + // from the call to Future.get(). + if (exception instanceof CompletionException) + exception = exception.getCause(); + + throw ConsumerUtils.maybeWrapAsKafkaException(exception); + }); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java new file mode 100644 index 0000000000000..8d77f29cd44c8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java @@ -0,0 +1,73 @@ +/* + * 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 java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +/** + * {@code SharedReference} serves as a primitive around a {@link AtomicReference} that provides utility methods + * for more ergonomic use. + * @param + */ +public class SharedReference { + + private final AtomicReference reference = new AtomicReference<>(); + + /** + * Thin wrapper around {@link AtomicReference#get()} that provides a null-safe API via {@link Optional}. + */ + public Optional get() { + return Optional.ofNullable(reference.get()); + } + + /** + * If the underlying reference is nonnull, the given {@link Consumer action} is invoked. + */ + public void ifPresent(Consumer action) { + get().ifPresent(action); + } + + /** + * Thin wrapper around {@link AtomicReference#getAndSet(Object)} that provides a null-safe API via {@link Optional}. + */ + public Optional getAndClear() { + return Optional.ofNullable(reference.getAndSet(null)); + } + + /** + * Wrapper around {@link #getAndClear()} and {@link Optional#ifPresent(Consumer)} that retrieves and clears out + * the underlying reference in a single, atomic operation, and then invokes the given {@link Consumer} if the + * value was present. Lastly, it returns the present/empty flag so that the caller can short-circuit with less + * boilerplate. + */ + public boolean getClearAndRun(Consumer action) { + Optional value = getAndClear(); + + if (value.isPresent()) { + action.accept(value.get()); + return true; + } else { + return false; + } + } + + public void set(T value) { + reference.set(value); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java index 6ab827b617c19..fd659b7c7d4a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; +import org.apache.kafka.clients.consumer.internals.SharedConsumerState; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.internals.IdempotentCloser; @@ -56,7 +57,8 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, - final AsyncConsumerMetrics asyncConsumerMetrics) { + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState) { this.log = logContext.logger(ApplicationEventHandler.class); this.time = time; this.applicationEventQueue = applicationEventQueue; @@ -68,7 +70,9 @@ public ApplicationEventHandler(final LogContext logContext, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, - asyncConsumerMetrics); + asyncConsumerMetrics, + sharedConsumerState + ); this.networkThread.start(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java index 3430719b16ee6..b84395abf30d2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java @@ -43,6 +43,7 @@ public class ApplicationEventHandlerTest { private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private final RequestManagers requestManagers = mock(RequestManagers.class); private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); + private final SharedConsumerState sharedConsumerState = mock(SharedConsumerState.class); @Test public void testRecordApplicationEventQueueSize() { @@ -56,7 +57,8 @@ public void testRecordApplicationEventQueueSize() { () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState )) { // add event applicationEventHandler.add(new PollEvent(time.milliseconds())); 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 0e3a6e79089d3..235032086ea9e 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NodeApiVersions; @@ -223,7 +222,7 @@ private AsyncKafkaConsumer newConsumerWithStreamRebalanceData( new StringDeserializer(), new StringDeserializer(), time, - (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics) -> applicationEventHandler, + (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, metadataError) -> applicationEventHandler, logContext -> backgroundEventReaper, (logContext, consumerMetadata, subscriptionState, fetchConfig, deserializers, fetchMetricsManager, time) -> fetchCollector, (consumerConfig, subscriptionState, logContext, clusterResourceListeners) -> metadata, @@ -238,7 +237,7 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics) -> applicationEventHandler, + (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, metadataError) -> applicationEventHandler, logContext -> backgroundEventReaper, (logContext, consumerMetadata, subscriptionState, fetchConfig, deserializers, fetchMetricsManager, time) -> fetchCollector, (consumerConfig, subscriptionState, logContext, clusterResourceListeners) -> metadata, @@ -258,8 +257,9 @@ private AsyncKafkaConsumer newConsumer( long retryBackoffMs = 100L; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; + LogContext logContext = new LogContext(); return new AsyncKafkaConsumer<>( - new LogContext(), + logContext, clientId, new Deserializers<>(new StringDeserializer(), new StringDeserializer(), metrics), fetchBuffer, @@ -278,7 +278,7 @@ private AsyncKafkaConsumer newConsumer( defaultApiTimeoutMs, groupId, autoCommitEnabled, - new CommitOffsetsSharedState(new LogContext(), metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); + new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 59b0a346a2315..7eaecb9c01cb3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -68,6 +68,7 @@ public class ConsumerNetworkThreadTest { private final RequestManagers requestManagers; private final CompletableEventReaper applicationEventReaper; private final AsyncConsumerMetrics asyncConsumerMetrics; + private final SharedConsumerState sharedConsumerState; ConsumerNetworkThreadTest() { this.networkClientDelegate = mock(NetworkClientDelegate.class); @@ -80,6 +81,7 @@ public class ConsumerNetworkThreadTest { this.time = new MockTime(); this.applicationEventQueue = new LinkedBlockingQueue<>(); this.asyncConsumerMetrics = mock(AsyncConsumerMetrics.class); + this.sharedConsumerState = mock(SharedConsumerState.class); LogContext logContext = new LogContext(); this.consumerNetworkThread = new ConsumerNetworkThread( @@ -90,7 +92,8 @@ public class ConsumerNetworkThreadTest { () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState ); } @@ -218,7 +221,8 @@ public void testRunOnceRecordTimeBetweenNetworkThreadPoll() { () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState )) { consumerNetworkThread.initializeResources(); @@ -252,7 +256,8 @@ public void testRunOnceRecordApplicationEventQueueSizeAndApplicationEventQueueTi () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState )) { consumerNetworkThread.initializeResources(); @@ -330,7 +335,8 @@ private void testInitializeResourcesError(Supplier networ () -> applicationEventProcessor, networkClientDelegateSupplier, requestManagersSupplier, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState )) { assertThrows(KafkaException.class, thread::initializeResources, "initializeResources should fail because one or more Supplier throws an error on get()"); assertDoesNotThrow(thread::cleanup, "cleanup() should not cause an error because all references are checked before use"); 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 0f83e28d9e4d8..4d8724e5892fd 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 @@ -4210,10 +4210,10 @@ public TestableNetworkClientDelegate(Time time, ConsumerConfig config, LogContext logContext, KafkaClient client, - Metadata metadata, + ConsumerMetadata metadata, BackgroundEventHandler backgroundEventHandler, boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions)); } @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 4ff967e1f021a..82054934be1b7 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 @@ -72,13 +72,18 @@ public class NetworkClientDelegateTest { private MockClient client; private Metadata metadata; private BackgroundEventHandler backgroundEventHandler; + private SharedConsumerState sharedConsumerState; @BeforeEach public void setup() { this.time = new MockTime(0); this.metadata = mock(Metadata.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); + this.sharedConsumerState = mock(SharedConsumerState.class); this.client = new MockClient(time, Collections.singletonList(mockNode())); + + SharedErrorReference metadataError = new SharedErrorReference(); + when(this.sharedConsumerState.metadataError()).thenReturn(metadataError); } @Test @@ -218,10 +223,11 @@ public void testPropagateMetadataError() { doThrow(authException).when(metadata).maybeThrowAnyException(); NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false); - assertTrue(networkClientDelegate.getAndClearMetadataError().isEmpty()); + SharedErrorReference metadataErrorRef = sharedConsumerState.metadataError(); + assertTrue(metadataErrorRef.getAndClear().isEmpty()); networkClientDelegate.poll(0, time.milliseconds()); - Optional metadataError = networkClientDelegate.getAndClearMetadataError(); + Optional metadataError = metadataErrorRef.getAndClear(); assertTrue(metadataError.isPresent()); assertInstanceOf(AuthenticationException.class, metadataError.get()); assertEquals(authException.getMessage(), metadataError.get().getMessage()); @@ -297,7 +303,8 @@ public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErro this.metadata, this.backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, - asyncConsumerMetrics + asyncConsumerMetrics, + sharedConsumerState ); } 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 b23f75ebdb8b3..c8a7853cc1886 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 @@ -117,7 +117,7 @@ public void setup() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new CommitOffsetsSharedState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + new SharedConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), logContext ); } @@ -796,6 +796,7 @@ public void testRemoteListOffsetsRequestTimeoutMs() { int requestTimeoutMs = 100; int defaultApiTimeoutMs = 500; // Overriding the requestManager to provide different request and default API timeout + LogContext logContext = new LogContext(); requestManager = new OffsetsRequestManager( subscriptionState, metadata, @@ -806,8 +807,8 @@ public void testRemoteListOffsetsRequestTimeoutMs() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new CommitOffsetsSharedState(new LogContext(), metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), - new LogContext() + new SharedConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + logContext ); Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index a3d41eb4fd4c6..817b6f9218e55 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -73,7 +73,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - new CommitOffsetsSharedState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -120,7 +120,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - new CommitOffsetsSharedState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); 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 68da71d7767c1..20fd44fa198fc 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 @@ -2749,10 +2749,10 @@ public TestableNetworkClientDelegate(Time time, ConsumerConfig config, LogContext logContext, KafkaClient client, - Metadata metadata, + ConsumerMetadata metadata, BackgroundEventHandler backgroundEventHandler, boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class)); + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java index 09fc99d8e24a4..e23b02294950c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java @@ -138,7 +138,7 @@ private ShareConsumerImpl newConsumer(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (a, b, c, d, e, f, g, h) -> applicationEventHandler, + (a, b, c, d, e, f, g, h, i) -> applicationEventHandler, a -> backgroundEventReaper, (a, b, c, d, e) -> fetchCollector, backgroundEventQueue From be5a40671ab54628a9376ba4f5de4b91088d19ff Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 26 Aug 2025 12:07:36 -0700 Subject: [PATCH 15/31] Clearing the metadata error exception when thrown --- .../internals/ConsumerNetworkThread.java | 2 +- .../internals/NetworkClientDelegate.java | 2 +- .../internals/OffsetsRequestManager.java | 2 +- .../internals/SharedConsumerState.java | 16 ++--- .../internals/SharedErrorReference.java | 44 ------------- .../internals/SharedExceptionReference.java | 64 +++++++++++++++++++ .../internals/NetworkClientDelegateTest.java | 4 +- 7 files changed, 77 insertions(+), 57 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 3a1413ff5efff..ead5ffcc1034a 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 @@ -63,7 +63,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; private final AsyncConsumerMetrics asyncConsumerMetrics; - private final SharedErrorReference metadataError; + private final SharedExceptionReference metadataError; private ApplicationEventProcessor applicationEventProcessor; private NetworkClientDelegate networkClientDelegate; private RequestManagers requestManagers; 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 9fc936f124960..7d63392391b73 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,7 +70,7 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private final SharedErrorReference metadataError; + private final SharedExceptionReference metadataError; private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics 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 1394ecc664b71..8d5816ea29078 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 @@ -92,7 +92,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; - private final SharedErrorReference updatePositionsError; + private final SharedExceptionReference updatePositionsError; /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java index 7891b60763ea6..887a592462b7c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java @@ -41,7 +41,7 @@ *

  • {@link ApiVersions}
  • *
  • {@link ConsumerMetadata}
  • *
  • {@link OffsetFetcherUtils}
  • - *
  • {@link SharedErrorReference}
  • + *
  • {@link SharedExceptionReference}
  • *
  • {@link SubscriptionState}
  • *
  • {@link Time}
  • * @@ -55,8 +55,8 @@ public class SharedConsumerState { private final SubscriptionState subscriptions; private final OffsetFetcherUtils offsetFetcherUtils; - private final SharedErrorReference updatePositionsError; - private final SharedErrorReference metadataError; + private final SharedExceptionReference updatePositionsError; + private final SharedExceptionReference metadataError; public SharedConsumerState(LogContext logContext, ConsumerMetadata metadata, @@ -94,19 +94,19 @@ public SharedConsumerState(LogContext logContext, retryBackoffMs, apiVersions ); - this.updatePositionsError = new SharedErrorReference(); - this.metadataError = new SharedErrorReference(); + this.updatePositionsError = new SharedExceptionReference(); + this.metadataError = new SharedExceptionReference(); } OffsetFetcherUtils offsetFetcherUtils() { return offsetFetcherUtils; } - public SharedErrorReference updatePositionsError() { + public SharedExceptionReference updatePositionsError() { return updatePositionsError; } - public SharedErrorReference metadataError() { + public SharedExceptionReference metadataError() { return metadataError; } @@ -144,7 +144,7 @@ public SharedErrorReference metadataError() { */ public boolean canSkipUpdateFetchPositions() { updatePositionsError.maybeThrowException(); - metadataError.maybeThrowException(); + metadataError.maybeClearAndThrowException(); // If the cached value is set and there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or // INITIALIZING states, it's ok to skip. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java deleted file mode 100644 index 9d16e8836e058..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedErrorReference.java +++ /dev/null @@ -1,44 +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 java.util.concurrent.CompletionException; - -/** - * {@code SharedErrorReference} builds on top of {@link SharedReference} both to be more explicit about the contents - * and to provide utility methods. - */ -public class SharedErrorReference extends SharedReference { - - /** - * If the underlying error is present, this will throw the error. Note: if the exception is wrapped in a - * {@link CompletionException}, it will be unwrapped. However, if the underlying error is not a subclass - * of {@link KafkaException}, it will be wrapped as such so that it is an unchecked exception. - */ - public void maybeThrowException() { - ifPresent(exception -> { - // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions - // from the call to Future.get(). - if (exception instanceof CompletionException) - exception = exception.getCause(); - - throw ConsumerUtils.maybeWrapAsKafkaException(exception); - }); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java new file mode 100644 index 0000000000000..4c2219ea4c190 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java @@ -0,0 +1,64 @@ +/* + * 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 java.util.concurrent.CompletionException; +import java.util.function.Consumer; + +/** + * {@code SharedErrorReference} builds on top of {@link SharedReference} both to be more explicit about the contents + * and to provide utility methods. + */ +public class SharedExceptionReference extends SharedReference { + + private static final Consumer THROW_EXCEPTION = exception -> { + // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions + // from the call to Future.get(). + if (exception instanceof CompletionException) + exception = exception.getCause(); + + throw ConsumerUtils.maybeWrapAsKafkaException(exception); + }; + + /** + * If the underlying error is present, this will throw the error and clear it. + * + *

    + * + * Note: if the exception is wrapped in a {@link CompletionException}, it will be unwrapped. However, if + * the underlying error is not a subclass of {@link KafkaException}, it will be wrapped as such + * so that it is an unchecked exception. + */ + public void maybeClearAndThrowException() { + getClearAndRun(THROW_EXCEPTION); + } + + /** + * If the underlying error is present, this will throw the error. + * + *

    + * + * Note: if the exception is wrapped in a {@link CompletionException}, it will be unwrapped. However, if + * the underlying error is not a subclass of {@link KafkaException}, it will be wrapped as such + * so that it is an unchecked exception. + */ + public void maybeThrowException() { + ifPresent(THROW_EXCEPTION); + } +} 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 82054934be1b7..7a7e14ea5b802 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 @@ -82,7 +82,7 @@ public void setup() { this.sharedConsumerState = mock(SharedConsumerState.class); this.client = new MockClient(time, Collections.singletonList(mockNode())); - SharedErrorReference metadataError = new SharedErrorReference(); + SharedExceptionReference metadataError = new SharedExceptionReference(); when(this.sharedConsumerState.metadataError()).thenReturn(metadataError); } @@ -223,7 +223,7 @@ public void testPropagateMetadataError() { doThrow(authException).when(metadata).maybeThrowAnyException(); NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false); - SharedErrorReference metadataErrorRef = sharedConsumerState.metadataError(); + SharedExceptionReference metadataErrorRef = sharedConsumerState.metadataError(); assertTrue(metadataErrorRef.getAndClear().isEmpty()); networkClientDelegate.poll(0, time.milliseconds()); From 0e28d8c76adc6b2aa3d538d12b44892cafc19f24 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 26 Aug 2025 14:18:28 -0700 Subject: [PATCH 16/31] Remove redundant autoCommitState timer update in poll --- .../kafka/clients/consumer/internals/CommitRequestManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 7dde1ea90a6c5..82fd00e9a9c5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -175,8 +175,6 @@ public CommitRequestManager( */ @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { - autoCommitState.updateTimer(currentTimeMs); - // poll when the coordinator node is known and fatal error is not present if (coordinatorRequestManager.coordinator().isEmpty()) { pendingRequests.maybeFailOnCoordinatorFatalError(); From 1ea59a1b8114abf08517ab0dc5381e5ea6b0e46d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 26 Aug 2025 17:43:48 -0700 Subject: [PATCH 17/31] Updates for clarity --- .../internals/AbstractMembershipManager.java | 7 ++- .../internals/AsyncKafkaConsumer.java | 5 +- .../consumer/internals/AutoCommitState.java | 60 ++++++++++--------- .../internals/CommitRequestManager.java | 6 +- 4 files changed, 43 insertions(+), 35 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 9f01e76f63391..7733d84cb9174 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -110,6 +110,11 @@ public abstract class AbstractMembershipManager impl */ private final ConsumerMetadata metadata; + /** + * Keeps track of the auto-commit state. + */ + protected final AutoCommitState autoCommitState; + /** * Logger. */ @@ -192,8 +197,6 @@ public abstract class AbstractMembershipManager impl */ private boolean isPollTimerExpired; - private final AutoCommitState autoCommitState; - /** * Indicate the operation on consumer group membership that the consumer will perform when leaving the group. * The property should remain {@code GroupMembershipOperation.DEFAULT} until the consumer is closing. 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 f893992215c8a..9493f3491a5be 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 @@ -865,7 +865,10 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - PollEvent event = new PollEvent(timer.currentTimeMs()); + long currentTimeMs = timer.currentTimeMs(); + autoCommitState.updateTimer(currentTimeMs); + PollEvent event = new PollEvent(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 diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java index e281b721b75ef..353b2413b4e03 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.utils.Timer; import org.slf4j.Logger; -import java.util.concurrent.atomic.AtomicBoolean; +import java.time.Duration; import static org.apache.kafka.clients.consumer.ConsumerConfig.DEFAULT_AUTO_COMMIT_INTERVAL_MS; @@ -39,11 +39,9 @@ public interface AutoCommitState { boolean shouldAutoCommit(); - boolean isExpired(); - /** - * Reset the auto-commit timer to the auto-commit interval, so that the next auto-commit is - * sent out on the interval starting from now. If auto-commit is disabled this will + * Reset the auto-commit timer to the {@link ConsumerConfig#AUTO_COMMIT_INTERVAL_MS_CONFIG auto-commit interval}, + * so that the next auto-commit is sent out on the interval starting from now. If auto-commit is disabled this will * perform no action. */ void resetTimer(); @@ -52,13 +50,28 @@ public interface AutoCommitState { * Reset the auto-commit timer to the provided time (backoff), so that the next auto-commit is * sent out then. If auto-commit is disabled this will perform no action. */ - void resetTimer(long retryBackoffMs); + void resetTimer(final long retryBackoffMs); - long remainingMs(final long currentTimeMs); + /** + * Return the number of milliseconds remaining on the timer based on the most previous call to + * {@link #updateTimer(long)} and {@link #resetTimer()}/{@link #resetTimer(long)}. + */ + long remainingMs(); + /** + * Updates the timer to the timestamp provided. + * + *

    + * + * Note that the timer doesn't update automatically on its own, nor is it updated periodically by the background + * thread. The timer's notion of the current time is only updated through this mechanism. It is expected that + * this method will only be called during {@link AsyncKafkaConsumer#poll(Duration)} invocation by the application + * thread. The network thread is free to update the auto-commit interval via either {@link #resetTimer()} or + * {@link #resetTimer(long)}. + */ void updateTimer(final long currentTimeMs); - void setInflightCommitStatus(final boolean inflightCommitStatus); + void setInflightCommit(final boolean hasInflightCommit); static AutoCommitState enabled(final LogContext logContext, final Time time, @@ -90,7 +103,7 @@ class AutoCommitStateEnabled implements AutoCommitState { private final Logger log; private final Timer timer; private final long autoCommitInterval; - private final AtomicBoolean hasInflightCommit; + private boolean hasInflightCommit; private AutoCommitStateEnabled(final LogContext logContext, final Time time, @@ -98,7 +111,7 @@ private AutoCommitStateEnabled(final LogContext logContext, this.log = logContext.logger(AutoCommitState.class); this.timer = time.timer(autoCommitInterval); this.autoCommitInterval = autoCommitInterval; - this.hasInflightCommit = new AtomicBoolean(); + this.hasInflightCommit = false; } @Override @@ -112,7 +125,7 @@ public synchronized boolean shouldAutoCommit() { return false; } - if (hasInflightCommit.get()) { + if (hasInflightCommit) { log.trace("Skipping auto-commit on the interval because a previous one is still in-flight."); return false; } @@ -120,24 +133,18 @@ public synchronized boolean shouldAutoCommit() { return true; } - @Override - public synchronized boolean isExpired() { - return timer.isExpired(); - } - @Override public synchronized void resetTimer() { timer.reset(autoCommitInterval); } @Override - public synchronized void resetTimer(long retryBackoffMs) { + public synchronized void resetTimer(final long retryBackoffMs) { timer.reset(retryBackoffMs); } @Override - public synchronized long remainingMs(final long currentTimeMs) { - timer.update(currentTimeMs); + public synchronized long remainingMs() { return timer.remainingMs(); } @@ -147,8 +154,8 @@ public synchronized void updateTimer(final long currentTimeMs) { } @Override - public synchronized void setInflightCommitStatus(final boolean inflightCommitStatus) { - hasInflightCommit.set(inflightCommitStatus); + public synchronized void setInflightCommit(final boolean hasInflightCommit) { + this.hasInflightCommit = hasInflightCommit; } } @@ -167,23 +174,18 @@ public boolean shouldAutoCommit() { return false; } - @Override - public boolean isExpired() { - return false; - } - @Override public void resetTimer() { // No op } @Override - public void resetTimer(long retryBackoffMs) { + public void resetTimer(final long retryBackoffMs) { // No op } @Override - public long remainingMs(final long currentTimeMs) { + public long remainingMs() { return Long.MAX_VALUE; } @@ -193,7 +195,7 @@ public void updateTimer(final long currentTimeMs) { } @Override - public void setInflightCommitStatus(final boolean inflightCommitStatus) { + public void setInflightCommit(final boolean inflightCommitStatus) { // No op } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 82fd00e9a9c5f..eb86e51fb454a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -209,7 +209,7 @@ public void signalClose() { */ @Override public long maximumTimeToWait(long currentTimeMs) { - return autoCommitState.remainingMs(currentTimeMs); + return autoCommitState.remainingMs(); } private static long findMinTime(final Collection requests, final long currentTimeMs) { @@ -240,7 +240,7 @@ private CompletableFuture> requestAutoCom if (requestState.offsets.isEmpty()) { result = CompletableFuture.completedFuture(Collections.emptyMap()); } else { - autoCommitState.setInflightCommitStatus(true); + autoCommitState.setInflightCommit(true); OffsetCommitRequestState request = pendingRequests.addOffsetCommitRequest(requestState); result = request.future; result.whenComplete(autoCommitCallback(request.offsets)); @@ -362,7 +362,7 @@ private void autoCommitSyncBeforeRebalanceWithRetries(OffsetCommitRequestState r */ private BiConsumer, ? super Throwable> autoCommitCallback(final Map allConsumedOffsets) { return (response, throwable) -> { - autoCommitState.setInflightCommitStatus(false); + autoCommitState.setInflightCommit(false); if (throwable == null) { offsetCommitCallbackInvoker.enqueueInterceptorInvocation(allConsumedOffsets); log.debug("Completed auto-commit of offsets {}", allConsumedOffsets); From fa7a887bd81bd9716431aa8b120566f99a67eeb1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 27 Aug 2025 10:14:51 -0700 Subject: [PATCH 18/31] Added comments and reverted unnecessary changes --- .../internals/AsyncKafkaConsumer.java | 11 +++++++ .../internals/NetworkClientDelegate.java | 31 +++++++------------ .../internals/OffsetsRequestManager.java | 6 +++- .../internals/SharedConsumerState.java | 28 ++++++++--------- 4 files changed, 40 insertions(+), 36 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 bd54977477abb..87ba06b0dca78 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 @@ -1813,6 +1813,9 @@ private Fetch pollForFetches(Timer timer) { // We do not want to be stuck blocking in poll if we are missing some positions // since the offset lookup may be backing off after a failure + + // NOTE: hasAllFetchPositions to return the correct answer, we MUST call + // updateAssignmentMetadataIfNeeded before this method. if (!subscriptions.hasAllFetchPositions() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; } @@ -1868,6 +1871,14 @@ private Fetch collectFetch() { * defined */ private boolean updateFetchPositions(final Timer timer) { + // Fetch position validation is in the hot path for poll() and the cost of thread interaction for + // event processing is *very* heavy, CPU-wise. In a stable system, the positions are valid; having the + // network thread check the validity yields the same answer 99%+ of the time. But calling the + // network thread to determine that is very expensive. + // + // Instead, let the *application thread* determine if any partitions need their positions updated. If not, + // the application thread can skip sending an event to the network thread that will simply end up coming + // to the same conclusion, albeit much slower. if (sharedConsumerState.canSkipUpdateFetchPositions()) return true; 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 7d63392391b73..829415222dc8e 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 @@ -74,15 +74,16 @@ public class NetworkClientDelegate implements AutoCloseable { private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; - public NetworkClientDelegate(final Time time, - final ConsumerConfig config, - final LogContext logContext, - final KafkaClient client, - final Metadata metadata, - final BackgroundEventHandler backgroundEventHandler, - final boolean notifyMetadataErrorsViaErrorQueue, - final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState) { + public NetworkClientDelegate( + final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client, + final Metadata metadata, + final BackgroundEventHandler backgroundEventHandler, + final boolean notifyMetadataErrorsViaErrorQueue, + final AsyncConsumerMetrics asyncConsumerMetrics, + final SharedConsumerState sharedConsumerState) { this.time = time; this.client = client; this.metadata = metadata; @@ -462,17 +463,7 @@ protected NetworkClientDelegate create() { metadata, throttleTimeSensor, clientTelemetrySender); - return new NetworkClientDelegate( - time, - config, - logContext, - client, - metadata, - backgroundEventHandler, - notifyMetadataErrorsViaErrorQueue, - asyncConsumerMetrics, - sharedConsumerState - ); + return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics, sharedConsumerState); } }; } 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 8d5816ea29078..4003300a2f6c9 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 @@ -229,7 +229,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { CompletableFuture result = new CompletableFuture<>(); try { - if (updatePositionsError.getClearAndRun(result::completeExceptionally)) { + if (maybeCompleteWithPreviousException(result)) { return result; } @@ -256,6 +256,10 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { return result; } + private boolean maybeCompleteWithPreviousException(CompletableFuture result) { + return updatePositionsError.getClearAndRun(result::completeExceptionally); + } + /** * 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 diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java index 887a592462b7c..b697dae5003bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java @@ -28,10 +28,10 @@ /** * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the - * background thread ({@link OffsetsRequestManager}) to avoid costly inter-thread communication, where possible. - * This class compromises on the ideal of keeping state only in the background thread. However, this class only - * relies on classes which are designed to be multithread-safe, thus they can be used in both the application and - * background threads. + * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread communication, where possible. + * This class compromises on the ideal of keeping state only in the network thread. However, this class only + * relies on classes which are designed to be thread-safe, thus they can be used in both the application + * and network threads. * *

    * @@ -63,14 +63,7 @@ public SharedConsumerState(LogContext logContext, SubscriptionState subscriptions, Time time, long retryBackoffMs) { - this( - logContext, - metadata, - subscriptions, - time, - retryBackoffMs, - new ApiVersions() - ); + this(logContext, metadata, subscriptions, time, retryBackoffMs, new ApiVersions()); } public SharedConsumerState(LogContext logContext, @@ -146,8 +139,13 @@ public boolean canSkipUpdateFetchPositions() { updatePositionsError.maybeThrowException(); metadataError.maybeClearAndThrowException(); - // If the cached value is set and there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or - // INITIALIZING states, it's ok to skip. - return offsetFetcherUtils.getPartitionsToValidate().isEmpty() && subscriptions.hasAllFetchPositions(); + // In cases of metadata updates, getPartitionsToValidate() will review the partitions and + // determine which, if any, need to be validated. If any partitions require validation, the + // update fetch positions step can't be skipped. + if (!offsetFetcherUtils.getPartitionsToValidate().isEmpty()) + return false; + + // If there are no partitions in the AWAIT_RESET, AWAIT_VALIDATION, or INITIALIZING states, it's ok to skip. + return subscriptions.hasAllFetchPositions(); } } From 12a4d0ed2b5c8c42b865a7f52ef5c0c59228f35f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 27 Aug 2025 10:15:42 -0700 Subject: [PATCH 19/31] Ugh. Typo --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 +- 1 file changed, 1 insertion(+), 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 87ba06b0dca78..4d69ea90b615e 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 @@ -1814,7 +1814,7 @@ private Fetch pollForFetches(Timer timer) { // We do not want to be stuck blocking in poll if we are missing some positions // since the offset lookup may be backing off after a failure - // NOTE: hasAllFetchPositions to return the correct answer, we MUST call + // NOTE: for hasAllFetchPositions to return the correct answer, we MUST call // updateAssignmentMetadataIfNeeded before this method. if (!subscriptions.hasAllFetchPositions() && pollTimeout > retryBackoffMs) { pollTimeout = retryBackoffMs; From a06ca4029ba7c8e11b505b36f03113075c48dacb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 27 Aug 2025 12:05:48 -0700 Subject: [PATCH 20/31] Refactoring to make the intention a little more obvious and in keeping with the style from KAFKA-19589 --- .../internals/AbstractMembershipManager.java | 23 ++-- .../internals/AsyncKafkaConsumer.java | 70 +++++++----- .../internals/CommitRequestManager.java | 15 +-- .../internals/ConsumerMembershipManager.java | 13 +-- .../consumer/internals/RequestManagers.java | 9 +- .../internals/ShareMembershipManager.java | 4 +- ...tState.java => SharedAutoCommitState.java} | 59 +++++----- .../internals/SharedConsumerState.java | 104 ++++++++++++++++++ .../internals/SharedReconciliationState.java | 53 +++++++++ .../internals/AsyncKafkaConsumerTest.java | 7 +- .../internals/CommitRequestManagerTest.java | 20 +++- .../ConsumerMembershipManagerTest.java | 25 ++++- .../internals/RequestManagersTest.java | 9 +- 13 files changed, 303 insertions(+), 108 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{AutoCommitState.java => SharedAutoCommitState.java} (73%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 7733d84cb9174..4b3c1f0f0d2ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -113,7 +113,7 @@ public abstract class AbstractMembershipManager impl /** * Keeps track of the auto-commit state. */ - protected final AutoCommitState autoCommitState; + protected final SharedAutoCommitState autoCommitState; /** * Logger. @@ -145,7 +145,7 @@ public abstract class AbstractMembershipManager impl * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered * after receiving a heartbeat response, or a metadata update. */ - private final AtomicBoolean reconciliationInProgress; + private final SharedReconciliationState reconciliationState; /** * True if a reconciliation is in progress and the member rejoined the group since the start @@ -211,8 +211,7 @@ public abstract class AbstractMembershipManager impl Logger log, Time time, RebalanceMetricsManager metricsManager, - AutoCommitState autoCommitState, - AtomicBoolean reconciliationInProgress) { + SharedConsumerState sharedConsumerState) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.subscriptions = subscriptions; @@ -224,8 +223,8 @@ public abstract class AbstractMembershipManager impl this.stateUpdatesListeners = new ArrayList<>(); this.time = time; this.metricsManager = metricsManager; - this.autoCommitState = autoCommitState; - this.reconciliationInProgress = reconciliationInProgress; + this.autoCommitState = sharedConsumerState.autoCommitState(); + this.reconciliationState = sharedConsumerState.reconciliationState(); } /** @@ -535,7 +534,7 @@ public void transitionToJoining() { "the member is in FATAL state"); return; } - if (reconciliationInProgress.get()) { + if (reconciliationInProgress()) { rejoinedWhileReconciliationInProgress = true; } resetEpoch(); @@ -835,7 +834,7 @@ public void maybeReconcile(boolean canCommit) { "current assignment."); return; } - if (reconciliationInProgress.get()) { + if (reconciliationInProgress()) { log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. " + "Assignment {} will be handled in the next reconciliation loop.", currentTargetAssignment); return; @@ -968,7 +967,7 @@ private void revokeAndAssign(LocalAssignment resolvedAssignment, log.error("Reconciliation failed.", error); markReconciliationCompleted(); } else { - if (reconciliationInProgress.get() && !maybeAbortReconciliation()) { + if (reconciliationInProgress() && !maybeAbortReconciliation()) { currentAssignment = resolvedAssignment; signalReconciliationCompleting(); @@ -1039,7 +1038,7 @@ protected CompletableFuture signalPartitionsLost(Set parti * Visible for testing. */ void markReconciliationInProgress() { - reconciliationInProgress.set(true); + reconciliationState.setInProgress(true); rejoinedWhileReconciliationInProgress = false; } @@ -1047,7 +1046,7 @@ void markReconciliationInProgress() { * Visible for testing. */ void markReconciliationCompleted() { - reconciliationInProgress.set(false); + reconciliationState.setInProgress(false); rejoinedWhileReconciliationInProgress = false; } @@ -1377,7 +1376,7 @@ Map> topicPartitionsAwaitingReconciliation() { * by a call to {@link #maybeReconcile(boolean)}. Visible for testing. */ boolean reconciliationInProgress() { - return reconciliationInProgress.get(); + return reconciliationState.isInProgress(); } /** 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 9493f3491a5be..5e1e5dec246e6 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 @@ -131,7 +131,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -356,8 +355,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { private final long retryBackoffMs; private final int requestTimeoutMs; private final Duration defaultApiTimeoutMs; - private final AtomicBoolean reconciliationInProgress = new AtomicBoolean(); - private final AutoCommitState autoCommitState; + private final SharedConsumerState sharedConsumerState; private volatile boolean closed = false; // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); @@ -478,7 +476,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); - this.autoCommitState = AutoCommitState.newInstance(logContext, config, time); + this.sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.newInstance( + requireNonNull(logContext), + requireNonNull(config), + requireNonNull(time) + ) + ); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -495,8 +499,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - autoCommitState, - reconciliationInProgress + sharedConsumerState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -567,7 +570,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int requestTimeoutMs, int defaultApiTimeoutMs, String groupId, - AutoCommitState autoCommitState) { + SharedConsumerState sharedConsumerState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -590,7 +593,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.applicationEventHandler = applicationEventHandler; this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics); this.clientTelemetryReporter = Optional.empty(); - this.autoCommitState = autoCommitState; + this.sharedConsumerState = sharedConsumerState; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, @@ -665,7 +668,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.autoCommitState = AutoCommitState.newInstance(logContext, config, time); + this.sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.newInstance( + requireNonNull(logContext), + requireNonNull(config), + requireNonNull(time) + ) + ); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, @@ -683,8 +692,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - autoCommitState, - reconciliationInProgress + sharedConsumerState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -865,20 +873,7 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - long currentTimeMs = timer.currentTimeMs(); - autoCommitState.updateTimer(currentTimeMs); - PollEvent event = new PollEvent(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); - - if (reconciliationInProgress.get() || autoCommitState.shouldAutoCommit()) { - // 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()); - } + sendPollEvent(timer); do { // We must not allow wake-ups between polling for fetches and returning the records. @@ -915,6 +910,29 @@ public ConsumerRecords poll(final Duration timeout) { } } + private void sendPollEvent(Timer timer) { + long currentTimeMs = timer.currentTimeMs(); + + // Make sure to let the background thread know that we are still polling. + PollEvent event = new PollEvent(currentTimeMs); + + if (sharedConsumerState.canSkipWaitingOnPoll(currentTimeMs)) { + // This will *not* trigger async auto-commits of consumed positions as the shared Timer for + // auto-commit interval will not change between the application thread and the network thread. This + // is true of the reconciliation state. The state will not change between the SharedConsumerState + // check above and the processing of the PollEvent. + applicationEventHandler.add(event); + } else { + // 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()); + } + } + /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. @@ -1526,7 +1544,7 @@ private void autoCommitOnClose(final Timer timer) { if (groupMetadata.get().isEmpty()) return; - if (autoCommitState.isAutoCommitEnabled()) + if (sharedConsumerState.isAutoCommitEnabled()) commitSyncAllConsumed(timer); applicationEventHandler.add(new CommitOnCloseEvent()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index eb86e51fb454a..f08536d640de9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -78,7 +78,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener private final ConsumerMetadata metadata; private final LogContext logContext; private final Logger log; - private final AutoCommitState autoCommitState; + private final SharedAutoCommitState autoCommitState; private final CoordinatorRequestManager coordinatorRequestManager; private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final OffsetCommitMetricsManager metricsManager; @@ -116,7 +116,7 @@ public CommitRequestManager( final Optional groupInstanceId, final Metrics metrics, final ConsumerMetadata metadata, - final AutoCommitState autoCommitState) { + final SharedConsumerState sharedConsumerState) { this(time, logContext, subscriptions, @@ -130,10 +130,11 @@ public CommitRequestManager( OptionalDouble.empty(), metrics, metadata, - autoCommitState); + sharedConsumerState); } // Visible for testing + @SuppressWarnings({"checkstyle:ParameterNumber"}) CommitRequestManager( final Time time, final LogContext logContext, @@ -148,13 +149,13 @@ public CommitRequestManager( final OptionalDouble jitter, final Metrics metrics, final ConsumerMetadata metadata, - final AutoCommitState autoCommitState) { + final SharedConsumerState sharedConsumerState) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.time = time; this.logContext = logContext; this.log = logContext.logger(getClass()); this.pendingRequests = new PendingRequests(); - this.autoCommitState = autoCommitState; + this.autoCommitState = sharedConsumerState.autoCommitState(); this.coordinatorRequestManager = coordinatorRequestManager; this.groupId = groupId; this.groupInstanceId = groupInstanceId; @@ -605,7 +606,7 @@ public boolean autoCommitEnabled() { * perform no action. */ public void resetAutoCommitTimer() { - autoCommitState.resetTimer(); + autoCommitState.resetInterval(); } /** @@ -613,7 +614,7 @@ public void resetAutoCommitTimer() { * sent out then. If auto-commit is not enabled this will perform no action. */ public void resetAutoCommitTimer(long retryBackoffMs) { - autoCommitState.resetTimer(retryBackoffMs); + autoCommitState.resetInterval(retryBackoffMs); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java index 520088decfa8d..d6f871eb66a9b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java @@ -44,7 +44,6 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.DEFAULT; import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.LEAVE_GROUP; @@ -152,8 +151,7 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, Metrics metrics, - AutoCommitState autoCommitState, - AtomicBoolean reconciliationInProgress) { + SharedConsumerState sharedConsumerState) { this(groupId, groupInstanceId, rackId, @@ -166,8 +164,7 @@ public ConsumerMembershipManager(String groupId, backgroundEventHandler, time, new ConsumerRebalanceMetricsManager(metrics), - autoCommitState, - reconciliationInProgress); + sharedConsumerState); } // Visible for testing @@ -183,16 +180,14 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, RebalanceMetricsManager metricsManager, - AutoCommitState autoCommitState, - AtomicBoolean reconciliationInProgress) { + SharedConsumerState sharedConsumerState) { super(groupId, subscriptions, metadata, logContext.logger(ConsumerMembershipManager.class), time, metricsManager, - autoCommitState, - reconciliationInProgress); + sharedConsumerState); this.groupInstanceId = groupInstanceId; this.rackId = rackId; this.rebalanceTimeoutMs = rebalanceTimeoutMs; 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 efab511324725..764d08500993e 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 @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import static java.util.Objects.requireNonNull; @@ -169,8 +168,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final AutoCommitState autoCommitState, - final AtomicBoolean cachedReconciliationInProgress + final SharedConsumerState sharedConsumerState ) { return new CachedSupplier<>() { @Override @@ -220,7 +218,7 @@ protected RequestManagers create() { groupRebalanceConfig.groupInstanceId, metrics, metadata, - autoCommitState); + sharedConsumerState); if (streamsRebalanceData.isPresent()) { streamsMembershipManager = new StreamsMembershipManager( groupRebalanceConfig.groupId, @@ -262,8 +260,7 @@ protected RequestManagers create() { backgroundEventHandler, time, metrics, - autoCommitState, - cachedReconciliationInProgress); + sharedConsumerState); // Update the group member ID label in the client telemetry reporter. // According to KIP-1082, the consumer will generate the member ID as the incarnation ID of the process. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java index 75f4cc5e6fc9b..34c9a625479a0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java @@ -31,7 +31,6 @@ import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; -import java.util.concurrent.atomic.AtomicBoolean; /** * Group manager for a single consumer that has a group id defined in the config @@ -108,8 +107,7 @@ public ShareMembershipManager(LogContext logContext, logContext.logger(ShareMembershipManager.class), time, metricsManager, - AutoCommitState.disabled(), - new AtomicBoolean()); + new SharedConsumerState(SharedAutoCommitState.disabled())); this.rackId = rackId; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java index 353b2413b4e03..f2a965850e34d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java @@ -20,6 +20,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 java.time.Duration; @@ -29,7 +30,7 @@ /** * Encapsulates the state of auto-committing and manages the auto-commit timer. */ -public interface AutoCommitState { +public interface SharedAutoCommitState { /** * @return {@code true} if auto-commit is enabled as defined in the configuration @@ -44,17 +45,17 @@ public interface AutoCommitState { * so that the next auto-commit is sent out on the interval starting from now. If auto-commit is disabled this will * perform no action. */ - void resetTimer(); + void resetInterval(); /** - * Reset the auto-commit timer to the provided time (backoff), so that the next auto-commit is + * Reset the auto-commit timer to the provided interval, so that the next auto-commit is * sent out then. If auto-commit is disabled this will perform no action. */ - void resetTimer(final long retryBackoffMs); + void resetInterval(final long interval); /** * Return the number of milliseconds remaining on the timer based on the most previous call to - * {@link #updateTimer(long)} and {@link #resetTimer()}/{@link #resetTimer(long)}. + * {@link #updateTimer(long)} and {@link #resetInterval()}/{@link #resetInterval(long)}. */ long remainingMs(); @@ -66,30 +67,30 @@ public interface AutoCommitState { * Note that the timer doesn't update automatically on its own, nor is it updated periodically by the background * thread. The timer's notion of the current time is only updated through this mechanism. It is expected that * this method will only be called during {@link AsyncKafkaConsumer#poll(Duration)} invocation by the application - * thread. The network thread is free to update the auto-commit interval via either {@link #resetTimer()} or - * {@link #resetTimer(long)}. + * thread. The network thread is free to update the auto-commit interval via either {@link #resetInterval()} or + * {@link #resetInterval(long)}. */ void updateTimer(final long currentTimeMs); void setInflightCommit(final boolean hasInflightCommit); - static AutoCommitState enabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { - return new AutoCommitStateEnabled(logContext, time, autoCommitInterval); + static SharedAutoCommitState enabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + return new SharedAutoCommitStateEnabled(logContext, time, autoCommitInterval); } - static AutoCommitState enabled(final LogContext logContext, final Time time) { + static SharedAutoCommitState enabled(final LogContext logContext, final Time time) { return enabled(logContext, time, DEFAULT_AUTO_COMMIT_INTERVAL_MS); } - static AutoCommitState disabled() { - return new AutoCommitStateDisabled(); + static SharedAutoCommitState disabled() { + return new SharedAutoCommitStateDisabled(); } - static AutoCommitState newInstance(final LogContext logContext, - final ConsumerConfig config, - final Time time) { + static SharedAutoCommitState newInstance(final LogContext logContext, + final ConsumerConfig config, + final Time time) { if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long interval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); return enabled(logContext, time, interval); @@ -98,17 +99,17 @@ static AutoCommitState newInstance(final LogContext logContext, } } - class AutoCommitStateEnabled implements AutoCommitState { + class SharedAutoCommitStateEnabled implements SharedAutoCommitState { private final Logger log; private final Timer timer; private final long autoCommitInterval; private boolean hasInflightCommit; - private AutoCommitStateEnabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { - this.log = logContext.logger(AutoCommitState.class); + private SharedAutoCommitStateEnabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + this.log = logContext.logger(SharedAutoCommitState.class); this.timer = time.timer(autoCommitInterval); this.autoCommitInterval = autoCommitInterval; this.hasInflightCommit = false; @@ -134,13 +135,13 @@ public synchronized boolean shouldAutoCommit() { } @Override - public synchronized void resetTimer() { + public synchronized void resetInterval() { timer.reset(autoCommitInterval); } @Override - public synchronized void resetTimer(final long retryBackoffMs) { - timer.reset(retryBackoffMs); + public synchronized void resetInterval(final long interval) { + timer.reset(interval); } @Override @@ -159,9 +160,9 @@ public synchronized void setInflightCommit(final boolean hasInflightCommit) { } } - class AutoCommitStateDisabled implements AutoCommitState { + class SharedAutoCommitStateDisabled implements SharedAutoCommitState { - private AutoCommitStateDisabled() { + private SharedAutoCommitStateDisabled() { } @Override @@ -175,12 +176,12 @@ public boolean shouldAutoCommit() { } @Override - public void resetTimer() { + public void resetInterval() { // No op } @Override - public void resetTimer(final long retryBackoffMs) { + public void resetInterval(final long interval) { // No op } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java new file mode 100644 index 0000000000000..0b8df307c9411 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java @@ -0,0 +1,104 @@ +/* + * 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.PollEvent; + +import static java.util.Objects.requireNonNull; + +/** + * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the + * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread communication, where possible. + * This class compromises on the ideal of keeping state only in the network thread. However, this class only + * relies on classes which are designed to be thread-safe, thus they can be used in both the application + * and network threads. + * + *

    + * + * The following thread-safe classes are used by this class: + * + *

      + *
    • {@link SharedAutoCommitState}
    • + *
    • {@link SharedReconciliationState}
    • + *
    + * + *

    + * + * In general, callers from the application thread should not mutate any of the state contained within this class. + * It should be considered as read-only, and only the network thread should mutate the state. + */ +public class SharedConsumerState { + + private final SharedAutoCommitState autoCommitState; + private final SharedReconciliationState sharedReconciliationState; + + public SharedConsumerState(SharedAutoCommitState autoCommitState) { + this.autoCommitState = requireNonNull(autoCommitState); + this.sharedReconciliationState = new SharedReconciliationState(); + } + + public SharedAutoCommitState autoCommitState() { + return autoCommitState; + } + + public SharedReconciliationState reconciliationState() { + return sharedReconciliationState; + } + + /** + * This method is used by {@code AsyncKafkaConsumer#poll()} to determine if it can skip waiting for the + * {@link PollEvent}. Sending the {@link PollEvent} is in the critical path, and if the application thread + * can determine that it doesn't need to wait for it to complete before continuing, that is a big performance + * savings. + * + *

    + * + * This method performs similar checks to the start of {@code ApplicationEventProcessor#process(PollEvent)}: + * + *

      + *
    1. + * Checks if there is already a reconciliation in process in + * {@link AbstractMembershipManager#maybeReconcile(boolean)} + *
    2. + *
    3. + * Checks if the auto-commit's interval has expired and needs to perform a commit offsets operation + * in {@link CommitRequestManager#updateTimerAndMaybeCommit(long)} + *
    4. + *
    + * + * If either of the above tests are satisfied, this method will return {@code false} to let the application thread + * know that it needs to block for the {@link PollEvent} to complete. Otherwise, this method will return + * {@code true}, which signals to the application thread that it can enqueue a {@link PollEvent} but it should + * not wait for it to complete. + * + * @return true if all checks pass, false if either of the latter two checks fail + */ + public boolean canSkipWaitingOnPoll(long currentTimeMs) { + if (sharedReconciliationState.isInProgress()) + return false; + + autoCommitState.updateTimer(currentTimeMs); + return !autoCommitState.shouldAutoCommit(); + } + + /** + * Determines if auto-commit is enabled. + */ + public boolean isAutoCommitEnabled() { + return autoCommitState.isAutoCommitEnabled(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java new file mode 100644 index 0000000000000..da87305411c8f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java @@ -0,0 +1,53 @@ +/* + * 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.PollEvent; + +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class is constructed from within the {@link SharedConsumerState} instance, which means it's available + * for both the application and network threads to use. The main user is the {@link AbstractMembershipManager} for + * mutations and the {@link SharedConsumerState#canSkipWaitingOnPoll(long)} method for determining if the costly + * {@link PollEvent} can be sent in the background or not. + * + *

    + * + * Yes, this class is a wrapper around a simple {@link AtomicBoolean}, but the intention behind dedicating a class + * to it hopefully makes the shared nature and its purpose more apparent. + */ +public class SharedReconciliationState { + + private final AtomicBoolean value; + + public SharedReconciliationState() { + this(false); + } + + public SharedReconciliationState(boolean value) { + this.value = new AtomicBoolean(value); + } + + public boolean isInProgress() { + return value.get(); + } + + public void setInProgress(boolean value) { + this.value.set(value); + } +} 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 646b8182c34bd..3b812778aa220 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 @@ -258,6 +258,9 @@ private AsyncKafkaConsumer newConsumer( int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; LogContext logContext = new LogContext(); + SharedConsumerState sharedConsumerState = new SharedConsumerState( + autoCommitEnabled ? SharedAutoCommitState.enabled(logContext, time) : SharedAutoCommitState.disabled() + ); return new AsyncKafkaConsumer<>( logContext, clientId, @@ -277,7 +280,7 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, groupId, - autoCommitEnabled ? AutoCommitState.enabled(logContext, time) : AutoCommitState.disabled()); + sharedConsumerState); } @Test @@ -1357,7 +1360,6 @@ private MemberStateListener captureGroupMetadataUpdateListener(final MockedStati any(), applicationThreadMemberStateListener.capture(), any(), - any(), any() )); return applicationThreadMemberStateListener.getValue(); @@ -1430,7 +1432,6 @@ private Optional captureStreamRebalanceData(final MockedSt any(), any(), streamRebalanceData.capture(), - any(), any() )); return streamRebalanceData.getValue(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index f133843087a40..d11842f95b993 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -137,7 +137,13 @@ public void setup() { @Test public void testOffsetFetchRequestStateToStringBase() { ConsumerConfig config = mock(ConsumerConfig.class); - AutoCommitState autoCommitState = AutoCommitState.newInstance(logContext, config, time); + SharedConsumerState sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.newInstance( + logContext, + config, + time + ) + ); CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, @@ -152,7 +158,7 @@ public void testOffsetFetchRequestStateToStringBase() { OptionalDouble.of(0), metrics, metadata, - autoCommitState); + sharedConsumerState); commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString()); Set requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1)); @@ -1566,7 +1572,13 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); ConsumerConfig config = new ConsumerConfig(props); - AutoCommitState autoCommitState = AutoCommitState.newInstance(logContext, config, time); + SharedConsumerState sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.newInstance( + logContext, + config, + time + ) + ); return spy(new CommitRequestManager( this.time, @@ -1582,7 +1594,7 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long OptionalDouble.of(0), metrics, metadata, - autoCommitState)); + sharedConsumerState)); } private ClientResponse buildOffsetFetchClientResponse( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index 57c9d6d0ea1fc..5150a985bd70f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -66,7 +66,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -142,10 +141,16 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup(String gro } private ConsumerMembershipManager createMembershipManager(String groupInstanceId) { + SharedConsumerState sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.enabled( + LOG_CONTEXT, + time + ) + ); ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean())); + backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState)); assertMemberIdIsGenerated(manager.memberId()); return manager; } @@ -155,10 +160,16 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( String serverAssignor, String rackId ) { + SharedConsumerState sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.enabled( + LOG_CONTEXT, + time + ) + ); ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(rackId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean())); + metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState)); assertMemberIdIsGenerated(manager.memberId()); manager.transitionToJoining(); return manager; @@ -243,10 +254,16 @@ public void testTransitionToFatal() { @Test public void testTransitionToFailedWhenTryingToJoin() { + SharedConsumerState sharedConsumerState = new SharedConsumerState( + SharedAutoCommitState.enabled( + LOG_CONTEXT, + time + ) + ); ConsumerMembershipManager membershipManager = new ConsumerMembershipManager( GROUP_ID, Optional.empty(), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, AutoCommitState.enabled(LOG_CONTEXT, time), new AtomicBoolean()); + backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 9b2ce81801972..8ffac9605a8cb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -30,7 +30,6 @@ import java.util.Optional; import java.util.Properties; import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -51,6 +50,7 @@ public void testMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); + final SharedConsumerState sharedConsumerState = new SharedConsumerState(SharedAutoCommitState.disabled()); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), @@ -68,8 +68,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - AutoCommitState.disabled(), - new AtomicBoolean() + sharedConsumerState ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -93,6 +92,7 @@ public void testStreamMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); + final SharedConsumerState sharedConsumerState = new SharedConsumerState(SharedAutoCommitState.disabled()); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), @@ -110,8 +110,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - AutoCommitState.disabled(), - new AtomicBoolean() + sharedConsumerState ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From d6fe1fd35ddb60d926325de3bdf575712dee11e8 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 27 Aug 2025 12:09:08 -0700 Subject: [PATCH 21/31] Reverting unnecessary whitespace diffs --- .../clients/consumer/internals/CommitRequestManagerTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index d11842f95b993..ba04bfa647fbe 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -117,6 +117,7 @@ public class CommitRequestManagerTest { private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final Metrics metrics = new Metrics(); private Properties props; + private final int defaultApiTimeoutMs = 60000; @@ -144,6 +145,7 @@ public void testOffsetFetchRequestStateToStringBase() { time ) ); + CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, From 278b06b6e0aedc3cc2dbad66b4d940ae4b5c7b8f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 28 Aug 2025 09:15:42 -0700 Subject: [PATCH 22/31] Optimization to send only one PollEvent per poll() does not work, need to send one for each loop --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 3 ++- 1 file changed, 2 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 5e1e5dec246e6..1c5db86d2009d 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 @@ -873,9 +873,10 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - sendPollEvent(timer); do { + sendPollEvent(timer); + // 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 From 9cf406f0e653f7770fe4c09f854625826dff0b3a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 28 Aug 2025 14:08:34 -0700 Subject: [PATCH 23/31] Updated AbstractMembershipManager comments related to use of SharedAutoCommitState and SharedReconciliationState --- .../internals/AbstractMembershipManager.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 4b3c1f0f0d2ac..c825bd16feba0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -112,6 +112,11 @@ public abstract class AbstractMembershipManager impl /** * Keeps track of the auto-commit state. + * + *

    + * + * Note: per its class name, this state is shared with the application thread, so care must be + * taken to evaluate how it's used elsewhere when updating related logic. */ protected final SharedAutoCommitState autoCommitState; @@ -142,8 +147,15 @@ public abstract class AbstractMembershipManager impl /** * If there is a reconciliation running (triggering commit, callbacks) for the - * assignmentReadyToReconcile. This will be true if {@link #maybeReconcile(boolean)} has been triggered - * after receiving a heartbeat response, or a metadata update. + * assignmentReadyToReconcile. {@link SharedReconciliationState#isInProgress()} will be true if + * {@link #maybeReconcile(boolean)} has been triggered after receiving a heartbeat response, or a metadata update. + * Calling code should generally favor {@link #reconciliationInProgress()} for its clarity over direct use of + * this state. + * + *

    + * + * Note: per its class name, this state is shared with the application thread, so care must be + * taken to evaluate how it's used elsewhere when updating related logic. */ private final SharedReconciliationState reconciliationState; From 9469b29a6b682c5766ed038c63db93b4cc3a487d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 29 Aug 2025 09:08:35 -0700 Subject: [PATCH 24/31] Refactoring and JavaDoc clarification --- .../internals/AsyncKafkaConsumer.java | 26 ++++---- .../internals/ConsumerNetworkThread.java | 6 +- .../internals/NetworkClientDelegate.java | 10 +-- .../internals/OffsetsRequestManager.java | 12 ++-- .../consumer/internals/RequestManagers.java | 4 +- .../consumer/internals/ShareConsumerImpl.java | 27 ++------ ...java => ThreadSafeAsyncConsumerState.java} | 64 ++++++++++--------- .../internals/ThreadSafeConsumerState.java | 49 ++++++++++++++ ...java => ThreadSafeExceptionReference.java} | 6 +- ...eference.java => ThreadSafeReference.java} | 7 +- .../events/ApplicationEventHandler.java | 6 +- .../ApplicationEventHandlerTest.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 2 +- .../internals/ConsumerNetworkThreadTest.java | 12 ++-- .../internals/FetchRequestManagerTest.java | 2 +- .../internals/NetworkClientDelegateTest.java | 12 ++-- .../internals/OffsetsRequestManagerTest.java | 4 +- .../internals/RequestManagersTest.java | 4 +- .../ShareConsumeRequestManagerTest.java | 2 +- 19 files changed, 149 insertions(+), 110 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedConsumerState.java => ThreadSafeAsyncConsumerState.java} (66%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedExceptionReference.java => ThreadSafeExceptionReference.java} (90%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedReference.java => ThreadSafeReference.java} (92%) 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 4d69ea90b615e..d131e242ca9e3 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 @@ -360,7 +360,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); - private final SharedConsumerState sharedConsumerState; + private final ThreadSafeAsyncConsumerState threadSafeConsumerState; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -462,7 +462,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); - this.sharedConsumerState = new SharedConsumerState( + this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( logContext, metadata, subscriptions, @@ -481,7 +481,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, backgroundEventHandler, false, kafkaConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); @@ -501,7 +501,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - sharedConsumerState + threadSafeConsumerState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -516,7 +516,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, networkClientDelegateSupplier, requestManagersSupplier, kafkaConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( logContext, @@ -574,7 +574,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int defaultApiTimeoutMs, String groupId, boolean autoCommitEnabled, - SharedConsumerState sharedConsumerState) { + ThreadSafeAsyncConsumerState threadSafeConsumerState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -604,7 +604,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, time, kafkaConsumerMetrics ); - this.sharedConsumerState = sharedConsumerState; + this.threadSafeConsumerState = threadSafeConsumerState; } AsyncKafkaConsumer(LogContext logContext, @@ -663,7 +663,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, new RebalanceCallbackMetricsManager(metrics) ); ApiVersions apiVersions = new ApiVersions(); - this.sharedConsumerState = new SharedConsumerState( + this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( logContext, metadata, subscriptions, @@ -680,7 +680,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, backgroundEventHandler, false, kafkaConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); Supplier requestManagersSupplier = RequestManagers.supplier( @@ -700,7 +700,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - sharedConsumerState + threadSafeConsumerState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -716,7 +716,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, networkClientDelegateSupplier, requestManagersSupplier, kafkaConsumerMetrics, - sharedConsumerState); + threadSafeConsumerState); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = new CompletableEventReaper(logContext); } @@ -733,7 +733,7 @@ ApplicationEventHandler build( final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState + final ThreadSafeConsumerState threadSafeConsumerState ); } @@ -1879,7 +1879,7 @@ private boolean updateFetchPositions(final Timer timer) { // Instead, let the *application thread* determine if any partitions need their positions updated. If not, // the application thread can skip sending an event to the network thread that will simply end up coming // to the same conclusion, albeit much slower. - if (sharedConsumerState.canSkipUpdateFetchPositions()) + if (threadSafeConsumerState.canSkipUpdateFetchPositions()) return true; try { 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 ead5ffcc1034a..5843009eb0bdb 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 @@ -63,7 +63,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; private final AsyncConsumerMetrics asyncConsumerMetrics; - private final SharedExceptionReference metadataError; + private final ThreadSafeExceptionReference metadataError; private ApplicationEventProcessor applicationEventProcessor; private NetworkClientDelegate networkClientDelegate; private RequestManagers requestManagers; @@ -81,7 +81,7 @@ public ConsumerNetworkThread(LogContext logContext, Supplier networkClientDelegateSupplier, Supplier requestManagersSupplier, AsyncConsumerMetrics asyncConsumerMetrics, - SharedConsumerState sharedConsumerState) { + ThreadSafeConsumerState threadSafeConsumerState) { super(BACKGROUND_THREAD_NAME, true); this.time = time; this.log = logContext.logger(getClass()); @@ -92,7 +92,7 @@ public ConsumerNetworkThread(LogContext logContext, this.requestManagersSupplier = requestManagersSupplier; this.running = true; this.asyncConsumerMetrics = asyncConsumerMetrics; - this.metadataError = sharedConsumerState.metadataError(); + this.metadataError = threadSafeConsumerState.metadataError(); } @Override 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 829415222dc8e..194f4b1a957d6 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,7 +70,7 @@ public class NetworkClientDelegate implements AutoCloseable { private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private final SharedExceptionReference metadataError; + private final ThreadSafeExceptionReference metadataError; private final boolean notifyMetadataErrorsViaErrorQueue; private final AsyncConsumerMetrics asyncConsumerMetrics; @@ -83,7 +83,7 @@ public NetworkClientDelegate( final BackgroundEventHandler backgroundEventHandler, final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState) { + final ThreadSafeConsumerState threadSafeConsumerState) { this.time = time; this.client = client; this.metadata = metadata; @@ -92,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 = sharedConsumerState.metadataError(); + this.metadataError = threadSafeConsumerState.metadataError(); this.notifyMetadataErrorsViaErrorQueue = notifyMetadataErrorsViaErrorQueue; this.asyncConsumerMetrics = asyncConsumerMetrics; } @@ -449,7 +449,7 @@ public static Supplier supplier(final Time time, final BackgroundEventHandler backgroundEventHandler, final boolean notifyMetadataErrorsViaErrorQueue, final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState) { + final ThreadSafeConsumerState threadSafeConsumerState) { return new CachedSupplier<>() { @Override protected NetworkClientDelegate create() { @@ -463,7 +463,7 @@ protected NetworkClientDelegate create() { metadata, throttleTimeSensor, clientTelemetrySender); - return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics, sharedConsumerState); + return new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics, threadSafeConsumerState); } }; } 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 4003300a2f6c9..e97cece405d5c 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 @@ -92,7 +92,7 @@ public final class OffsetsRequestManager implements RequestManager, ClusterResou private final NetworkClientDelegate networkClientDelegate; private final CommitRequestManager commitRequestManager; private final long defaultApiTimeoutMs; - private final SharedExceptionReference updatePositionsError; + private final ThreadSafeExceptionReference positionsUpdateError; /** * This holds the last OffsetFetch request triggered to retrieve committed offsets to update @@ -112,7 +112,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final CommitRequestManager commitRequestManager, - final SharedConsumerState sharedConsumerState, + final ThreadSafeAsyncConsumerState threadSafeConsumerState, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -133,13 +133,13 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; - this.offsetFetcherUtils = sharedConsumerState.offsetFetcherUtils(); + this.offsetFetcherUtils = threadSafeConsumerState.offsetFetcherUtils(); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); this.commitRequestManager = commitRequestManager; - this.updatePositionsError = sharedConsumerState.updatePositionsError(); + this.positionsUpdateError = threadSafeConsumerState.positionsUpdateError(); } private static class PendingFetchCommittedRequest { @@ -257,7 +257,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { } private boolean maybeCompleteWithPreviousException(CompletableFuture result) { - return updatePositionsError.getClearAndRun(result::completeExceptionally); + return positionsUpdateError.getClearAndRun(result::completeExceptionally); } /** @@ -307,7 +307,7 @@ private void cacheExceptionIfEventExpired(CompletableFuture result, long d result.whenComplete((__, error) -> { boolean updatePositionsExpired = time.milliseconds() >= deadlineMs; if (error != null && updatePositionsExpired) { - updatePositionsError.set(error); + positionsUpdateError.set(error); } }); } 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 f6d6c8606e514..73f89b5ea6c8d 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 @@ -168,7 +168,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final SharedConsumerState sharedConsumerState + final ThreadSafeAsyncConsumerState threadSafeConsumerState ) { return new CachedSupplier<>() { @Override @@ -292,7 +292,7 @@ protected RequestManagers create() { apiVersions, networkClientDelegate, commitRequestManager, - sharedConsumerState, + threadSafeConsumerState, logContext); return new RequestManagers( 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 57d04479b9551..03e767a3de301 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 @@ -268,14 +268,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new ShareFetchBuffer(logContext); - SharedConsumerState sharedConsumerState = new SharedConsumerState( - logContext, - metadata, - subscriptions, - time, - groupRebalanceConfig.retryBackoffMs, - apiVersions - ); + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeConsumerState(); final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier( time, logContext, @@ -288,7 +281,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { backgroundEventHandler, true, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); this.completedAcknowledgements = new LinkedList<>(); @@ -321,7 +314,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, - sharedConsumerState); + threadSafeConsumerState); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); @@ -392,15 +385,9 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, time, asyncConsumerMetrics); - SharedConsumerState sharedConsumerState = new SharedConsumerState( - logContext, - metadata, - subscriptions, - time, - config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG) - ); + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeConsumerState(); final Supplier networkClientDelegateSupplier = - () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics, sharedConsumerState); + () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics, threadSafeConsumerState); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, @@ -435,7 +422,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) { networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, - sharedConsumerState); + threadSafeConsumerState); this.backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventProcessor = new BackgroundEventProcessor(); @@ -500,7 +487,7 @@ ApplicationEventHandler build( final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState + final ThreadSafeConsumerState threadSafeConsumerState ); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java similarity index 66% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java index b697dae5003bf..9d6c7e32db60b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java @@ -18,7 +18,11 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -27,11 +31,10 @@ import static java.util.Objects.requireNonNull; /** - * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the - * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread communication, where possible. - * This class compromises on the ideal of keeping state only in the network thread. However, this class only - * relies on classes which are designed to be thread-safe, thus they can be used in both the application - * and network threads. + * This class stores shared state needed by both the application thread and the background thread to avoid costly + * inter-thread communication, where possible. This class compromises on the ideal of keeping state only in the + * background thread. However, this class (and its subclasses) only relies on classes which are designed to be + * thread-safe, thus they can be used in both the application and background threads. * *

    * @@ -41,7 +44,7 @@ *

  • {@link ApiVersions}
  • *
  • {@link ConsumerMetadata}
  • *
  • {@link OffsetFetcherUtils}
  • - *
  • {@link SharedExceptionReference}
  • + *
  • {@link ThreadSafeExceptionReference}
  • *
  • {@link SubscriptionState}
  • *
  • {@link Time}
  • * @@ -49,29 +52,28 @@ *

    * * In general, callers from the application thread should not mutate any of the state contained within this class. - * It should be considered as read-only, and only the network thread should mutate the state. + * It should be considered as read-only, and only the background thread should mutate the state. */ -public class SharedConsumerState { +public class ThreadSafeAsyncConsumerState extends ThreadSafeConsumerState { private final SubscriptionState subscriptions; private final OffsetFetcherUtils offsetFetcherUtils; - private final SharedExceptionReference updatePositionsError; - private final SharedExceptionReference metadataError; + private final ThreadSafeExceptionReference positionsUpdateError; - public SharedConsumerState(LogContext logContext, - ConsumerMetadata metadata, - SubscriptionState subscriptions, - Time time, - long retryBackoffMs) { + public ThreadSafeAsyncConsumerState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs) { this(logContext, metadata, subscriptions, time, retryBackoffMs, new ApiVersions()); } - public SharedConsumerState(LogContext logContext, - ConsumerMetadata metadata, - SubscriptionState subscriptions, - Time time, - long retryBackoffMs, - ApiVersions apiVersions) { + public ThreadSafeAsyncConsumerState(LogContext logContext, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + Time time, + long retryBackoffMs, + ApiVersions apiVersions) { requireNonNull(logContext); requireNonNull(metadata); requireNonNull(subscriptions); @@ -87,20 +89,15 @@ public SharedConsumerState(LogContext logContext, retryBackoffMs, apiVersions ); - this.updatePositionsError = new SharedExceptionReference(); - this.metadataError = new SharedExceptionReference(); + this.positionsUpdateError = new ThreadSafeExceptionReference(); } OffsetFetcherUtils offsetFetcherUtils() { return offsetFetcherUtils; } - public SharedExceptionReference updatePositionsError() { - return updatePositionsError; - } - - public SharedExceptionReference metadataError() { - return metadataError; + public ThreadSafeExceptionReference positionsUpdateError() { + return positionsUpdateError; } /** @@ -134,9 +131,16 @@ public SharedExceptionReference metadataError() { * application thread that the {@link CheckAndUpdatePositionsEvent} can be skipped. * * @return true if all checks pass, false if either of the latter two checks fail + * + * @exception InvalidTopicException Thrown if one or more of the subscribed topics are invalid + * @exception NoOffsetForPartitionException Thrown if no offset could be found for one or more partitions and no + * offset reset policy was configured + * @exception TopicAuthorizationException Thrown if the user is not authorized to fetch from one or more of the + * subscribed topics + * @exception KafkaException Thrown on other unexpected errors */ public boolean canSkipUpdateFetchPositions() { - updatePositionsError.maybeThrowException(); + positionsUpdateError.maybeThrowException(); metadataError.maybeClearAndThrowException(); // In cases of metadata updates, getPartitionsToValidate() will review the partitions and diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java new file mode 100644 index 0000000000000..d8cc198df48d6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java @@ -0,0 +1,49 @@ +/* + * 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; + +/** + * This class stores shared state needed by both the application thread and the background thread to avoid costly + * inter-thread communication, where possible. This class compromises on the ideal of keeping state only in the + * background thread. However, this class (and its subclasses) only relies on classes which are designed to be + * thread-safe, thus they can be used in both the application and background threads. + * + *

    + * + * The following thread-safe classes are used by this class: + * + *

      + *
    • {@link ThreadSafeExceptionReference}
    • + *
    + * + *

    + * + * In general, callers from the application thread should not mutate any of the state contained within this class. + * It should be considered as read-only, and only the background thread should mutate the state. + */ +public class ThreadSafeConsumerState { + + protected final ThreadSafeExceptionReference metadataError; + + public ThreadSafeConsumerState() { + this.metadataError = new ThreadSafeExceptionReference(); + } + + public ThreadSafeExceptionReference metadataError() { + return metadataError; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeExceptionReference.java similarity index 90% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeExceptionReference.java index 4c2219ea4c190..2e0df0cdd490f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedExceptionReference.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeExceptionReference.java @@ -22,10 +22,10 @@ import java.util.function.Consumer; /** - * {@code SharedErrorReference} builds on top of {@link SharedReference} both to be more explicit about the contents - * and to provide utility methods. + * {@code ThreadSafeExceptionReference} builds on top of {@link ThreadSafeReference} both to be more explicit + * about the contents and to provide utility methods. */ -public class SharedExceptionReference extends SharedReference { +public class ThreadSafeExceptionReference extends ThreadSafeReference { private static final Consumer THROW_EXCEPTION = exception -> { // Unwrap the ExecutionException to model what ConsumerUtils.getResult() does when handling exceptions diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReference.java similarity index 92% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReference.java index 8d77f29cd44c8..d5d4bfd59a6a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReference.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReference.java @@ -21,11 +21,10 @@ import java.util.function.Consumer; /** - * {@code SharedReference} serves as a primitive around a {@link AtomicReference} that provides utility methods - * for more ergonomic use. - * @param + * {@code ThreadSafeReference} serves as a thread-safe primitive around an object reference that provides + * utility methods for more ergonomic use. */ -public class SharedReference { +public class ThreadSafeReference { private final AtomicReference reference = new AtomicReference<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java index fd659b7c7d4a5..1123b9b54f35e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -20,7 +20,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; -import org.apache.kafka.clients.consumer.internals.SharedConsumerState; +import org.apache.kafka.clients.consumer.internals.ThreadSafeConsumerState; import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.internals.IdempotentCloser; @@ -58,7 +58,7 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier, final AsyncConsumerMetrics asyncConsumerMetrics, - final SharedConsumerState sharedConsumerState) { + final ThreadSafeConsumerState threadSafeConsumerState) { this.log = logContext.logger(ApplicationEventHandler.class); this.time = time; this.applicationEventQueue = applicationEventQueue; @@ -71,7 +71,7 @@ public ApplicationEventHandler(final LogContext logContext, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); this.networkThread.start(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java index b84395abf30d2..8354e6ab836af 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ApplicationEventHandlerTest.java @@ -43,7 +43,7 @@ public class ApplicationEventHandlerTest { private final NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); private final RequestManagers requestManagers = mock(RequestManagers.class); private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); - private final SharedConsumerState sharedConsumerState = mock(SharedConsumerState.class); + private final ThreadSafeConsumerState threadSafeConsumerState = mock(ThreadSafeConsumerState.class); @Test public void testRecordApplicationEventQueueSize() { @@ -58,7 +58,7 @@ public void testRecordApplicationEventQueueSize() { () -> networkClientDelegate, () -> requestManagers, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState )) { // add event applicationEventHandler.add(new PollEvent(time.milliseconds())); 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 a30f9b0639a9f..9601f2359af2c 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 @@ -275,7 +275,7 @@ private AsyncKafkaConsumer newConsumer( defaultApiTimeoutMs, "group-id", false, - new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs)); + new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 7eaecb9c01cb3..0e02aab34510a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -68,7 +68,7 @@ public class ConsumerNetworkThreadTest { private final RequestManagers requestManagers; private final CompletableEventReaper applicationEventReaper; private final AsyncConsumerMetrics asyncConsumerMetrics; - private final SharedConsumerState sharedConsumerState; + private final ThreadSafeConsumerState threadSafeConsumerState; ConsumerNetworkThreadTest() { this.networkClientDelegate = mock(NetworkClientDelegate.class); @@ -81,7 +81,7 @@ public class ConsumerNetworkThreadTest { this.time = new MockTime(); this.applicationEventQueue = new LinkedBlockingQueue<>(); this.asyncConsumerMetrics = mock(AsyncConsumerMetrics.class); - this.sharedConsumerState = mock(SharedConsumerState.class); + this.threadSafeConsumerState = mock(ThreadSafeConsumerState.class); LogContext logContext = new LogContext(); this.consumerNetworkThread = new ConsumerNetworkThread( @@ -93,7 +93,7 @@ public class ConsumerNetworkThreadTest { () -> networkClientDelegate, () -> requestManagers, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); } @@ -222,7 +222,7 @@ public void testRunOnceRecordTimeBetweenNetworkThreadPoll() { () -> networkClientDelegate, () -> requestManagers, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState )) { consumerNetworkThread.initializeResources(); @@ -257,7 +257,7 @@ public void testRunOnceRecordApplicationEventQueueSizeAndApplicationEventQueueTi () -> networkClientDelegate, () -> requestManagers, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState )) { consumerNetworkThread.initializeResources(); @@ -336,7 +336,7 @@ private void testInitializeResourcesError(Supplier networ networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState )) { assertThrows(KafkaException.class, thread::initializeResources, "initializeResources should fail because one or more Supplier throws an error on get()"); assertDoesNotThrow(thread::cleanup, "cleanup() should not cause an error because all references are checked before use"); 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 4d8724e5892fd..002bae733d1f6 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 @@ -4213,7 +4213,7 @@ public TestableNetworkClientDelegate(Time time, ConsumerMetadata metadata, BackgroundEventHandler backgroundEventHandler, boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions)); + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions)); } @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 7a7e14ea5b802..67df327ba5031 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 @@ -72,18 +72,18 @@ public class NetworkClientDelegateTest { private MockClient client; private Metadata metadata; private BackgroundEventHandler backgroundEventHandler; - private SharedConsumerState sharedConsumerState; + private ThreadSafeConsumerState threadSafeConsumerState; @BeforeEach public void setup() { this.time = new MockTime(0); this.metadata = mock(Metadata.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); - this.sharedConsumerState = mock(SharedConsumerState.class); + this.threadSafeConsumerState = mock(ThreadSafeConsumerState.class); this.client = new MockClient(time, Collections.singletonList(mockNode())); - SharedExceptionReference metadataError = new SharedExceptionReference(); - when(this.sharedConsumerState.metadataError()).thenReturn(metadataError); + ThreadSafeExceptionReference metadataError = new ThreadSafeExceptionReference(); + when(this.threadSafeConsumerState.metadataError()).thenReturn(metadataError); } @Test @@ -223,7 +223,7 @@ public void testPropagateMetadataError() { doThrow(authException).when(metadata).maybeThrowAnyException(); NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate(false); - SharedExceptionReference metadataErrorRef = sharedConsumerState.metadataError(); + ThreadSafeExceptionReference metadataErrorRef = threadSafeConsumerState.metadataError(); assertTrue(metadataErrorRef.getAndClear().isEmpty()); networkClientDelegate.poll(0, time.milliseconds()); @@ -304,7 +304,7 @@ public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErro this.backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, asyncConsumerMetrics, - sharedConsumerState + threadSafeConsumerState ); } 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 c8a7853cc1886..292732c492df7 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 @@ -117,7 +117,7 @@ public void setup() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new SharedConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), logContext ); } @@ -807,7 +807,7 @@ public void testRemoteListOffsetsRequestTimeoutMs() { apiVersions, mock(NetworkClientDelegate.class), commitRequestManager, - new SharedConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), + new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions), logContext ); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 817b6f9218e55..3ac39a3a3e55e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -73,7 +73,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -120,7 +120,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) + new ThreadSafeAsyncConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs, apiVersions) ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); 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 20fd44fa198fc..d703c9dd04303 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 @@ -2752,7 +2752,7 @@ public TestableNetworkClientDelegate(Time time, ConsumerMetadata metadata, BackgroundEventHandler backgroundEventHandler, boolean notifyMetadataErrorsViaErrorQueue) { - super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new SharedConsumerState(logContext, metadata, subscriptions, time, retryBackoffMs)); + super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), new ThreadSafeConsumerState()); } @Override From 3c008531bd29869553fd4e1f7425be8155b33d50 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 29 Aug 2025 09:22:12 -0700 Subject: [PATCH 25/31] Removed superfluous boolean return value from CheckAndUpdatePositionsEvent --- .../internals/OffsetsRequestManager.java | 10 +++---- .../ThreadSafeAsyncConsumerState.java | 5 ++++ .../events/ApplicationEventProcessor.java | 2 +- .../events/CheckAndUpdatePositionsEvent.java | 4 +-- .../internals/AsyncKafkaConsumerTest.java | 26 +++++++++---------- .../internals/OffsetsRequestManagerTest.java | 10 +++---- 6 files changed, 31 insertions(+), 26 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 e97cece405d5c..45a4b6b25640e 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 @@ -225,8 +225,8 @@ public CompletableFuture> fetchO * on {@link SubscriptionState#hasAllFetchPositions()}). It will complete immediately, with true, if all positions * are already available. If some positions are missing, the future will complete once the offsets are retrieved and positions are updated. */ - public CompletableFuture updateFetchPositions(long deadlineMs) { - CompletableFuture result = new CompletableFuture<>(); + public CompletableFuture updateFetchPositions(long deadlineMs) { + CompletableFuture result = new CompletableFuture<>(); try { if (maybeCompleteWithPreviousException(result)) { @@ -237,7 +237,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (subscriptionState.hasAllFetchPositions()) { // All positions are already available - result.complete(true); + result.complete(null); return result; } @@ -246,7 +246,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { if (error != null) { result.completeExceptionally(error); } else { - result.complete(subscriptionState.hasAllFetchPositions()); + result.complete(null); } }); @@ -256,7 +256,7 @@ public CompletableFuture updateFetchPositions(long deadlineMs) { return result; } - private boolean maybeCompleteWithPreviousException(CompletableFuture result) { + private boolean maybeCompleteWithPreviousException(CompletableFuture result) { return positionsUpdateError.getClearAndRun(result::completeExceptionally); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java index 9d6c7e32db60b..284405c948310 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java @@ -58,6 +58,11 @@ public class ThreadSafeAsyncConsumerState extends ThreadSafeConsumerState { private final SubscriptionState subscriptions; private final OffsetFetcherUtils offsetFetcherUtils; + + /** + * 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 ThreadSafeExceptionReference positionsUpdateError; public ThreadSafeAsyncConsumerState(LogContext logContext, 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..e6ad32777157f 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 @@ -457,7 +457,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 = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); future.whenComplete(complete(event.future())); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java index 5f1ced33e3a09..6c8f45598f6ca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java @@ -27,10 +27,10 @@ * offsets and update positions when it gets them. This will first attempt to use the committed offsets if available. If * no committed offsets available, it will use the partition offsets retrieved from the leader. *

    - * The event completes with a boolean indicating if all assigned partitions have valid fetch positions + * The event completes when all assigned partitions have valid fetch positions * (based on {@link SubscriptionState#hasAllFetchPositions()}). */ -public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent { +public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent { public CheckAndUpdatePositionsEvent(long deadlineMs) { super(Type.CHECK_AND_UPDATE_POSITIONS, deadlineMs); 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 9601f2359af2c..812026136b5df 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 @@ -418,7 +418,7 @@ public void testWakeupBeforeCallingPoll() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -440,7 +440,7 @@ public void testWakeupAfterEmptyFetch() { consumer.wakeup(); return Fetch.empty(); }).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -464,7 +464,7 @@ public void testWakeupAfterNonEmptyFetch() { consumer.wakeup(); return Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), "")); }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -483,7 +483,7 @@ public void testCommitInRebalanceCallback() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); @@ -523,7 +523,7 @@ public void testClearWakeupTriggerAfterPoll() { ); doReturn(Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), ""))) .when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -670,7 +670,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { completeCommitAsyncApplicationEventSuccessfully(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); when(metadata.updateVersion()).thenReturn(-1); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); completeAssignmentChangeEventSuccessfully(); consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); @@ -1485,7 +1485,7 @@ public void testListenerCallbacksInvoke(List consumer.poll(Duration.ZERO)); } @@ -1645,7 +1645,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { doAnswer(invocation -> Fetch.forPartition(tp, records, true, new OffsetAndMetadata(3, Optional.of(0), ""))) .when(fetchCollector) .collectFetch(Mockito.any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(singletonList("topic1")); @@ -1665,7 +1665,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); when(metadata.updateVersion()).thenReturn(-1); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(new TopicPartition("t1", 1))); @@ -1704,7 +1704,7 @@ public void testLongPollWaitIsLimited() { }).doAnswer(invocation -> Fetch.forPartition(tp, records, true, nextOffsetAndMetadata) ).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); markReconcileAndAutoCommitCompleteForPollEvent(); // And then poll for up to 10000ms, which should return 2 records without timing out @@ -1802,7 +1802,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); consumer.assign(singleton(tp)); @@ -1841,7 +1841,7 @@ void testReaperInvokedInPoll() { doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); completeTopicSubscriptionChangeEventSuccessfully(); consumer.subscribe(Collections.singletonList("topic")); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); markReconcileAndAutoCommitCompleteForPollEvent(); consumer.poll(Duration.ZERO); verify(backgroundEventReaper).reap(time.milliseconds()); @@ -1898,7 +1898,7 @@ public void testSeekToEnd() { public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() { consumer = newConsumer(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); + when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(null); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); completeAssignmentChangeEventSuccessfully(); completeTopicPatternSubscriptionChangeEventSuccessfully(); 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 292732c492df7..27ef947490800 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 @@ -676,7 +676,7 @@ public void testUpdatePositionsWithCommittedOffsets() { // Call to updateFetchPositions. Should send an OffsetFetch request and use the response to set positions CompletableFuture> fetchResult = new CompletableFuture<>(); when(commitRequestManager.fetchOffsets(initPartitions1, internalFetchCommittedTimeout)).thenReturn(fetchResult); - CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); + CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); assertFalse(updatePositions1.isDone(), "Update positions should wait for the OffsetFetch request"); verify(commitRequestManager).fetchOffsets(initPartitions1, internalFetchCommittedTimeout); @@ -705,13 +705,13 @@ public void testUpdatePositionsWithCommittedOffsetsReusesRequest() { // call to updateFetchPositions. Should send an OffsetFetch request CompletableFuture> fetchResult = new CompletableFuture<>(); when(commitRequestManager.fetchOffsets(initPartitions1, internalFetchCommittedTimeout)).thenReturn(fetchResult); - CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); + CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); assertFalse(updatePositions1.isDone(), "Update positions should wait for the OffsetFetch request"); verify(commitRequestManager).fetchOffsets(initPartitions1, internalFetchCommittedTimeout); clearInvocations(commitRequestManager); // Call to updateFetchPositions again with the same set of initializing partitions should reuse request - CompletableFuture updatePositions2 = requestManager.updateFetchPositions(time.milliseconds()); + CompletableFuture updatePositions2 = requestManager.updateFetchPositions(time.milliseconds()); verify(commitRequestManager, never()).fetchOffsets(initPartitions1, internalFetchCommittedTimeout); // Receive response with committed offsets, should complete both calls @@ -738,7 +738,7 @@ public void testUpdatePositionsDoesNotApplyOffsetsIfPartitionNotInitializingAnym // call to updateFetchPositions will trigger an OffsetFetch request for tp1 (won't complete just yet) CompletableFuture> fetchResult = new CompletableFuture<>(); when(commitRequestManager.fetchOffsets(initPartitions1, internalFetchCommittedTimeout)).thenReturn(fetchResult); - CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); + CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); assertFalse(updatePositions1.isDone()); verify(commitRequestManager).fetchOffsets(initPartitions1, internalFetchCommittedTimeout); clearInvocations(commitRequestManager); @@ -766,7 +766,7 @@ public void testUpdatePositionsDoesNotResetPositionBeforeRetrievingOffsetsForNew // call to updateFetchPositions will trigger an OffsetFetch request for tp1 (won't complete just yet) CompletableFuture> fetchResult = new CompletableFuture<>(); when(commitRequestManager.fetchOffsets(initPartitions1, internalFetchCommittedTimeout)).thenReturn(fetchResult); - CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); + CompletableFuture updatePositions1 = requestManager.updateFetchPositions(time.milliseconds()); assertFalse(updatePositions1.isDone()); verify(commitRequestManager).fetchOffsets(initPartitions1, internalFetchCommittedTimeout); clearInvocations(commitRequestManager); From 04fcedd830c9617ea2f8f2e35cb5edd90642bf29 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 29 Aug 2025 09:27:23 -0700 Subject: [PATCH 26/31] Updated CheckAndUpdatePositionsEvent doc to specify that event completion criteria --- .../internals/events/CheckAndUpdatePositionsEvent.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java index 6c8f45598f6ca..efd4450fb2a35 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CheckAndUpdatePositionsEvent.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.OffsetsRequestManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.TopicPartition; @@ -27,8 +28,9 @@ * offsets and update positions when it gets them. This will first attempt to use the committed offsets if available. If * no committed offsets available, it will use the partition offsets retrieved from the leader. *

    - * The event completes when all assigned partitions have valid fetch positions - * (based on {@link SubscriptionState#hasAllFetchPositions()}). + * The event completes when {@link OffsetsRequestManager} has completed its attempt to update the positions. There + * is no guarantee that {@link SubscriptionState#hasAllFetchPositions()} will return {@code true} just because the + * event has completed. */ public class CheckAndUpdatePositionsEvent extends CompletableApplicationEvent { From 10deac6ab5252992dfc1af7d341e8e5f3fac14b6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 31 Aug 2025 21:07:03 -0700 Subject: [PATCH 27/31] s/Shared/ThreadSafe/g Renames and refactors SharedConsumerState, SharedAutoCommitState, and SharedReconciliationState to ThreadSafe* equivalents. Introduces ThreadSafeConsumerState and ThreadSafeShareConsumerState abstractions. Updates all usages, constructors, and tests to use the new thread-safe classes, improving clarity and consistency in shared state management between application and network threads. --- .../internals/AbstractMembershipManager.java | 12 ++--- .../internals/AsyncKafkaConsumer.java | 22 ++++---- .../internals/CommitRequestManager.java | 10 ++-- .../internals/ConsumerMembershipManager.java | 8 +-- .../consumer/internals/RequestManagers.java | 6 +-- .../internals/ShareMembershipManager.java | 2 +- ...java => ThreadSafeAsyncConsumerState.java} | 35 +++--------- ...te.java => ThreadSafeAutoCommitState.java} | 40 +++++++------- .../internals/ThreadSafeConsumerState.java | 53 +++++++++++++++++++ ...ava => ThreadSafeReconciliationState.java} | 10 ++-- .../ThreadSafeShareConsumerState.java | 31 +++++++++++ .../internals/AsyncKafkaConsumerTest.java | 4 +- .../internals/CommitRequestManagerTest.java | 12 ++--- .../ConsumerMembershipManagerTest.java | 18 +++---- .../internals/RequestManagersTest.java | 8 +-- 15 files changed, 164 insertions(+), 107 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedConsumerState.java => ThreadSafeAsyncConsumerState.java} (69%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedAutoCommitState.java => ThreadSafeAutoCommitState.java} (84%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{SharedReconciliationState.java => ThreadSafeReconciliationState.java} (81%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeShareConsumerState.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index c825bd16feba0..132684abc66be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -118,7 +118,7 @@ public abstract class AbstractMembershipManager impl * Note: per its class name, this state is shared with the application thread, so care must be * taken to evaluate how it's used elsewhere when updating related logic. */ - protected final SharedAutoCommitState autoCommitState; + protected final ThreadSafeAutoCommitState autoCommitState; /** * Logger. @@ -147,7 +147,7 @@ public abstract class AbstractMembershipManager impl /** * If there is a reconciliation running (triggering commit, callbacks) for the - * assignmentReadyToReconcile. {@link SharedReconciliationState#isInProgress()} will be true if + * assignmentReadyToReconcile. {@link ThreadSafeReconciliationState#isInProgress()} will be true if * {@link #maybeReconcile(boolean)} has been triggered after receiving a heartbeat response, or a metadata update. * Calling code should generally favor {@link #reconciliationInProgress()} for its clarity over direct use of * this state. @@ -157,7 +157,7 @@ public abstract class AbstractMembershipManager impl * Note: per its class name, this state is shared with the application thread, so care must be * taken to evaluate how it's used elsewhere when updating related logic. */ - private final SharedReconciliationState reconciliationState; + private final ThreadSafeReconciliationState reconciliationState; /** * True if a reconciliation is in progress and the member rejoined the group since the start @@ -223,7 +223,7 @@ public abstract class AbstractMembershipManager impl Logger log, Time time, RebalanceMetricsManager metricsManager, - SharedConsumerState sharedConsumerState) { + ThreadSafeConsumerState threadSafeConsumerState) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.subscriptions = subscriptions; @@ -235,8 +235,8 @@ public abstract class AbstractMembershipManager impl this.stateUpdatesListeners = new ArrayList<>(); this.time = time; this.metricsManager = metricsManager; - this.autoCommitState = sharedConsumerState.autoCommitState(); - this.reconciliationState = sharedConsumerState.reconciliationState(); + this.autoCommitState = threadSafeConsumerState.autoCommitState(); + this.reconciliationState = threadSafeConsumerState.reconciliationState(); } /** 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 1c5db86d2009d..b2e05c76e48b1 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 @@ -355,7 +355,7 @@ private StreamsRebalanceListener streamsRebalanceListener() { private final long retryBackoffMs; private final int requestTimeoutMs; private final Duration defaultApiTimeoutMs; - private final SharedConsumerState sharedConsumerState; + private final ThreadSafeAsyncConsumerState threadSafeConsumerState; private volatile boolean closed = false; // Init value is needed to avoid NPE in case of exception raised in the constructor private Optional clientTelemetryReporter = Optional.empty(); @@ -476,8 +476,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); - this.sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.newInstance( + this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.newInstance( requireNonNull(logContext), requireNonNull(config), requireNonNull(time) @@ -499,7 +499,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, streamsRebalanceData, - sharedConsumerState + threadSafeConsumerState ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, @@ -570,7 +570,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, int requestTimeoutMs, int defaultApiTimeoutMs, String groupId, - SharedConsumerState sharedConsumerState) { + ThreadSafeAsyncConsumerState threadSafeConsumerState) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = clientId; @@ -593,7 +593,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, this.applicationEventHandler = applicationEventHandler; this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics); this.clientTelemetryReporter = Optional.empty(); - this.sharedConsumerState = sharedConsumerState; + this.threadSafeConsumerState = threadSafeConsumerState; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.backgroundEventHandler = new BackgroundEventHandler( backgroundEventQueue, @@ -668,8 +668,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.newInstance( + this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.newInstance( requireNonNull(logContext), requireNonNull(config), requireNonNull(time) @@ -692,7 +692,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, offsetCommitCallbackInvoker, memberStateListener, Optional.empty(), - sharedConsumerState + threadSafeConsumerState ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -917,7 +917,7 @@ private void sendPollEvent(Timer timer) { // Make sure to let the background thread know that we are still polling. PollEvent event = new PollEvent(currentTimeMs); - if (sharedConsumerState.canSkipWaitingOnPoll(currentTimeMs)) { + if (threadSafeConsumerState.canSkipWaitingOnPoll(currentTimeMs)) { // This will *not* trigger async auto-commits of consumed positions as the shared Timer for // auto-commit interval will not change between the application thread and the network thread. This // is true of the reconciliation state. The state will not change between the SharedConsumerState @@ -1545,7 +1545,7 @@ private void autoCommitOnClose(final Timer timer) { if (groupMetadata.get().isEmpty()) return; - if (sharedConsumerState.isAutoCommitEnabled()) + if (threadSafeConsumerState.isAutoCommitEnabled()) commitSyncAllConsumed(timer); applicationEventHandler.add(new CommitOnCloseEvent()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index f08536d640de9..3499fe8cb53aa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -78,7 +78,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener private final ConsumerMetadata metadata; private final LogContext logContext; private final Logger log; - private final SharedAutoCommitState autoCommitState; + private final ThreadSafeAutoCommitState autoCommitState; private final CoordinatorRequestManager coordinatorRequestManager; private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final OffsetCommitMetricsManager metricsManager; @@ -116,7 +116,7 @@ public CommitRequestManager( final Optional groupInstanceId, final Metrics metrics, final ConsumerMetadata metadata, - final SharedConsumerState sharedConsumerState) { + final ThreadSafeConsumerState threadSafeConsumerState) { this(time, logContext, subscriptions, @@ -130,7 +130,7 @@ public CommitRequestManager( OptionalDouble.empty(), metrics, metadata, - sharedConsumerState); + threadSafeConsumerState); } // Visible for testing @@ -149,13 +149,13 @@ public CommitRequestManager( final OptionalDouble jitter, final Metrics metrics, final ConsumerMetadata metadata, - final SharedConsumerState sharedConsumerState) { + final ThreadSafeConsumerState threadSafeConsumerState) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.time = time; this.logContext = logContext; this.log = logContext.logger(getClass()); this.pendingRequests = new PendingRequests(); - this.autoCommitState = sharedConsumerState.autoCommitState(); + this.autoCommitState = threadSafeConsumerState.autoCommitState(); this.coordinatorRequestManager = coordinatorRequestManager; this.groupId = groupId; this.groupInstanceId = groupInstanceId; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java index d6f871eb66a9b..3c9c76cdd149d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java @@ -151,7 +151,7 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, Metrics metrics, - SharedConsumerState sharedConsumerState) { + ThreadSafeConsumerState threadSafeConsumerState) { this(groupId, groupInstanceId, rackId, @@ -164,7 +164,7 @@ public ConsumerMembershipManager(String groupId, backgroundEventHandler, time, new ConsumerRebalanceMetricsManager(metrics), - sharedConsumerState); + threadSafeConsumerState); } // Visible for testing @@ -180,14 +180,14 @@ public ConsumerMembershipManager(String groupId, BackgroundEventHandler backgroundEventHandler, Time time, RebalanceMetricsManager metricsManager, - SharedConsumerState sharedConsumerState) { + ThreadSafeConsumerState threadSafeConsumerState) { super(groupId, subscriptions, metadata, logContext.logger(ConsumerMembershipManager.class), time, metricsManager, - sharedConsumerState); + threadSafeConsumerState); this.groupInstanceId = groupInstanceId; this.rackId = rackId; this.rebalanceTimeoutMs = rebalanceTimeoutMs; 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 764d08500993e..6bfe31556bd10 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 @@ -168,7 +168,7 @@ public static Supplier supplier(final Time time, final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, final MemberStateListener applicationThreadMemberStateListener, final Optional streamsRebalanceData, - final SharedConsumerState sharedConsumerState + final ThreadSafeConsumerState threadSafeConsumerState ) { return new CachedSupplier<>() { @Override @@ -218,7 +218,7 @@ protected RequestManagers create() { groupRebalanceConfig.groupInstanceId, metrics, metadata, - sharedConsumerState); + threadSafeConsumerState); if (streamsRebalanceData.isPresent()) { streamsMembershipManager = new StreamsMembershipManager( groupRebalanceConfig.groupId, @@ -260,7 +260,7 @@ protected RequestManagers create() { backgroundEventHandler, time, metrics, - sharedConsumerState); + threadSafeConsumerState); // Update the group member ID label in the client telemetry reporter. // According to KIP-1082, the consumer will generate the member ID as the incarnation ID of the process. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java index 34c9a625479a0..162326f57801b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java @@ -107,7 +107,7 @@ public ShareMembershipManager(LogContext logContext, logContext.logger(ShareMembershipManager.class), time, metricsManager, - new SharedConsumerState(SharedAutoCommitState.disabled())); + new ThreadSafeShareConsumerState()); this.rackId = rackId; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java similarity index 69% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java index 0b8df307c9411..716600711cfad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java @@ -22,43 +22,20 @@ /** * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the - * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread communication, where possible. - * This class compromises on the ideal of keeping state only in the network thread. However, this class only - * relies on classes which are designed to be thread-safe, thus they can be used in both the application - * and network threads. - * - *

    - * - * The following thread-safe classes are used by this class: - * - *

      - *
    • {@link SharedAutoCommitState}
    • - *
    • {@link SharedReconciliationState}
    • - *
    - * - *

    - * - * In general, callers from the application thread should not mutate any of the state contained within this class. - * It should be considered as read-only, and only the network thread should mutate the state. + * network thread ({@link ConsumerNetworkThread}) for the {@link AsyncKafkaConsumer}. */ -public class SharedConsumerState { +public class ThreadSafeAsyncConsumerState extends ThreadSafeConsumerState { - private final SharedAutoCommitState autoCommitState; - private final SharedReconciliationState sharedReconciliationState; + private final ThreadSafeAutoCommitState autoCommitState; - public SharedConsumerState(SharedAutoCommitState autoCommitState) { + public ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState autoCommitState) { this.autoCommitState = requireNonNull(autoCommitState); - this.sharedReconciliationState = new SharedReconciliationState(); } - public SharedAutoCommitState autoCommitState() { + public ThreadSafeAutoCommitState autoCommitState() { return autoCommitState; } - public SharedReconciliationState reconciliationState() { - return sharedReconciliationState; - } - /** * This method is used by {@code AsyncKafkaConsumer#poll()} to determine if it can skip waiting for the * {@link PollEvent}. Sending the {@link PollEvent} is in the critical path, and if the application thread @@ -88,7 +65,7 @@ public SharedReconciliationState reconciliationState() { * @return true if all checks pass, false if either of the latter two checks fail */ public boolean canSkipWaitingOnPoll(long currentTimeMs) { - if (sharedReconciliationState.isInProgress()) + if (reconciliationState.isInProgress()) return false; autoCommitState.updateTimer(currentTimeMs); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java similarity index 84% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java index f2a965850e34d..28289f7f76f81 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedAutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java @@ -30,7 +30,7 @@ /** * Encapsulates the state of auto-committing and manages the auto-commit timer. */ -public interface SharedAutoCommitState { +public interface ThreadSafeAutoCommitState { /** * @return {@code true} if auto-commit is enabled as defined in the configuration @@ -74,23 +74,23 @@ public interface SharedAutoCommitState { void setInflightCommit(final boolean hasInflightCommit); - static SharedAutoCommitState enabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { - return new SharedAutoCommitStateEnabled(logContext, time, autoCommitInterval); + static ThreadSafeAutoCommitState enabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + return new AutoCommitEnabled(logContext, time, autoCommitInterval); } - static SharedAutoCommitState enabled(final LogContext logContext, final Time time) { + static ThreadSafeAutoCommitState enabled(final LogContext logContext, final Time time) { return enabled(logContext, time, DEFAULT_AUTO_COMMIT_INTERVAL_MS); } - static SharedAutoCommitState disabled() { - return new SharedAutoCommitStateDisabled(); + static ThreadSafeAutoCommitState disabled() { + return AUTO_COMMIT_DISABLED; } - static SharedAutoCommitState newInstance(final LogContext logContext, - final ConsumerConfig config, - final Time time) { + static ThreadSafeAutoCommitState newInstance(final LogContext logContext, + final ConsumerConfig config, + final Time time) { if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long interval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); return enabled(logContext, time, interval); @@ -99,17 +99,17 @@ static SharedAutoCommitState newInstance(final LogContext logContext, } } - class SharedAutoCommitStateEnabled implements SharedAutoCommitState { + class AutoCommitEnabled implements ThreadSafeAutoCommitState { private final Logger log; private final Timer timer; private final long autoCommitInterval; private boolean hasInflightCommit; - private SharedAutoCommitStateEnabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { - this.log = logContext.logger(SharedAutoCommitState.class); + private AutoCommitEnabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { + this.log = logContext.logger(ThreadSafeAutoCommitState.class); this.timer = time.timer(autoCommitInterval); this.autoCommitInterval = autoCommitInterval; this.hasInflightCommit = false; @@ -160,11 +160,7 @@ public synchronized void setInflightCommit(final boolean hasInflightCommit) { } } - class SharedAutoCommitStateDisabled implements SharedAutoCommitState { - - private SharedAutoCommitStateDisabled() { - } - + ThreadSafeAutoCommitState AUTO_COMMIT_DISABLED = new ThreadSafeAutoCommitState() { @Override public boolean isAutoCommitEnabled() { return false; @@ -199,5 +195,5 @@ public void updateTimer(final long currentTimeMs) { public void setInflightCommit(final boolean inflightCommitStatus) { // No op } - } + }; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java new file mode 100644 index 0000000000000..7a46eec586ead --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeConsumerState.java @@ -0,0 +1,53 @@ +/* + * 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; + +/** + * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the + * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread communication, where possible. + * This class compromises on the ideal of keeping state only in the network thread. However, this class only + * relies on classes which are designed to be thread-safe, thus they can be used in both the application + * and network threads. + * + *

    + * + * The following thread-safe classes are used by this class: + * + *

      + *
    • {@link ThreadSafeAutoCommitState}
    • + *
    • {@link ThreadSafeReconciliationState}
    • + *
    + * + *

    + * + * In general, callers from the application thread should not mutate any of the state contained within this class. + * It should be considered as read-only, and only the network thread should mutate the state. + */ +public abstract class ThreadSafeConsumerState { + + protected final ThreadSafeReconciliationState reconciliationState; + + protected ThreadSafeConsumerState() { + this.reconciliationState = new ThreadSafeReconciliationState(); + } + + public abstract ThreadSafeAutoCommitState autoCommitState(); + + public ThreadSafeReconciliationState reconciliationState() { + return reconciliationState; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReconciliationState.java similarity index 81% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReconciliationState.java index da87305411c8f..b2d7d36c3f210 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedReconciliationState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeReconciliationState.java @@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * This class is constructed from within the {@link SharedConsumerState} instance, which means it's available + * This class is constructed from within the {@link ThreadSafeConsumerState} instance, which means it's available * for both the application and network threads to use. The main user is the {@link AbstractMembershipManager} for - * mutations and the {@link SharedConsumerState#canSkipWaitingOnPoll(long)} method for determining if the costly + * mutations and the {@link ThreadSafeConsumerState#canSkipWaitingOnPoll(long)} method for determining if the costly * {@link PollEvent} can be sent in the background or not. * *

    @@ -31,15 +31,15 @@ * Yes, this class is a wrapper around a simple {@link AtomicBoolean}, but the intention behind dedicating a class * to it hopefully makes the shared nature and its purpose more apparent. */ -public class SharedReconciliationState { +public class ThreadSafeReconciliationState { private final AtomicBoolean value; - public SharedReconciliationState() { + public ThreadSafeReconciliationState() { this(false); } - public SharedReconciliationState(boolean value) { + public ThreadSafeReconciliationState(boolean value) { this.value = new AtomicBoolean(value); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeShareConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeShareConsumerState.java new file mode 100644 index 0000000000000..d471a4f816d54 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeShareConsumerState.java @@ -0,0 +1,31 @@ +/* + * 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.ShareConsumer; + +/** + * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the + * network thread ({@link ConsumerNetworkThread}) for the {@link ShareConsumer}. + */ +public class ThreadSafeShareConsumerState extends ThreadSafeConsumerState { + + @Override + public ThreadSafeAutoCommitState autoCommitState() { + return ThreadSafeAutoCommitState.AUTO_COMMIT_DISABLED; + } +} 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 6928e14512b6a..315f440614f11 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 @@ -255,7 +255,7 @@ private AsyncKafkaConsumer newConsumer( int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; LogContext logContext = new LogContext(); - SharedConsumerState sharedConsumerState = new SharedConsumerState(SharedAutoCommitState.disabled()); + ThreadSafeAsyncConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); return new AsyncKafkaConsumer<>( logContext, "client-id", @@ -275,7 +275,7 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, "group-id", - sharedConsumerState); + threadSafeConsumerState); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index ba04bfa647fbe..27b55dad5f875 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -138,8 +138,8 @@ public void setup() { @Test public void testOffsetFetchRequestStateToStringBase() { ConsumerConfig config = mock(ConsumerConfig.class); - SharedConsumerState sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.newInstance( + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.newInstance( logContext, config, time @@ -160,7 +160,7 @@ public void testOffsetFetchRequestStateToStringBase() { OptionalDouble.of(0), metrics, metadata, - sharedConsumerState); + threadSafeConsumerState); commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString()); Set requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1)); @@ -1574,8 +1574,8 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); ConsumerConfig config = new ConsumerConfig(props); - SharedConsumerState sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.newInstance( + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.newInstance( logContext, config, time @@ -1596,7 +1596,7 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long OptionalDouble.of(0), metrics, metadata, - sharedConsumerState)); + threadSafeConsumerState)); } private ClientResponse buildOffsetFetchClientResponse( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index 5150a985bd70f..1010f6acc0b49 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -141,8 +141,8 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup(String gro } private ConsumerMembershipManager createMembershipManager(String groupInstanceId) { - SharedConsumerState sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.enabled( + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.enabled( LOG_CONTEXT, time ) @@ -150,7 +150,7 @@ private ConsumerMembershipManager createMembershipManager(String groupInstanceId ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState)); + backgroundEventHandler, time, rebalanceMetricsManager, threadSafeConsumerState)); assertMemberIdIsGenerated(manager.memberId()); return manager; } @@ -160,8 +160,8 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( String serverAssignor, String rackId ) { - SharedConsumerState sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.enabled( + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.enabled( LOG_CONTEXT, time ) @@ -169,7 +169,7 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(rackId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState)); + metadata, LOG_CONTEXT, backgroundEventHandler, time, rebalanceMetricsManager, threadSafeConsumerState)); assertMemberIdIsGenerated(manager.memberId()); manager.transitionToJoining(); return manager; @@ -254,8 +254,8 @@ public void testTransitionToFatal() { @Test public void testTransitionToFailedWhenTryingToJoin() { - SharedConsumerState sharedConsumerState = new SharedConsumerState( - SharedAutoCommitState.enabled( + ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( + ThreadSafeAutoCommitState.enabled( LOG_CONTEXT, time ) @@ -263,7 +263,7 @@ public void testTransitionToFailedWhenTryingToJoin() { ConsumerMembershipManager membershipManager = new ConsumerMembershipManager( GROUP_ID, Optional.empty(), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, LOG_CONTEXT, - backgroundEventHandler, time, rebalanceMetricsManager, sharedConsumerState); + backgroundEventHandler, time, rebalanceMetricsManager, threadSafeConsumerState); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 8ffac9605a8cb..bd9fb156f45dd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -50,7 +50,7 @@ public void testMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); - final SharedConsumerState sharedConsumerState = new SharedConsumerState(SharedAutoCommitState.disabled()); + final ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), @@ -68,7 +68,7 @@ public void testMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.empty(), - sharedConsumerState + threadSafeConsumerState ).get(); assertTrue(requestManagers.consumerMembershipManager.isPresent()); assertTrue(requestManagers.streamsMembershipManager.isEmpty()); @@ -92,7 +92,7 @@ public void testStreamMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); - final SharedConsumerState sharedConsumerState = new SharedConsumerState(SharedAutoCommitState.disabled()); + final ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), @@ -110,7 +110,7 @@ public void testStreamMemberStateListenerRegistered() { mock(OffsetCommitCallbackInvoker.class), listener, Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())), - sharedConsumerState + threadSafeConsumerState ).get(); assertTrue(requestManagers.streamsMembershipManager.isPresent()); assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); From b057da60ccad76260d3ecfab7df2630bc716ff84 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 31 Aug 2025 21:11:10 -0700 Subject: [PATCH 28/31] Refactor auto-commit check in consumer state Replaces the isAutoCommitEnabled() method in ThreadSafeAsyncConsumerState with direct access to autoCommitState().isAutoCommitEnabled(). Removes the redundant method and updates AsyncKafkaConsumer to use the new approach. --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 2 +- .../consumer/internals/ThreadSafeAsyncConsumerState.java | 7 ------- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b2e05c76e48b1..14c48cf2f100c 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 @@ -1545,7 +1545,7 @@ private void autoCommitOnClose(final Timer timer) { if (groupMetadata.get().isEmpty()) return; - if (threadSafeConsumerState.isAutoCommitEnabled()) + if (threadSafeConsumerState.autoCommitState().isAutoCommitEnabled()) commitSyncAllConsumed(timer); applicationEventHandler.add(new CommitOnCloseEvent()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java index 716600711cfad..b2f30e6f2efac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java @@ -71,11 +71,4 @@ public boolean canSkipWaitingOnPoll(long currentTimeMs) { autoCommitState.updateTimer(currentTimeMs); return !autoCommitState.shouldAutoCommit(); } - - /** - * Determines if auto-commit is enabled. - */ - public boolean isAutoCommitEnabled() { - return autoCommitState.isAutoCommitEnabled(); - } } From 6e01aa0923f3eb4f4213cfaec68223c7a6a269bb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 31 Aug 2025 21:16:06 -0700 Subject: [PATCH 29/31] Reverted unnecessary changes --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 3 +-- .../consumer/internals/ThreadSafeAutoCommitState.java | 5 ++--- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 3 +-- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 81ef26d2a15ec..3fcdf20953cba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -145,7 +145,6 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true."; - public static final int DEFAULT_AUTO_COMMIT_INTERVAL_MS = 5000; /** * partition.assignment.strategy @@ -463,7 +462,7 @@ public class ConsumerConfig extends AbstractConfig { ENABLE_AUTO_COMMIT_DOC) .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.INT, - DEFAULT_AUTO_COMMIT_INTERVAL_MS, + 5000, atLeast(0), Importance.LOW, AUTO_COMMIT_INTERVAL_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java index 28289f7f76f81..6648b4ea6701c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java @@ -25,8 +25,6 @@ import java.time.Duration; -import static org.apache.kafka.clients.consumer.ConsumerConfig.DEFAULT_AUTO_COMMIT_INTERVAL_MS; - /** * Encapsulates the state of auto-committing and manages the auto-commit timer. */ @@ -81,7 +79,8 @@ static ThreadSafeAutoCommitState enabled(final LogContext logContext, } static ThreadSafeAutoCommitState enabled(final LogContext logContext, final Time time) { - return enabled(logContext, time, DEFAULT_AUTO_COMMIT_INTERVAL_MS); + int defaultAutoCommitIntervalMs = 5000; // This is as defined in ConsumerConfig + return enabled(logContext, time, defaultAutoCommitIntervalMs); } static ThreadSafeAutoCommitState disabled() { 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 315f440614f11..3cb4b6bb59ecd 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 @@ -254,10 +254,9 @@ private AsyncKafkaConsumer newConsumer( long retryBackoffMs = 100L; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; - LogContext logContext = new LogContext(); ThreadSafeAsyncConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); return new AsyncKafkaConsumer<>( - logContext, + new LogContext(), "client-id", new Deserializers<>(new StringDeserializer(), new StringDeserializer(), metrics), fetchBuffer, From f523d996be500449616c0f51ba43b724b9304f71 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 1 Sep 2025 10:30:33 -0700 Subject: [PATCH 30/31] Refactor ThreadSafeAutoCommitState and ThreadSafeConsumerState --- .../internals/AsyncKafkaConsumer.java | 16 ++-------- .../ThreadSafeAsyncConsumerState.java | 23 ++++++++++++++ .../internals/ThreadSafeAutoCommitState.java | 31 +++++-------------- .../internals/AsyncKafkaConsumerTest.java | 2 +- .../internals/CommitRequestManagerTest.java | 22 +++++-------- .../ConsumerMembershipManagerTest.java | 24 ++++++-------- .../internals/RequestManagersTest.java | 4 +-- 7 files changed, 53 insertions(+), 69 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 14c48cf2f100c..0041d08a97bc2 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 @@ -476,13 +476,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); - this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.newInstance( - requireNonNull(logContext), - requireNonNull(config), - requireNonNull(time) - ) - ); + this.threadSafeConsumerState = ThreadSafeAsyncConsumerState.fromConfig(logContext, config, time); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -668,13 +662,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config, kafkaConsumerMetrics ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); - this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.newInstance( - requireNonNull(logContext), - requireNonNull(config), - requireNonNull(time) - ) - ); + this.threadSafeConsumerState = ThreadSafeAsyncConsumerState.fromConfig(logContext, config, time); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java index b2f30e6f2efac..48bde7f735220 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAsyncConsumerState.java @@ -16,9 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; import static java.util.Objects.requireNonNull; +import static org.apache.kafka.clients.consumer.internals.ThreadSafeAutoCommitState.AUTO_COMMIT_DISABLED; /** * This class stores shared state needed by both the application thread ({@link AsyncKafkaConsumer}) and the @@ -32,6 +36,25 @@ public ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState autoCommitState) { this.autoCommitState = requireNonNull(autoCommitState); } + public static ThreadSafeAsyncConsumerState fromConfig(LogContext logContext, ConsumerConfig config, Time time) { + return new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.fromConfig(logContext, config, time)); + } + + static ThreadSafeAsyncConsumerState withAutoCommitEnabled(final LogContext logContext, final Time time) { + int defaultAutoCommitIntervalMs = 5000; // This is as defined in ConsumerConfig + return new ThreadSafeAsyncConsumerState( + new ThreadSafeAutoCommitState.AutoCommitEnabled( + logContext, + time, + defaultAutoCommitIntervalMs + ) + ); + } + + static ThreadSafeAsyncConsumerState withAutoCommitDisabled() { + return new ThreadSafeAsyncConsumerState(AUTO_COMMIT_DISABLED); + } + public ThreadSafeAutoCommitState autoCommitState() { return autoCommitState; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java index 6648b4ea6701c..af450d9a2263f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ThreadSafeAutoCommitState.java @@ -72,29 +72,14 @@ public interface ThreadSafeAutoCommitState { void setInflightCommit(final boolean hasInflightCommit); - static ThreadSafeAutoCommitState enabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { - return new AutoCommitEnabled(logContext, time, autoCommitInterval); - } - - static ThreadSafeAutoCommitState enabled(final LogContext logContext, final Time time) { - int defaultAutoCommitIntervalMs = 5000; // This is as defined in ConsumerConfig - return enabled(logContext, time, defaultAutoCommitIntervalMs); - } - - static ThreadSafeAutoCommitState disabled() { - return AUTO_COMMIT_DISABLED; - } - - static ThreadSafeAutoCommitState newInstance(final LogContext logContext, - final ConsumerConfig config, - final Time time) { + static ThreadSafeAutoCommitState fromConfig(final LogContext logContext, + final ConsumerConfig config, + final Time time) { if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { final long interval = Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - return enabled(logContext, time, interval); + return new AutoCommitEnabled(logContext, time, interval); } else { - return disabled(); + return AUTO_COMMIT_DISABLED; } } @@ -105,9 +90,9 @@ class AutoCommitEnabled implements ThreadSafeAutoCommitState { private final long autoCommitInterval; private boolean hasInflightCommit; - private AutoCommitEnabled(final LogContext logContext, - final Time time, - final long autoCommitInterval) { + public AutoCommitEnabled(final LogContext logContext, + final Time time, + final long autoCommitInterval) { this.log = logContext.logger(ThreadSafeAutoCommitState.class); this.timer = time.timer(autoCommitInterval); this.autoCommitInterval = autoCommitInterval; 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 3cb4b6bb59ecd..8198ae7b25e8a 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 @@ -254,7 +254,7 @@ private AsyncKafkaConsumer newConsumer( long retryBackoffMs = 100L; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; - ThreadSafeAsyncConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); + ThreadSafeAsyncConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitDisabled(); return new AsyncKafkaConsumer<>( new LogContext(), "client-id", diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index 27b55dad5f875..fb1fc4f904574 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -138,14 +138,11 @@ public void setup() { @Test public void testOffsetFetchRequestStateToStringBase() { ConsumerConfig config = mock(ConsumerConfig.class); - ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.newInstance( - logContext, - config, - time - ) + ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.fromConfig( + logContext, + config, + time ); - CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, @@ -1574,14 +1571,11 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); ConsumerConfig config = new ConsumerConfig(props); - ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.newInstance( - logContext, - config, - time - ) + ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.fromConfig( + logContext, + config, + time ); - return spy(new CommitRequestManager( this.time, this.logContext, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index 1010f6acc0b49..6d3cda0344aa5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -141,11 +141,9 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup(String gro } private ConsumerMembershipManager createMembershipManager(String groupInstanceId) { - ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.enabled( - LOG_CONTEXT, - time - ) + ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitEnabled( + LOG_CONTEXT, + time ); ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), @@ -160,11 +158,9 @@ private ConsumerMembershipManager createMembershipManagerJoiningGroup( String serverAssignor, String rackId ) { - ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.enabled( - LOG_CONTEXT, - time - ) + ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitEnabled( + LOG_CONTEXT, + time ); ConsumerMembershipManager manager = spy(new ConsumerMembershipManager( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(rackId), REBALANCE_TIMEOUT, @@ -254,11 +250,9 @@ public void testTransitionToFatal() { @Test public void testTransitionToFailedWhenTryingToJoin() { - ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.enabled( - LOG_CONTEXT, - time - ) + ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitEnabled( + LOG_CONTEXT, + time ); ConsumerMembershipManager membershipManager = new ConsumerMembershipManager( GROUP_ID, Optional.empty(), Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index bd9fb156f45dd..ba744179b0d62 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -50,7 +50,7 @@ public void testMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); - final ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); + final ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitDisabled(); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), @@ -92,7 +92,7 @@ public void testStreamMemberStateListenerRegistered() { config, GroupRebalanceConfig.ProtocolType.CONSUMER ); - final ThreadSafeConsumerState threadSafeConsumerState = new ThreadSafeAsyncConsumerState(ThreadSafeAutoCommitState.disabled()); + final ThreadSafeConsumerState threadSafeConsumerState = ThreadSafeAsyncConsumerState.withAutoCommitDisabled(); final RequestManagers requestManagers = RequestManagers.supplier( new MockTime(), new LogContext(), From 8f563d2eab083e0c8dd72face1f775e1523c9ae9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 18 Sep 2025 13:29:54 -0700 Subject: [PATCH 31/31] Minor changes to reduce diff noise --- .../consumer/internals/AsyncKafkaConsumer.java | 15 ++++++++++----- .../consumer/internals/ConsumerNetworkThread.java | 4 ++-- .../internals/AsyncKafkaConsumerTest.java | 5 ++--- .../internals/FetchRequestManagerTest.java | 2 +- .../internals/OffsetsRequestManagerTest.java | 7 +++---- 5 files changed, 18 insertions(+), 15 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 b4dfe6bccb8ab..96ae317361c60 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 @@ -427,10 +427,12 @@ public AsyncKafkaConsumer(final ConsumerConfig config, // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); - - ; this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.fromConfig(logContext, config, time), + ThreadSafeAutoCommitState.fromConfig( + logContext, + config, + time + ), logContext, metadata, subscriptions, @@ -634,7 +636,11 @@ public AsyncKafkaConsumer(final ConsumerConfig config, ); ApiVersions apiVersions = new ApiVersions(); this.threadSafeConsumerState = new ThreadSafeAsyncConsumerState( - ThreadSafeAutoCommitState.fromConfig(logContext, config, time), + ThreadSafeAutoCommitState.fromConfig( + logContext, + config, + time + ), logContext, metadata, subscriptions, @@ -855,7 +861,6 @@ public ConsumerRecords poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - do { sendPollEvent(timer); 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 5843009eb0bdb..5f2e1dbadfea2 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 @@ -381,8 +381,8 @@ private void maybeFailOnMetadataError(List> events) { if (subscriptionMetadataEvent.isEmpty()) return; - metadataError.getClearAndRun(e -> - subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(e)) + metadataError.getClearAndRun(metadataError -> + subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError)) ); } } 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 af2dabef5b603..4b7f9d947e91d 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 @@ -257,9 +257,8 @@ private AsyncKafkaConsumer newConsumer( long retryBackoffMs = 100L; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 1000; - LogContext logContext = new LogContext(); return new AsyncKafkaConsumer<>( - logContext, + new LogContext(), "client-id", new Deserializers<>(new StringDeserializer(), new StringDeserializer(), metrics), fetchBuffer, @@ -277,7 +276,7 @@ private AsyncKafkaConsumer newConsumer( requestTimeoutMs, defaultApiTimeoutMs, "group-id", - new ThreadSafeAsyncConsumerState(AUTO_COMMIT_DISABLED, logContext, metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); + new ThreadSafeAsyncConsumerState(AUTO_COMMIT_DISABLED, new LogContext(), metadata, subscriptions, time, retryBackoffMs, new ApiVersions())); } @Test 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 efb3b81594b71..c971ead835368 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 @@ -4211,7 +4211,7 @@ public TestableNetworkClientDelegate(Time time, ConsumerConfig config, LogContext logContext, KafkaClient client, - ConsumerMetadata metadata, + Metadata metadata, BackgroundEventHandler backgroundEventHandler, boolean notifyMetadataErrorsViaErrorQueue) { super(time, config, logContext, client, metadata, backgroundEventHandler, notifyMetadataErrorsViaErrorQueue, mock(AsyncConsumerMetrics.class), mock(ThreadSafeAsyncConsumerState.class)); 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 29327aa57c47c..bfea920bfcd1e 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 @@ -799,7 +799,6 @@ public void testRemoteListOffsetsRequestTimeoutMs() { int requestTimeoutMs = 100; int defaultApiTimeoutMs = 500; // Overriding the requestManager to provide different request and default API timeout - LogContext logContext = new LogContext(); requestManager = new OffsetsRequestManager( subscriptionState, metadata, @@ -811,10 +810,10 @@ public void testRemoteListOffsetsRequestTimeoutMs() { mock(NetworkClientDelegate.class), commitRequestManager, new ThreadSafeAsyncConsumerState( - new ThreadSafeAutoCommitState.AutoCommitEnabled(logContext, time, 1000), - logContext, metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions + new ThreadSafeAutoCommitState.AutoCommitEnabled(new LogContext(), time, 1000), + new LogContext(), metadata, subscriptionState, time, RETRY_BACKOFF_MS, apiVersions ), - logContext + new LogContext() ); Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1,