From 666d74b2394964547492d4ce34c62f03b8fd456f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 22 Jan 2024 16:41:19 -0800 Subject: [PATCH 001/130] WIP MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Don't look at me—I'm hideous! --- .../internals/AsyncKafkaConsumer.java | 107 ++++++++++-------- .../internals/CommitRequestManager.java | 11 +- .../consumer/internals/ConsumerUtils.java | 11 ++ .../internals/CoordinatorRequestManager.java | 12 +- .../internals/FetchRequestManager.java | 10 +- .../internals/HeartbeatRequestManager.java | 15 ++- .../consumer/internals/MembershipManager.java | 2 +- .../internals/MembershipManagerImpl.java | 6 +- .../internals/NetworkClientDelegate.java | 21 +--- .../internals/OffsetsRequestManager.java | 65 +++++++---- .../consumer/internals/RequestManagers.java | 8 +- .../TopicMetadataRequestManager.java | 41 +++---- .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventHandler.java | 9 +- .../events/ApplicationEventProcessor.java | 95 ++++++++++------ .../events/CommitAsyncApplicationEvent.java | 83 ++++++++++++++ ...t.java => CommitSyncApplicationEvent.java} | 24 +--- .../events/CompletableApplicationEvent.java | 28 ++--- ...FetchCommittedOffsetsApplicationEvent.java | 16 +-- .../events/LeaveOnCloseApplicationEvent.java | 7 +- .../events/ListOffsetsApplicationEvent.java | 21 +--- .../ResetPositionsApplicationEvent.java | 6 +- .../events/TopicMetadataApplicationEvent.java | 32 +++--- .../events/UnsubscribeApplicationEvent.java | 7 +- .../ValidatePositionsApplicationEvent.java | 6 +- .../internals/AsyncKafkaConsumerTest.java | 26 ++--- .../internals/ConsumerNetworkThreadTest.java | 27 ++--- .../internals/ConsumerTestBuilder.java | 7 +- .../internals/NetworkClientDelegateTest.java | 33 ++---- .../TopicMetadataRequestManagerTest.java | 3 +- .../events/ApplicationEventProcessorTest.java | 2 +- 31 files changed, 430 insertions(+), 313 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CommitApplicationEvent.java => CommitSyncApplicationEvent.java} (75%) 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 c6b9efa23aef5..b59d3e887da2d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -42,7 +42,8 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitAsyncApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; @@ -95,6 +96,7 @@ import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -749,9 +751,19 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { + precommitCheck(offsets); + CompletableFuture future; + + if (offsets.isEmpty()) { + future = CompletableFuture.completedFuture(null); + } else { + final CommitAsyncApplicationEvent commitEvent = new CommitAsyncApplicationEvent(offsets); + applicationEventHandler.add(commitEvent); + future = commitEvent.future(); + } + // Commit without timer to indicate that the commit should be triggered without // waiting for a response. - CompletableFuture future = commit(offsets, false, Optional.empty()); future.whenComplete((r, t) -> { if (callback == null) { if (t != null) { @@ -767,28 +779,13 @@ public void commitAsync(Map offsets, OffsetCo } } - // Visible for testing - CompletableFuture commit(final Map offsets, - final boolean isWakeupable, - final Optional retryTimeoutMs) { + void precommitCheck(final Map offsets) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); maybeThrowInvalidGroupIdException(); log.debug("Committing offsets: {}", offsets); offsets.forEach(this::updateLastSeenEpochIfNewer); - - if (offsets.isEmpty()) { - return CompletableFuture.completedFuture(null); - } - - final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, retryTimeoutMs); - if (isWakeupable) { - // the task can only be woken up if the top level API call is commitSync - wakeupTrigger.setActiveTask(commitEvent.future()); - } - applicationEventHandler.add(commitEvent); - return commitEvent.future(); } @Override @@ -920,13 +917,15 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - time.timer(timeout)); + applicationEventHandler.add(event); + final Map committedOffsets = event.get(); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -973,11 +972,11 @@ public List partitionsFor(String topic, Duration timeout) { } final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timeout.toMillis()); + new TopicMetadataApplicationEvent(topic, time.timer(timeout)); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1002,10 +1001,10 @@ public Map> listTopics(Duration timeout) { } final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timeout.toMillis()); + new TopicMetadataApplicationEvent(time.timer(timeout)); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); } finally { wakeupTrigger.clearTask(); } @@ -1073,16 +1072,23 @@ public Map offsetsForTimes(Map offsetsByTimes = new HashMap<>(timestampsToSearch.size()); + for (Map.Entry entry : timestampsToSearch.entrySet()) + offsetsByTimes.put(entry.getKey(), null); + return offsetsByTimes; + } + + final ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( + timestampsToSearch, + true, + time.timer(timeout) + ); - return applicationEventHandler.addAndGet(listOffsetsEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(listOffsetsEvent); } finally { release(); } @@ -1123,12 +1129,15 @@ private Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + Timer timer = time.timer(timeout); ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( timestampToSearch, - false); + false, + timer + ); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent, - time.timer(timeout)); + listOffsetsEvent + ); return offsetAndTimestampMap .entrySet() .stream() @@ -1254,7 +1263,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", this::maybeInvokeCommitCallbacks, @@ -1333,11 +1342,16 @@ public void commitSync(Map offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - Timer requestTimer = time.timer(timeout.toMillis()); - // Commit with a timer to control how long the request should be retried until it - // gets a successful response or non-retriable error. - CompletableFuture commitFuture = commit(offsets, true, Optional.of(timeout.toMillis())); - ConsumerUtils.getResult(commitFuture, requestTimer); + precommitCheck(offsets); + + if (!offsets.isEmpty()) { + Timer timer = time.timer(timeout); + final CommitSyncApplicationEvent commitEvent = new CommitSyncApplicationEvent(offsets, timer); + + // the task can only be woken up if the top level API call is commitSync + wakeupTrigger.setActiveTask(commitEvent.future()); + applicationEventHandler.addAndGet(commitEvent); + } } finally { wakeupTrigger.clearTask(); kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart); @@ -1447,10 +1461,10 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); + Timer timer = time.timer(Long.MAX_VALUE); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(timer); applicationEventHandler.add(unsubscribeApplicationEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); - Timer timer = time.timer(Long.MAX_VALUE); try { processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); @@ -1548,7 +1562,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer)); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1571,7 +1585,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer)); return true; } catch (TimeoutException e) { return false; @@ -1604,8 +1618,9 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsApplicationEvent event = new FetchCommittedOffsetsApplicationEvent( initializingPartitions, - timer.remainingMs()); - final Map offsets = applicationEventHandler.addAndGet(event, timer); + timer + ); + final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { 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 4ab1084eaa479..54139787df39c 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 @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; +import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.DisconnectException; @@ -303,17 +304,17 @@ public CompletableFuture maybeAutoCommitAllConsumedNow( } /** - * Handles {@link org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent}. It creates an + * Handles {@link CommitSyncApplicationEvent}. It creates an * {@link OffsetCommitRequestState} and enqueue it to send later. */ public CompletableFuture addOffsetCommitRequest(final Map offsets, - final Optional expirationTimeMs, + final long timeoutMs, final boolean retryOnStaleEpoch) { if (offsets.isEmpty()) { log.debug("Skipping commit of empty offsets"); return CompletableFuture.completedFuture(null); } - return pendingRequests.addOffsetCommitRequest(offsets, expirationTimeMs, retryOnStaleEpoch).future; + return pendingRequests.addOffsetCommitRequest(offsets, timeoutMs, retryOnStaleEpoch).future; } /** @@ -495,7 +496,9 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { NetworkClientDelegate.UnsentRequest resp = new NetworkClientDelegate.UnsentRequest( builder, - coordinatorRequestManager.coordinator()); + coordinatorRequestManager.coordinator(), + timer + ); resp.whenComplete( (response, throwable) -> { try { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index bc7590c38f819..7c68fc8d07535 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -50,6 +50,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -206,6 +207,16 @@ public static void refreshCommittedOffsets(final Map void chain(final CompletableFuture a, final CompletableFuture b) { + a.whenComplete((value, exception) -> { + if (exception != null) { + b.completeExceptionally(exception); + } else { + b.complete(value); + } + }); + } + public static T getResult(Future future, Timer timer) { try { return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index d6a72812a52f9..447ddd1ba91d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.requests.FindCoordinatorResponse; 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.Objects; @@ -52,6 +53,7 @@ public class CoordinatorRequestManager implements RequestManager { private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000; private final Time time; + private final long requestTimeoutMs; private final Logger log; private final BackgroundEventHandler backgroundEventHandler; private final String groupId; @@ -64,15 +66,17 @@ public class CoordinatorRequestManager implements RequestManager { public CoordinatorRequestManager( final Time time, final LogContext logContext, + final long requestTimeoutMs, final long retryBackoffMs, final long retryBackoffMaxMs, - final BackgroundEventHandler errorHandler, + final BackgroundEventHandler backgroundEventHandler, final String groupId ) { Objects.requireNonNull(groupId); this.time = time; this.log = logContext.logger(this.getClass()); - this.backgroundEventHandler = errorHandler; + this.requestTimeoutMs = requestTimeoutMs; + this.backgroundEventHandler = backgroundEventHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( logContext, @@ -110,9 +114,11 @@ NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long curren FindCoordinatorRequestData data = new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) .setKey(this.groupId); + Timer timer = time.timer(requestTimeoutMs); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder(data), - Optional.empty() + Optional.empty(), + timer ); return unsentRequest.whenComplete((clientResponse, throwable) -> { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index bef9f2c9cad5d..c68953956edde 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; @@ -41,6 +42,7 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager { private final NetworkClientDelegate networkClientDelegate; + private final long requestTimeoutMs; FetchRequestManager(final LogContext logContext, final Time time, @@ -50,9 +52,11 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager final FetchBuffer fetchBuffer, final FetchMetricsManager metricsManager, final NetworkClientDelegate networkClientDelegate, - final ApiVersions apiVersions) { + final ApiVersions apiVersions, + final long requestTimeoutMs) { super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time, apiVersions); this.networkClientDelegate = networkClientDelegate; + this.requestTimeoutMs = requestTimeoutMs; } @Override @@ -102,6 +106,8 @@ public PollResult pollOnClose() { private PollResult pollInternal(Map fetchRequests, ResponseHandler successHandler, ResponseHandler errorHandler) { + Timer timer = time.timer(requestTimeoutMs); + List requests = fetchRequests.entrySet().stream().map(entry -> { final Node fetchTarget = entry.getKey(); final FetchSessionHandler.FetchRequestData data = entry.getValue(); @@ -113,7 +119,7 @@ private PollResult pollInternal(Map successHandler.handle(fetchTarget, data, clientResponse); }; - return new UnsentRequest(request, Optional.of(fetchTarget)).whenComplete(responseHandler); + return new UnsentRequest(request, Optional.of(fetchTarget), timer).whenComplete(responseHandler); }).collect(Collectors.toList()); return new PollResult(requests); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index 11846eedeafdc..55b2ca1e7c3ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; +import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; @@ -39,6 +40,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.SortedSet; import java.util.TreeSet; import java.util.stream.Collectors; @@ -249,12 +251,16 @@ private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final long curr } private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final boolean ignoreResponse) { + Optional coordinator = coordinatorRequestManager.coordinator(); + ConsumerGroupHeartbeatRequest.Builder requestBuilder = new ConsumerGroupHeartbeatRequest.Builder(heartbeatState.buildRequestData()); NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( - new ConsumerGroupHeartbeatRequest.Builder(this.heartbeatState.buildRequestData()), - coordinatorRequestManager.coordinator()); - if (ignoreResponse) + requestBuilder, + coordinator, + pollTimer + ); + if (ignoreResponse) { return logResponse(request); - else + } else { return request.whenComplete((response, exception) -> { if (response != null) { onResponse((ConsumerGroupHeartbeatResponse) response.responseBody(), request.handler().completionTimeMs()); @@ -262,6 +268,7 @@ private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final boolean i onFailure(exception, request.handler().completionTimeMs()); } }); + } } private NetworkClientDelegate.UnsentRequest logResponse(final NetworkClientDelegate.UnsentRequest request) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index f2e1291a52e58..cc664f269b670 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -124,7 +124,7 @@ public interface MembershipManager { * @return Future that will complete when the callback execution completes and the heartbeat * to leave the group has been sent out. */ - CompletableFuture leaveGroup(); + CompletableFuture leaveGroup(long timeoutMs); /** * @return True if the member should send heartbeat to the coordinator without waiting for diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index ee95a0e3a509d..0da1de3ff692c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -203,7 +203,7 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource * requests in cases where a currently assigned topic is in the target assignment (new * partition assigned, or revoked), but it is not present the Metadata cache at that moment. * The cache is cleared when the subscription changes ({@link #transitionToJoining()}, the - * member fails ({@link #transitionToFatal()} or leaves the group ({@link #leaveGroup()}). + * member fails ({@link #transitionToFatal()} or leaves the group ({@link MembershipManager#leaveGroup(long)}). */ private final Map assignedTopicNamesCache; @@ -240,7 +240,7 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource private int memberEpochOnReconciliationStart; /** - * If the member is currently leaving the group after a call to {@link #leaveGroup()}}, this + * If the member is currently leaving the group after a call to {@link MembershipManager#leaveGroup(long)}}, this * will have a future that will complete when the ongoing leave operation completes * (callbacks executed and heartbeat request to leave is sent out). This will be empty is the * member is not leaving. @@ -584,7 +584,7 @@ private void registerForMetadataUpdates() { * {@inheritDoc} */ @Override - public CompletableFuture leaveGroup() { + public CompletableFuture leaveGroup(long timeoutMs) { if (state == MemberState.UNSUBSCRIBED || state == MemberState.FATAL) { // Member is not part of the group. No-op and return completed future to avoid // unnecessary transitions. 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 141f5f955c8b5..b0cad1d4523c5 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 @@ -59,7 +59,6 @@ public class NetworkClientDelegate implements AutoCloseable { private final KafkaClient client; private final Time time; private final Logger log; - private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; @@ -72,7 +71,6 @@ public NetworkClientDelegate( this.client = client; this.log = logContext.logger(getClass()); this.unsentRequests = new ArrayDeque<>(); - this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); } @@ -234,9 +232,7 @@ public void addAll(final List requests) { } public void add(final UnsentRequest r) { - Objects.requireNonNull(r); - r.setTimer(this.time, this.requestTimeoutMs); - unsentRequests.add(r); + unsentRequests.add(Objects.requireNonNull(r)); } public static class PollResult { @@ -267,23 +263,16 @@ public static class UnsentRequest { private final AbstractRequest.Builder requestBuilder; private final FutureCompletionHandler handler; private final Optional node; // empty if random node can be chosen - - private Timer timer; + private final Timer timer; public UnsentRequest(final AbstractRequest.Builder requestBuilder, - final Optional node) { + final Optional node, + final Timer timer) { Objects.requireNonNull(requestBuilder); this.requestBuilder = requestBuilder; this.node = node; this.handler = new FutureCompletionHandler(); - } - - void setTimer(final Time time, final long requestTimeoutMs) { - this.timer = time.timer(requestTimeoutMs); - } - - Timer timer() { - return timer; + this.timer = timer; } CompletableFuture future() { 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 34f4b30c44dd3..fa8e0bb9b3748 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 @@ -39,6 +39,7 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; 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.ArrayList; @@ -81,7 +82,6 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis private final Set requestsToRetry; private final List requestsToSend; - private final long requestTimeoutMs; private final Time time; private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; @@ -93,7 +93,6 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, - final long requestTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final BackgroundEventHandler backgroundEventHandler, @@ -114,7 +113,6 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; this.time = time; - this.requestTimeoutMs = requestTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; @@ -153,7 +151,8 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { */ public CompletableFuture> fetchOffsets( final Map timestampsToSearch, - final boolean requireTimestamps) { + final boolean requireTimestamps, + long timeoutMs) { if (timestampsToSearch.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyMap()); } @@ -175,7 +174,8 @@ public CompletableFuture> fetchOffsets( } }); - fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState); + Timer timer = time.timer(timeoutMs); + fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState, timer); return listOffsetsRequestState.globalResult.thenApply(result -> OffsetFetcherUtils.buildOffsetsForTimesResult(timestampsToSearch, result.fetchedOffsets)); @@ -193,7 +193,7 @@ public CompletableFuture> fetchOffsets( * an error is received in the response, it will be saved to be thrown on the next call to * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ - public CompletableFuture resetPositionsIfNeeded() { + public CompletableFuture resetPositionsIfNeeded(long timeoutMs) { Map offsetResetTimestamps; try { @@ -206,7 +206,7 @@ public CompletableFuture resetPositionsIfNeeded() { if (offsetResetTimestamps.isEmpty()) return CompletableFuture.completedFuture(null); - return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps); + return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps, timeoutMs); } /** @@ -221,14 +221,14 @@ public CompletableFuture resetPositionsIfNeeded() { * detected, a {@link LogTruncationException} will be saved in memory, to be thrown on the * next call to this function. */ - public CompletableFuture validatePositionsIfNeeded() { + public CompletableFuture validatePositionsIfNeeded(long timeoutMs) { Map partitionsToValidate = offsetFetcherUtils.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return CompletableFuture.completedFuture(null); } - return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate); + return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate, timeoutMs); } /** @@ -237,7 +237,8 @@ public CompletableFuture validatePositionsIfNeeded() { */ private void fetchOffsetsByTimes(final Map timestampsToSearch, final boolean requireTimestamps, - final ListOffsetsRequestState listOffsetsRequestState) { + final ListOffsetsRequestState listOffsetsRequestState, + final Timer timer) { if (timestampsToSearch.isEmpty()) { // Early return if empty map to avoid wrongfully raising StaleMetadataException on // empty grouping @@ -245,7 +246,7 @@ private void fetchOffsetsByTimes(final Map timestampsToSea } try { List unsentRequests = buildListOffsetsRequests( - timestampsToSearch, requireTimestamps, listOffsetsRequestState); + timestampsToSearch, requireTimestamps, listOffsetsRequestState, timer); requestsToSend.addAll(unsentRequests); } catch (StaleMetadataException e) { requestsToRetry.add(listOffsetsRequestState); @@ -259,11 +260,12 @@ public void onUpdate(ClusterResource clusterResource) { // fetchOffsetsByTimes call if any of the requests being retried fails List requestsToProcess = new ArrayList<>(requestsToRetry); requestsToRetry.clear(); + Timer timer = time.timer(55); requestsToProcess.forEach(requestState -> { Map timestampsToSearch = new HashMap<>(requestState.remainingToSearch); requestState.remainingToSearch.clear(); - fetchOffsetsByTimes(timestampsToSearch, requestState.requireTimestamps, requestState); + fetchOffsetsByTimes(timestampsToSearch, requestState.requireTimestamps, requestState, timer); }); } @@ -280,7 +282,8 @@ public void onUpdate(ClusterResource clusterResource) { private List buildListOffsetsRequests( final Map timestampsToSearch, final boolean requireTimestamps, - final ListOffsetsRequestState listOffsetsRequestState) { + final ListOffsetsRequestState listOffsetsRequestState, + final Timer timer) { log.debug("Building ListOffsets request for partitions {}", timestampsToSearch); Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch, Optional.of(listOffsetsRequestState)); @@ -319,7 +322,9 @@ private List buildListOffsetsRequests( node, entry.getValue(), requireTimestamps, - unsentRequests); + unsentRequests, + timer + ); partialResult.whenComplete((result, error) -> { if (error != null) { @@ -340,7 +345,8 @@ private CompletableFuture buildListOffsetRequestToNode( Node node, Map targetTimes, boolean requireTimestamps, - List unsentRequests) { + List unsentRequests, + Timer timer) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder .forConsumer(requireTimestamps, isolationLevel, false) .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); @@ -350,7 +356,9 @@ private CompletableFuture buildListOffsetRequestToNode( NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( builder, - Optional.ofNullable(node)); + Optional.ofNullable(node), + timer + ); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, error) -> { @@ -385,23 +393,27 @@ private CompletableFuture buildListOffsetRequestToNode( * complete. */ private CompletableFuture sendListOffsetsRequestsAndResetPositions( - final Map timestampsToSearch) { + final Map timestampsToSearch, + long timeoutMs) { Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch, Optional.empty()); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); + final Timer timer = time.timer(timeoutMs); timestampsToSearchByNode.forEach((node, resetTimestamps) -> { subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), - time.milliseconds() + requestTimeoutMs); + time.milliseconds() + timeoutMs); CompletableFuture partialResult = buildListOffsetRequestToNode( node, resetTimestamps, false, - unsentRequests); + unsentRequests, + timer + ); partialResult.whenComplete((result, error) -> { if (error == null) { @@ -446,15 +458,17 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( */ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePositions( - Map partitionsToValidate) { + Map partitionsToValidate, + long timeoutMs) { final Map> regrouped = regroupFetchPositionsByLeader(partitionsToValidate); - long nextResetTimeMs = time.milliseconds() + requestTimeoutMs; + long nextResetTimeMs = time.milliseconds() + timeoutMs; final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); + final Timer timer = time.timer(timeoutMs); regrouped.forEach((node, fetchPositions) -> { if (node.isEmpty()) { @@ -481,7 +495,7 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi subscriptionState.setNextAllowedRetry(fetchPositions.keySet(), nextResetTimeMs); CompletableFuture partialResult = - buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests); + buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests, timer); partialResult.whenComplete((offsetsResult, error) -> { if (error == null) { @@ -520,7 +534,8 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi private CompletableFuture buildOffsetsForLeaderEpochRequestToNode( final Node node, final Map fetchPositions, - List unsentRequests) { + final List unsentRequests, + final Timer timer) { AbstractRequest.Builder builder = OffsetsForLeaderEpochUtils.prepareRequest(fetchPositions); @@ -528,7 +543,9 @@ private CompletableFuture build NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( builder, - Optional.ofNullable(node)); + Optional.ofNullable(node), + timer + ); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, 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 6dd9e01216343..1cbb53704159d 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 @@ -131,7 +131,6 @@ protected RequestManagers create() { fetchConfig.isolationLevel, time, retryBackoffMs, - requestTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, @@ -144,10 +143,12 @@ protected RequestManagers create() { fetchBuffer, fetchMetricsManager, networkClientDelegate, - apiVersions); + apiVersions, + requestTimeoutMs); final TopicMetadataRequestManager topic = new TopicMetadataRequestManager( logContext, - config); + config, + time); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; CoordinatorRequestManager coordinator = null; @@ -157,6 +158,7 @@ protected RequestManagers create() { Optional serverAssignor = Optional.ofNullable(config.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); coordinator = new CoordinatorRequestManager(time, logContext, + requestTimeoutMs, retryBackoffMs, retryBackoffMaxMs, backgroundEventHandler, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 75a5ed08d1554..ae5655c1e9b39 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -29,6 +29,8 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; 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.Collections; @@ -67,21 +69,23 @@ public class TopicMetadataRequestManager implements RequestManager { private final long retryBackoffMaxMs; private final Logger log; private final LogContext logContext; + private final Time time; - public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config) { + public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config, final Time time) { logContext = context; log = logContext.logger(getClass()); inflightRequests = new LinkedList<>(); retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); allowAutoTopicCreation = config.getBoolean(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG); + this.time = time; } @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { // Prune any requests which have timed out List expiredRequests = inflightRequests.stream() - .filter(req -> req.isExpired(currentTimeMs)) + .filter(req -> req.timer.isExpired()) .collect(Collectors.toList()); expiredRequests.forEach(TopicMetadataRequestState::expire); @@ -99,10 +103,10 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * * @return the future of the metadata request. */ - public CompletableFuture>> requestAllTopicsMetadata(final long expirationTimeMs) { + public CompletableFuture>> requestAllTopicsMetadata(final long timeoutMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, - expirationTimeMs, + time.timer(timeoutMs), retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); @@ -115,11 +119,12 @@ public CompletableFuture>> requestAllTopicsMetad * @param topic to be requested. * @return the future of the metadata request. */ - public CompletableFuture>> requestTopicMetadata(final String topic, final long expirationTimeMs) { + public CompletableFuture>> requestTopicMetadata(final String topic, + final long timeoutMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, topic, - expirationTimeMs, + time.timer(timeoutMs), retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); @@ -134,11 +139,11 @@ List inflightRequests() { class TopicMetadataRequestState extends RequestState { private final String topic; private final boolean allTopics; - private final long expirationTimeMs; + private final Timer timer; CompletableFuture>> future; public TopicMetadataRequestState(final LogContext logContext, - final long expirationTimeMs, + final Timer timer, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, @@ -146,12 +151,12 @@ public TopicMetadataRequestState(final LogContext logContext, future = new CompletableFuture<>(); this.topic = null; this.allTopics = true; - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } public TopicMetadataRequestState(final LogContext logContext, final String topic, - final long expirationTimeMs, + final Timer timer, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, @@ -159,7 +164,7 @@ public TopicMetadataRequestState(final LogContext logContext, future = new CompletableFuture<>(); this.topic = topic; this.allTopics = false; - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } /** @@ -167,7 +172,7 @@ public TopicMetadataRequestState(final LogContext logContext, * {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest} if needed. */ private Optional send(final long currentTimeMs) { - if (currentTimeMs >= expirationTimeMs) { + if (timer.isExpired()) { return Optional.empty(); } @@ -183,10 +188,6 @@ private Optional send(final long currentTim return Optional.of(createUnsentRequest(request)); } - private boolean isExpired(final long currentTimeMs) { - return currentTimeMs >= expirationTimeMs; - } - private void expire() { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); @@ -196,7 +197,9 @@ private NetworkClientDelegate.UnsentRequest createUnsentRequest( final MetadataRequest.Builder request) { NetworkClientDelegate.UnsentRequest unsent = new NetworkClientDelegate.UnsentRequest( request, - Optional.empty()); + Optional.empty(), + timer + ); return unsent.whenComplete((response, exception) -> { if (response == null) { @@ -210,7 +213,7 @@ private NetworkClientDelegate.UnsentRequest createUnsentRequest( private void handleError(final Throwable exception, final long completionTimeMs) { if (exception instanceof RetriableException) { - if (completionTimeMs >= expirationTimeMs) { + if (timer.isExpired()) { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); } else { @@ -228,7 +231,7 @@ private void handleResponse(final ClientResponse response) { future.complete(res); inflightRequests.remove(this); } catch (RetriableException e) { - if (responseTimeMs >= expirationTimeMs) { + if (timer.isExpired()) { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); } else { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index 4396df2785368..078fc2137c103 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -24,7 +24,7 @@ public abstract class ApplicationEvent { public enum Type { - COMMIT, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + COMMIT_SYNC, COMMIT_ASYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, COMMIT_ON_CLOSE, LEAVE_ON_CLOSE 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 7535edf5970b3..f7ac2e30481be 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 @@ -22,7 +22,6 @@ import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -99,18 +98,16 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link CompletableApplicationEvent#get()} and {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread - * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { + public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); - Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return event.get(timer); + return event.get(); } @Override 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 24a7acf39a9c7..67c2ce1703b10 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 @@ -16,11 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.MembershipManager; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; @@ -32,7 +34,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; @@ -69,8 +70,12 @@ public boolean process() { @Override public void process(ApplicationEvent event) { switch (event.type()) { - case COMMIT: - process((CommitApplicationEvent) event); + case COMMIT_SYNC: + process((CommitSyncApplicationEvent) event); + return; + + case COMMIT_ASYNC: + process((CommitAsyncApplicationEvent) event); return; case POLL: @@ -139,7 +144,7 @@ private void process(final PollApplicationEvent event) { requestManagers.heartbeatRequestManager.ifPresent(hrm -> hrm.resetPollTimer(event.pollTimeMs())); } - private void process(final CommitApplicationEvent event) { + private void process(final CommitSyncApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id // upfront, so we should never get to this block. @@ -149,8 +154,30 @@ private void process(final CommitApplicationEvent event) { } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - Optional expirationTimeMs = event.retryTimeoutMs().map(this::getExpirationTimeForTimeout); - event.chain(manager.addOffsetCommitRequest(event.offsets(), expirationTimeMs, false)); + CompletableFuture future = manager.addOffsetCommitRequest( + event.offsets(), + event.remainingMs(), + false + ); + chain(event, future); + } + + private void process(final CommitAsyncApplicationEvent event) { + if (!requestManagers.commitRequestManager.isPresent()) { + // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id + // upfront, so we should never get to this block. + Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set"); + event.future().completeExceptionally(exception); + return; + } + + CommitRequestManager manager = requestManagers.commitRequestManager.get(); + CompletableFuture future = manager.addOffsetCommitRequest( + event.offsets(), + Long.MAX_VALUE, + false + ); + ConsumerUtils.chain(future, event.future()); } private void process(final FetchCommittedOffsetsApplicationEvent event) { @@ -160,15 +187,17 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); - event.chain(manager.addOffsetFetchRequest(event.partitions(), expirationTimeMs)); + CompletableFuture> future = manager.addOffsetFetchRequest( + event.partitions(), + event.remainingMs() + ); + chain(event, future); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { metadata.requestUpdateForNewTopics(); } - /** * Commit all consumed if auto-commit is enabled. Note this will trigger an async commit, * that will not be retried if the commit request fails. @@ -183,10 +212,13 @@ private void process(final AssignmentChangeApplicationEvent event) { } private void process(final ListOffsetsApplicationEvent event) { - final CompletableFuture> future = - requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), - event.requireTimestamps()); - event.chain(future); + final CompletableFuture> future; + future = requestManagers.offsetsRequestManager.fetchOffsets( + event.timestampsToSearch(), + event.requireTimestamps(), + event.remainingMs() + ); + chain(event, future); } /** @@ -218,31 +250,31 @@ private void process(final UnsubscribeApplicationEvent event) { return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - CompletableFuture result = membershipManager.leaveGroup(); - event.chain(result); + CompletableFuture result = membershipManager.leaveGroup(event.remainingMs()); + chain(event, result); } private void process(final ResetPositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); - event.chain(result); + CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(event.remainingMs()); + chain(event, result); } private void process(final ValidatePositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); - event.chain(result); + CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(event.remainingMs()); + chain(event, result); } private void process(final TopicMetadataApplicationEvent event) { + final long timeoutMs = event.remainingMs(); final CompletableFuture>> future; - long expirationTimeMs = getExpirationTimeForTimeout(event.getTimeoutMs()); if (event.isAllTopics()) { - future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); + future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(timeoutMs); } else { - future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); + future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), timeoutMs); } - event.chain(future); + chain(event, future); } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { @@ -273,22 +305,13 @@ private void process(final LeaveOnCloseApplicationEvent event) { Objects.requireNonNull(requestManagers.heartbeatRequestManager.get().membershipManager(), "Expecting " + "membership manager to be non-null"); log.debug("Leaving group before closing"); - CompletableFuture future = membershipManager.leaveGroup(); + CompletableFuture future = membershipManager.leaveGroup(event.remainingMs()); // The future will be completed on heartbeat sent - event.chain(future); + chain(event, future); } - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; + private void chain(CompletableApplicationEvent event, CompletableFuture future) { + ConsumerUtils.chain(future, event.future()); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java new file mode 100644 index 0000000000000..6ab9530593ac9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +public class CommitAsyncApplicationEvent extends ApplicationEvent { + + private final CompletableFuture future; + + /** + * Offsets to commit per partition. + */ + private final Map offsets; + + /** + * Create new event to commit offsets. + */ + public CommitAsyncApplicationEvent(final Map offsets) { + super(Type.COMMIT_ASYNC); + this.offsets = Collections.unmodifiableMap(offsets); + this.future = new CompletableFuture<>(); + + for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { + if (offsetAndMetadata.offset() < 0) { + throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); + } + } + } + + public Map offsets() { + return offsets; + } + + public CompletableFuture future() { + return future; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + CommitAsyncApplicationEvent that = (CommitAsyncApplicationEvent) o; + + return offsets.equals(that.offsets) && future.equals(that.future); + } + + @Override + public int hashCode() { + return 31 * super.hashCode() + Objects.hash(offsets, future); + } + + @Override + public String toString() { + return "CommitAsyncApplicationEvent{" + + toStringBase() + + ", offsets=" + offsets + + ", future=" + future + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java similarity index 75% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java index d2205227c4979..aede924ab0c44 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java @@ -18,35 +18,28 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; -import java.util.Optional; -public class CommitApplicationEvent extends CompletableApplicationEvent { +public class CommitSyncApplicationEvent extends CompletableApplicationEvent { /** * Offsets to commit per partition. */ private final Map offsets; - /** - * Time to wait for a response, retrying on retriable errors. If not present, the request is - * triggered without waiting for a response or being retried. - */ - private final Optional retryTimeoutMs; - /** * Create new event to commit offsets. If timer is present, the request will be retried on * retriable errors until the timer expires (sync commit offsets request). If the timer is * not present, the request will be sent without waiting for a response of retrying (async * commit offsets request). */ - public CommitApplicationEvent(final Map offsets, - final Optional retryTimeoutMs) { - super(Type.COMMIT); + public CommitSyncApplicationEvent(final Map offsets, + final Timer timer) { + super(Type.COMMIT_SYNC, timer); this.offsets = Collections.unmodifiableMap(offsets); - this.retryTimeoutMs = retryTimeoutMs; for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { @@ -59,17 +52,13 @@ public Map offsets() { return offsets; } - public Optional retryTimeoutMs() { - return retryTimeoutMs; - } - @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; if (!super.equals(o)) return false; - CommitApplicationEvent that = (CommitApplicationEvent) o; + CommitSyncApplicationEvent that = (CommitSyncApplicationEvent) o; return offsets.equals(that.offsets); } @@ -86,7 +75,6 @@ public String toString() { return "CommitApplicationEvent{" + toStringBase() + ", offsets=" + offsets + - ", retryTimeout=" + (retryTimeoutMs.map(t -> t + "ms").orElse("none")) + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 365c620e0c0c0..b057b1db8b85d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; +import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -30,28 +31,25 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; + private final Timer timer; - protected CompletableApplicationEvent(Type type) { + protected CompletableApplicationEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); + this.timer = timer; } public CompletableFuture future() { return future; } - public T get(Timer timer) { - return ConsumerUtils.getResult(future, timer); + public long remainingMs() { + return timer.remainingMs(); } - public void chain(final CompletableFuture providedFuture) { - providedFuture.whenComplete((value, exception) -> { - if (exception != null) { - this.future.completeExceptionally(exception); - } else { - this.future.complete(value); - } - }); + public T get() { + timer.update(); + return ConsumerUtils.getResult(future, timer); } @Override @@ -62,19 +60,17 @@ public boolean equals(Object o) { CompletableApplicationEvent that = (CompletableApplicationEvent) o; - return future.equals(that.future); + return future.equals(that.future) && timer.equals(that.timer); } @Override public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; + return Objects.hash(future, timer); } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; + return super.toStringBase() + ", future=" + future + ", timer=" + timer; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index 34b2d97705cd9..396ddc9338fb3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -30,26 +31,16 @@ public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicatio */ private final Set partitions; - /** - * Time until which the request will be retried if it fails with a retriable error. - */ - private final long timeoutMs; - public FetchCommittedOffsetsApplicationEvent(final Set partitions, - final long timeoutMs) { - super(Type.FETCH_COMMITTED_OFFSETS); + final Timer timer) { + super(Type.FETCH_COMMITTED_OFFSETS, timer); this.partitions = Collections.unmodifiableSet(partitions); - this.timeoutMs = timeoutMs; } public Set partitions() { return partitions; } - public long timeout() { - return timeoutMs; - } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -73,7 +64,6 @@ public String toString() { return getClass().getSimpleName() + "{" + toStringBase() + ", partitions=" + partitions + - ", timeout=" + timeoutMs + "ms" + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index ee0b6ffa61c7d..15910c609bdd9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -16,9 +16,12 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent() { - super(Type.LEAVE_ON_CLOSE); + + public LeaveOnCloseApplicationEvent(Timer timer) { + super(Type.LEAVE_ON_CLOSE, timer); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index 2466d062726f8..ab5a73a2bf89c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -18,9 +18,9 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; -import java.util.HashMap; import java.util.Map; /** @@ -36,25 +36,14 @@ public class ListOffsetsApplicationEvent extends CompletableApplicationEvent timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps) { - super(Type.LIST_OFFSETS); + public ListOffsetsApplicationEvent(Map timestampToSearch, + boolean requireTimestamps, + Timer timer) { + super(Type.LIST_OFFSETS, timer); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } - /** - * Build result representing that no offsets were found as part of the current event. - * - * @return Map containing all the partitions the event was trying to get offsets for, and - * null {@link OffsetAndTimestamp} as value - */ - public Map emptyResult() { - HashMap offsetsByTimes = new HashMap<>(timestampsToSearch.size()); - for (Map.Entry entry : timestampsToSearch.entrySet()) - offsetsByTimes.put(entry.getKey(), null); - return offsetsByTimes; - } - public Map timestampsToSearch() { return timestampsToSearch; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java index 5d9b07f9de05f..c296adbc68e95 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -24,7 +26,7 @@ */ public class ResetPositionsApplicationEvent extends CompletableApplicationEvent { - public ResetPositionsApplicationEvent() { - super(Type.RESET_POSITIONS); + public ResetPositionsApplicationEvent(Timer timer) { + super(Type.RESET_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index dd6f842cc2674..3d381aa71c22f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -17,28 +17,27 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; import java.util.Objects; public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { + private final String topic; private final boolean allTopics; - private final long timeoutMs; - public TopicMetadataApplicationEvent(final long timeoutMs) { - super(Type.TOPIC_METADATA); + public TopicMetadataApplicationEvent(final Timer timer) { + super(Type.TOPIC_METADATA, timer); this.topic = null; this.allTopics = true; - this.timeoutMs = timeoutMs; } - public TopicMetadataApplicationEvent(final String topic, final long timeoutMs) { - super(Type.TOPIC_METADATA); + public TopicMetadataApplicationEvent(final String topic, final Timer timer) { + super(Type.TOPIC_METADATA, timer); this.topic = topic; this.allTopics = false; - this.timeoutMs = timeoutMs; } public String topic() { @@ -49,15 +48,9 @@ public boolean isAllTopics() { return allTopics; } - public long getTimeoutMs() { - return timeoutMs; - } @Override - public String toString() { - return getClass().getSimpleName() + " {" + toStringBase() + - ", topic=" + topic + - ", allTopics=" + allTopics + - ", timeoutMs=" + timeoutMs + "}"; + public int hashCode() { + return Objects.hash(super.hashCode(), topic, allTopics); } @Override @@ -68,11 +61,14 @@ public boolean equals(Object o) { TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - return topic.equals(that.topic) && (allTopics == that.allTopics) && (timeoutMs == that.timeoutMs); + // TODO: fix this potential NPE + return topic.equals(that.topic) && (allTopics == that.allTopics); } @Override - public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics, timeoutMs); + public String toString() { + return getClass().getSimpleName() + " {" + toStringBase() + + ", topic=" + topic + + ", allTopics=" + allTopics + "}"; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java index a1ccb896fdf57..85dc2d225fcce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -25,8 +27,9 @@ * leave group heartbeat, without waiting for any response or considering timeouts). */ public class UnsubscribeApplicationEvent extends CompletableApplicationEvent { - public UnsubscribeApplicationEvent() { - super(Type.UNSUBSCRIBE); + + public UnsubscribeApplicationEvent(final Timer timer) { + super(Type.UNSUBSCRIBE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java index 3b093e0b68353..a74882a313ca9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -24,7 +26,7 @@ */ public class ValidatePositionsApplicationEvent extends CompletableApplicationEvent { - public ValidatePositionsApplicationEvent() { - super(Type.VALIDATE_POSITIONS); + public ValidatePositionsApplicationEvent(Timer timer) { + super(Type.VALIDATE_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index f6fb2a8928631..50125f4f662dd 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 @@ -33,7 +33,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; @@ -245,9 +245,9 @@ public void testCommitAsyncWithNullCallback() { consumer.commitAsync(offsets, null); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitSyncApplicationEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final CommitSyncApplicationEvent commitEvent = commitEventCaptor.getValue(); assertEquals(offsets, commitEvent.offsets()); assertDoesNotThrow(() -> commitEvent.future().complete(null)); assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); @@ -298,9 +298,9 @@ public void testCommitAsyncWithFencedException() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitSyncApplicationEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final CommitSyncApplicationEvent commitEvent = commitEventCaptor.getValue(); commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); @@ -486,7 +486,7 @@ public void testCommitSyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); } @Test @@ -520,7 +520,7 @@ public void testCommitAsyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); } @Test @@ -633,7 +633,7 @@ public void testAutoCommitSyncEnabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(true, time.timer(100), null); - verify(applicationEventHandler).add(any(CommitApplicationEvent.class)); + verify(applicationEventHandler).add(any(CommitSyncApplicationEvent.class)); } @Test @@ -651,7 +651,7 @@ public void testAutoCommitSyncDisabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(false, time.timer(100), null); - verify(applicationEventHandler, never()).add(any(CommitApplicationEvent.class)); + verify(applicationEventHandler, never()).add(any(CommitSyncApplicationEvent.class)); } private void assertMockCommitCallbackInvoked(final Executable task, @@ -1427,18 +1427,18 @@ private HashMap mockTimestampToSearch() { private void completeCommitApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - CommitApplicationEvent event = invocation.getArgument(0); + CommitSyncApplicationEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); } private void completeCommitApplicationEventExceptionally() { doAnswer(invocation -> { - CommitApplicationEvent event = invocation.getArgument(0); + CommitSyncApplicationEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { 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 4ff652ff49318..cb9e54228148e 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 @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -135,7 +135,7 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testApplicationEvent() { - ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); + ApplicationEvent e = new CommitSyncApplicationEvent(new HashMap<>(), Optional.empty()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor, times(1)).process(e); @@ -151,16 +151,16 @@ public void testMetadataUpdateEvent() { @Test public void testCommitEvent() { - ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); + ApplicationEvent e = new CommitSyncApplicationEvent(new HashMap<>(), Optional.empty()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(CommitApplicationEvent.class)); + verify(applicationEventProcessor).process(any(CommitSyncApplicationEvent.class)); } @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, time.timer(Long.MAX_VALUE)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); @@ -169,7 +169,7 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); + ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(time.timer(Long.MAX_VALUE)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); @@ -180,7 +180,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(time.timer(Long.MAX_VALUE)); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -189,7 +189,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(time.timer(Long.MAX_VALUE)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); @@ -214,7 +214,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", Long.MAX_VALUE)); + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", time.timer(Long.MAX_VALUE))); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); } @@ -226,8 +226,9 @@ void testPollResultTimer() { new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) .setKey("foobar")), - Optional.empty()); - req.setTimer(time, DEFAULT_REQUEST_TIMEOUT_MS); + Optional.empty(), + time.timer(DEFAULT_REQUEST_TIMEOUT_MS) + ); // purposely setting a non-MAX time to ensure it is returning Long.MAX_VALUE upon success NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult( @@ -273,8 +274,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap(), Optional.empty())); - ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap(), Optional.empty()); + CompletableApplicationEvent event1 = spy(new CommitSyncApplicationEvent(Collections.emptyMap(), Optional.empty())); + ApplicationEvent event2 = new CommitSyncApplicationEvent(Collections.emptyMap(), Optional.empty()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 17489eab716c0..c4888c030bcf4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -165,19 +165,19 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA fetchConfig.isolationLevel, time, retryBackoffMs, - requestTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, logContext)); - this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config)); + this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config, time)); if (groupInfo.isPresent()) { GroupInformation gi = groupInfo.get(); CoordinatorRequestManager coordinator = spy(new CoordinatorRequestManager( time, logContext, + requestTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, backgroundEventHandler, @@ -251,7 +251,8 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA fetchBuffer, metricsManager, networkClientDelegate, - apiVersions)); + apiVersions, + requestTimeoutMs)); this.requestManagers = new RequestManagers(logContext, offsetsRequestManager, topicMetadataRequestManager, 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 4fdcf917d6c35..336d71a501cc5 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 @@ -44,7 +44,6 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -64,7 +63,7 @@ public void setup() { @Test public void testSuccessfulResponse() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); prepareFindCoordinatorResponse(Errors.NONE); ncd.add(unsentRequest); @@ -79,7 +78,7 @@ public void testSuccessfulResponse() throws Exception { public void testTimeoutBeforeSend() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { client.setUnreachable(mockNode(), REQUEST_TIMEOUT_MS); - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); time.sleep(REQUEST_TIMEOUT_MS); @@ -92,7 +91,7 @@ public void testTimeoutBeforeSend() throws Exception { @Test public void testTimeoutAfterSend() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); time.sleep(REQUEST_TIMEOUT_MS); @@ -104,7 +103,7 @@ public void testTimeoutAfterSend() throws Exception { @Test public void testEnsureCorrectCompletionTimeOnFailure() { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); long timeMs = time.milliseconds(); unsentRequest.handler().onFailure(timeMs, new TimeoutException()); @@ -114,7 +113,7 @@ public void testEnsureCorrectCompletionTimeOnFailure() { @Test public void testEnsureCorrectCompletionTimeOnComplete() { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); long timeMs = time.milliseconds(); final ClientResponse response = mock(ClientResponse.class); when(response.receivedTimeMs()).thenReturn(timeMs); @@ -123,23 +122,6 @@ public void testEnsureCorrectCompletionTimeOnComplete() { assertEquals(timeMs, unsentRequest.handler().completionTimeMs()); } - @Test - public void testEnsureTimerSetOnAdd() { - NetworkClientDelegate ncd = newNetworkClientDelegate(); - NetworkClientDelegate.UnsentRequest findCoordRequest = newUnsentFindCoordinatorRequest(); - assertNull(findCoordRequest.timer()); - - // NetworkClientDelegate#add - ncd.add(findCoordRequest); - assertEquals(1, ncd.unsentRequests().size()); - assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); - - // NetworkClientDelegate#addAll - ncd.addAll(Collections.singletonList(findCoordRequest)); - assertEquals(1, ncd.unsentRequests().size()); - assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); - } - public NetworkClientDelegate newNetworkClientDelegate() { LogContext logContext = new LogContext(); Properties properties = new Properties(); @@ -150,14 +132,15 @@ public NetworkClientDelegate newNetworkClientDelegate() { return new NetworkClientDelegate(this.time, new ConsumerConfig(properties), logContext, this.client); } - public NetworkClientDelegate.UnsentRequest newUnsentFindCoordinatorRequest() { + public NetworkClientDelegate.UnsentRequest newUnsentFindCoordinatorRequest(long timeoutMs) { Objects.requireNonNull(GROUP_ID); NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() .setKey(GROUP_ID) .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) ), - Optional.empty() + Optional.empty(), + time.timer(timeoutMs) ); return req; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index c7b23150602c4..f759770036389 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -73,7 +73,8 @@ public void setup() { props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager( new LogContext(), - new ConsumerConfig(props))); + new ConsumerConfig(props), + time)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 075552fcd5558..4d6c72e1f4b24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -105,7 +105,7 @@ public void testExpirationCalculation() { @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(); + LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(Long.MAX_VALUE); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); From 9d6ec5840082fed9a5a80e22264a71b27871f8ca Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 15:40:17 -0800 Subject: [PATCH 002/130] Lots more changes --- .../internals/AsyncKafkaConsumer.java | 4 +- .../internals/CommitRequestManager.java | 120 ++++++++++-------- .../consumer/internals/MembershipManager.java | 3 +- .../internals/MembershipManagerImpl.java | 19 +-- .../internals/OffsetsRequestManager.java | 21 ++- .../TopicMetadataRequestManager.java | 8 +- .../internals/events/ApplicationEvent.java | 8 +- .../events/ApplicationEventHandler.java | 12 +- .../events/ApplicationEventProcessor.java | 71 +++++------ .../AssignmentChangeApplicationEvent.java | 8 +- .../internals/events/BackgroundEvent.java | 6 +- .../events/BackgroundEventHandler.java | 3 +- ...Event.java => CommitApplicationEvent.java} | 14 +- .../events/CommitAsyncApplicationEvent.java | 83 ------------ .../events/CommitOnCloseApplicationEvent.java | 7 - .../events/CompletableApplicationEvent.java | 29 ++--- .../events/CompletableBackgroundEvent.java | 28 ++-- .../internals/events/CompletableEvent.java | 1 + ...balanceListenerCallbackCompletedEvent.java | 15 +-- ...rRebalanceListenerCallbackNeededEvent.java | 13 +- .../events/ErrorBackgroundEvent.java | 10 +- ...FetchCommittedOffsetsApplicationEvent.java | 7 +- .../events/GroupMetadataUpdateEvent.java | 8 -- .../events/LeaveOnCloseApplicationEvent.java | 7 - .../events/ListOffsetsApplicationEvent.java | 7 +- .../NewTopicsMetadataUpdateRequestEvent.java | 7 - .../events/PollApplicationEvent.java | 7 +- .../events/TopicMetadataApplicationEvent.java | 6 +- .../internals/AsyncKafkaConsumerTest.java | 26 ++-- .../internals/ConsumerNetworkThreadTest.java | 12 +- 30 files changed, 207 insertions(+), 363 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CommitSyncApplicationEvent.java => CommitApplicationEvent.java} (83%) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.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 b59d3e887da2d..849b00e45dace 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 @@ -43,7 +43,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.CommitAsyncApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; @@ -1346,7 +1346,7 @@ public void commitSync(Map offsets, Duration if (!offsets.isEmpty()) { Timer timer = time.timer(timeout); - final CommitSyncApplicationEvent commitEvent = new CommitSyncApplicationEvent(offsets, timer); + final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, timer); // the task can only be woken up if the top level API call is commitSync wakeupTrigger.setActiveTask(commitEvent.future()); 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 54139787df39c..99b246508d181 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 @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; -import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.DisconnectException; @@ -72,6 +72,8 @@ import static org.apache.kafka.common.protocol.Errors.COORDINATOR_LOAD_IN_PROGRESS; public class CommitRequestManager implements RequestManager, MemberStateListener { + + private final Time time; private final SubscriptionState subscriptions; private final LogContext logContext; private final Logger log; @@ -133,6 +135,7 @@ public CommitRequestManager( final OptionalDouble jitter, final String metricGroupPrefix, final Metrics metrics) { + this.time = time; Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); this.logContext = logContext; this.log = logContext.logger(getClass()); @@ -212,7 +215,7 @@ private static long findMinTime(final Collection request * has elapsed. * * @param offsets Offsets to commit - * @param expirationTimeMs Time until which the request will continue to be retried if it + * @param timer Timer that defines how long to continue to retry the request if it * fails with a retriable error. If not present, the request will be * sent but not retried. * @param checkInterval True if the auto-commit interval expiration should be checked for @@ -226,7 +229,7 @@ private static long findMinTime(final Collection request * completed future if no request is generated. */ private CompletableFuture maybeAutoCommit(final Map offsets, - final Optional expirationTimeMs, + final Timer timer, boolean checkInterval, boolean retryOnStaleEpoch) { if (!autoCommitEnabled()) { @@ -239,7 +242,7 @@ private CompletableFuture maybeAutoCommit(final Map result = addOffsetCommitRequest(offsets, expirationTimeMs, retryOnStaleEpoch) + CompletableFuture result = addOffsetCommitRequest(offsets, timer, retryOnStaleEpoch) .whenComplete(autoCommitCallback(offsets)); autocommit.resetTimer(); autocommit.setInflightCommitStatus(true); @@ -261,8 +264,9 @@ public CompletableFuture maybeAutoCommitAllConsumedAsync() { // Early return to ensure that no action/logging is performed. return CompletableFuture.completedFuture(null); } + Timer timer = time.timer(requestTimeoutMs); Map offsets = subscriptions.allConsumed(); - CompletableFuture result = maybeAutoCommit(offsets, Optional.empty(), true, true); + CompletableFuture result = maybeAutoCommit(offsets, timer, true, true); result.whenComplete((__, error) -> { if (error != null) { if (error instanceof RetriableCommitFailedException) { @@ -284,9 +288,10 @@ public CompletableFuture maybeAutoCommitAllConsumedAsync() { * until the request succeeds or fails with a fatal error. */ public CompletableFuture maybeAutoCommitAllConsumedNow( - final Optional expirationTimeMs, + final long timeoutMs, final boolean retryOnStaleEpoch) { - return maybeAutoCommit(subscriptions.allConsumed(), expirationTimeMs, false, retryOnStaleEpoch); + Timer timer = time.timer(timeoutMs); + return maybeAutoCommit(subscriptions.allConsumed(), timer, false, retryOnStaleEpoch); } private BiConsumer autoCommitCallback(final Map allConsumedOffsets) { @@ -304,17 +309,32 @@ public CompletableFuture maybeAutoCommitAllConsumedNow( } /** - * Handles {@link CommitSyncApplicationEvent}. It creates an + * Handles {@link CommitApplicationEvent}. It creates an + * {@link OffsetCommitRequestState} and enqueue it to send later. + */ + public CompletableFuture addOffsetCommitRequest(final Map offsets, + final Optional deadlineMs, + final boolean retryOnStaleEpoch) { + + if (offsets.isEmpty()) { + log.debug("Skipping commit of empty offsets"); + return CompletableFuture.completedFuture(null); + } + return pendingRequests.addOffsetCommitRequest(offsets, timer, retryOnStaleEpoch).future; + } + + /** + * Handles {@link CommitApplicationEvent}. It creates an * {@link OffsetCommitRequestState} and enqueue it to send later. */ public CompletableFuture addOffsetCommitRequest(final Map offsets, - final long timeoutMs, + final Timer timer, final boolean retryOnStaleEpoch) { if (offsets.isEmpty()) { log.debug("Skipping commit of empty offsets"); return CompletableFuture.completedFuture(null); } - return pendingRequests.addOffsetCommitRequest(offsets, timeoutMs, retryOnStaleEpoch).future; + return pendingRequests.addOffsetCommitRequest(offsets, timer, retryOnStaleEpoch).future; } /** @@ -323,8 +343,8 @@ public CompletableFuture addOffsetCommitRequest(final Map> addOffsetFetchRequest( final Set partitions, - final long expirationTimeMs) { - return pendingRequests.addOffsetFetchRequest(partitions, expirationTimeMs); + final Timer timer) { + return pendingRequests.addOffsetFetchRequest(partitions, timer); } public void updateAutoCommitTimer(final long currentTimeMs) { @@ -417,17 +437,12 @@ private class OffsetCommitRequestState extends RetriableRequestState { private final CompletableFuture future; - /** - * Time until which the request should be retried if it fails with retriable - * errors. If not present, the request is triggered without waiting for a response or - * retrying. - */ - private final Optional expirationTimeMs; + private final Timer timer; OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Optional expirationTimeMs, + final Timer timer, final long retryBackoffMs, final long retryBackoffMaxMs, final MemberInfo memberInfo, @@ -438,14 +453,14 @@ private class OffsetCommitRequestState extends RetriableRequestState { this.groupId = groupId; this.groupInstanceId = groupInstanceId; this.future = new CompletableFuture<>(); - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } // Visible for testing OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Optional expirationTimeMs, + final Timer timer, final long retryBackoffMs, final long retryBackoffMaxMs, final double jitter, @@ -457,7 +472,7 @@ private class OffsetCommitRequestState extends RetriableRequestState { this.groupId = groupId; this.groupInstanceId = groupInstanceId; this.future = new CompletableFuture<>(); - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } public NetworkClientDelegate.UnsentRequest toUnsentRequest() { @@ -600,7 +615,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { future.completeExceptionally(commitExceptionForRetriableError(throwable)); return; } - if (isExpired(currentTimeMs)) { + if (timer.isExpired()) { // Fail requests that allowed retries (sync requests), but expired. future.completeExceptionally(throwable); return; @@ -612,10 +627,6 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { pendingRequests.addOffsetCommitRequest(this); } - private boolean isExpired(final long currentTimeMs) { - return expirationTimeMs.isPresent() && expirationTimeMs.get() <= currentTimeMs; - } - /** * @return True if the requests allows to be retried (sync requests that provide an * expiration time to bound the retries). False if the request does not allow to be @@ -632,8 +643,8 @@ private boolean allowsRetries() { * * @return True if the request expired. */ - private boolean maybeExpire(final long currentTimeMs) { - if (isExpired(currentTimeMs)) { + private boolean maybeExpire() { + if (timer.isExpired()) { future.completeExceptionally(new TimeoutException("OffsetCommit could not complete " + "before timeout expired.")); return true; @@ -719,34 +730,31 @@ class OffsetFetchRequestState extends RetriableRequestState { private final CompletableFuture> future; - /** - * Time until which the request should be retried if it fails with retriable errors. - */ - private final long expirationTimeMs; + private final Timer timer; public OffsetFetchRequestState(final Set partitions, final long retryBackoffMs, final long retryBackoffMaxMs, - final long expirationTimeMs, + final Timer timer, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, retryBackoffMaxMs, memberInfo, true); this.requestedPartitions = partitions; this.future = new CompletableFuture<>(); - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } public OffsetFetchRequestState(final Set partitions, final long retryBackoffMs, final long retryBackoffMaxMs, - final long expirationTimeMs, + final Timer timer, final double jitter, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, 2, retryBackoffMaxMs, jitter, memberInfo, true); this.requestedPartitions = partitions; this.future = new CompletableFuture<>(); - this.expirationTimeMs = expirationTimeMs; + this.timer = timer; } public boolean sameRequest(final OffsetFetchRequestState request) { @@ -773,7 +781,11 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { new ArrayList<>(this.requestedPartitions), throwOnFetchStableOffsetUnsupported); } - return new NetworkClientDelegate.UnsentRequest(builder, coordinatorRequestManager.coordinator()) + return new NetworkClientDelegate.UnsentRequest( + builder, + coordinatorRequestManager.coordinator(), + timer + ) .whenComplete((r, t) -> onResponse(r.receivedTimeMs(), (OffsetFetchResponse) r.responseBody())); } @@ -842,18 +854,14 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { pendingRequests.addOffsetFetchRequest(this); } - private boolean isExpired(final long currentTimeMs) { - return expirationTimeMs <= currentTimeMs; - } - /** * Complete the request future with a TimeoutException if the request expired. No action * taken if the request is still active. * * @return True if the request expired. */ - private boolean maybeExpire(final long currentTimeMs) { - if (isExpired(currentTimeMs)) { + private boolean maybeExpire() { + if (timer.isExpired()) { future.completeExceptionally(new TimeoutException("OffsetFetch request could not " + "complete before timeout expired.")); return true; @@ -962,13 +970,13 @@ boolean hasUnsentRequests() { OffsetCommitRequestState addOffsetCommitRequest( final Map offsets, - final Optional expirationTimeMs, + final Timer timer, final boolean retryOnStaleEpoch) { // TODO: Dedupe committing the same offsets to the same partitions OffsetCommitRequestState requestState = createOffsetCommitRequest( offsets, jitter, - expirationTimeMs, + timer, retryOnStaleEpoch); return addOffsetCommitRequest(requestState); } @@ -981,14 +989,14 @@ OffsetCommitRequestState addOffsetCommitRequest(OffsetCommitRequestState request OffsetCommitRequestState createOffsetCommitRequest(final Map offsets, final OptionalDouble jitter, - final Optional expirationTimeMs, + final Timer timer, final boolean retryOnStaleEpoch) { return jitter.isPresent() ? new OffsetCommitRequestState( offsets, groupId, groupInstanceId, - expirationTimeMs, + timer, retryBackoffMs, retryBackoffMaxMs, jitter.getAsDouble(), @@ -998,7 +1006,7 @@ OffsetCommitRequestState createOffsetCommitRequest(final Map> addOffsetFetch } private CompletableFuture> addOffsetFetchRequest(final Set partitions, - final long expirationTimeMs) { + final Timer timer) { OffsetFetchRequestState request = jitter.isPresent() ? new OffsetFetchRequestState( partitions, retryBackoffMs, retryBackoffMaxMs, - expirationTimeMs, + timer, jitter.getAsDouble(), memberInfo) : new OffsetFetchRequestState( partitions, retryBackoffMs, retryBackoffMaxMs, - expirationTimeMs, + timer, memberInfo); return addOffsetFetchRequest(request); } @@ -1103,16 +1111,16 @@ List drain(final long currentTimeMs) { * Find the unsent commit requests that have expired, remove them and complete their * futures with a TimeoutException. */ - private void failAndRemoveExpiredCommitRequests(final long currentTimeMs) { - unsentOffsetCommits.removeIf(req -> req.maybeExpire(currentTimeMs)); + private void failAndRemoveExpiredCommitRequests() { + unsentOffsetCommits.removeIf(OffsetCommitRequestState::maybeExpire); } /** * Find the unsent fetch requests that have expired, remove them and complete their * futures with a TimeoutException. */ - private void failAndRemoveExpiredFetchRequests(final long currentTimeMs) { - unsentOffsetFetches.removeIf(req -> req.maybeExpire(currentTimeMs)); + private void failAndRemoveExpiredFetchRequests() { + unsentOffsetFetches.removeIf(OffsetFetchRequestState::maybeExpire); } private void clearAll() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index cc664f269b670..6c22dbcaf281c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.utils.Timer; import java.util.Map; import java.util.Optional; @@ -124,7 +125,7 @@ public interface MembershipManager { * @return Future that will complete when the callback execution completes and the heartbeat * to leave the group has been sent out. */ - CompletableFuture leaveGroup(long timeoutMs); + CompletableFuture leaveGroup(Timer timer); /** * @return True if the member should send heartbeat to the coordinator without waiting for diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 0da1de3ff692c..fc4323feb0031 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -38,7 +38,6 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -276,8 +275,6 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private final BackgroundEventHandler backgroundEventHandler; - private final Time time; - public MembershipManagerImpl(String groupId, Optional groupInstanceId, int rebalanceTimeoutMs, @@ -287,8 +284,7 @@ public MembershipManagerImpl(String groupId, ConsumerMetadata metadata, LogContext logContext, Optional clientTelemetryReporter, - BackgroundEventHandler backgroundEventHandler, - Time time) { + BackgroundEventHandler backgroundEventHandler) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -305,7 +301,6 @@ public MembershipManagerImpl(String groupId, this.clientTelemetryReporter = clientTelemetryReporter; this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.backgroundEventHandler = backgroundEventHandler; - this.time = time; } /** @@ -837,9 +832,7 @@ boolean reconcile() { // best effort to commit the offsets in the case where the epoch might have changed while // the current reconciliation is in process. Note this is using the rebalance timeout as // it is the limit enforced by the broker to complete the reconciliation process. - commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow( - Optional.of(getExpirationTimeForTimeout(rebalanceTimeoutMs)), - true); + commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow(rebalanceTimeoutMs, true); // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. commitResult.whenComplete((commitReqResult, commitReqError) -> { @@ -859,14 +852,6 @@ boolean reconcile() { return true; } - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = time.milliseconds() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; - } - /** * Trigger onPartitionsRevoked callbacks if any partitions where revoked. If it succeeds, * proceed to trigger the onPartitionsAssigned (even if no new partitions were added), and 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 fa8e0bb9b3748..ba54a0f146a3a 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 @@ -152,7 +152,7 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { public CompletableFuture> fetchOffsets( final Map timestampsToSearch, final boolean requireTimestamps, - long timeoutMs) { + final Timer timer) { if (timestampsToSearch.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyMap()); } @@ -174,7 +174,6 @@ public CompletableFuture> fetchOffsets( } }); - Timer timer = time.timer(timeoutMs); fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState, timer); return listOffsetsRequestState.globalResult.thenApply(result -> @@ -193,7 +192,7 @@ public CompletableFuture> fetchOffsets( * an error is received in the response, it will be saved to be thrown on the next call to * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ - public CompletableFuture resetPositionsIfNeeded(long timeoutMs) { + public CompletableFuture resetPositionsIfNeeded(Timer timer) { Map offsetResetTimestamps; try { @@ -206,7 +205,7 @@ public CompletableFuture resetPositionsIfNeeded(long timeoutMs) { if (offsetResetTimestamps.isEmpty()) return CompletableFuture.completedFuture(null); - return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps, timeoutMs); + return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps, timer); } /** @@ -221,14 +220,14 @@ public CompletableFuture resetPositionsIfNeeded(long timeoutMs) { * detected, a {@link LogTruncationException} will be saved in memory, to be thrown on the * next call to this function. */ - public CompletableFuture validatePositionsIfNeeded(long timeoutMs) { + public CompletableFuture validatePositionsIfNeeded(Timer timer) { Map partitionsToValidate = offsetFetcherUtils.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return CompletableFuture.completedFuture(null); } - return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate, timeoutMs); + return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate, timer); } /** @@ -394,18 +393,17 @@ private CompletableFuture buildListOffsetRequestToNode( */ private CompletableFuture sendListOffsetsRequestsAndResetPositions( final Map timestampsToSearch, - long timeoutMs) { + final Timer timer) { Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch, Optional.empty()); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); - final Timer timer = time.timer(timeoutMs); timestampsToSearchByNode.forEach((node, resetTimestamps) -> { subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), - time.milliseconds() + timeoutMs); + time.milliseconds() + timer.remainingMs()); CompletableFuture partialResult = buildListOffsetRequestToNode( node, @@ -459,16 +457,15 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( */ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePositions( Map partitionsToValidate, - long timeoutMs) { + Timer timer) { final Map> regrouped = regroupFetchPositionsByLeader(partitionsToValidate); - long nextResetTimeMs = time.milliseconds() + timeoutMs; + long nextResetTimeMs = time.milliseconds() + timer.remainingMs(); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); - final Timer timer = time.timer(timeoutMs); regrouped.forEach((node, fetchPositions) -> { if (node.isEmpty()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index ae5655c1e9b39..71fcb4c93fba9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -103,10 +103,10 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * * @return the future of the metadata request. */ - public CompletableFuture>> requestAllTopicsMetadata(final long timeoutMs) { + public CompletableFuture>> requestAllTopicsMetadata(final Timer timer) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, - time.timer(timeoutMs), + timer, retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); @@ -120,11 +120,11 @@ public CompletableFuture>> requestAllTopicsMetad * @return the future of the metadata request. */ public CompletableFuture>> requestTopicMetadata(final String topic, - final long timeoutMs) { + final Timer timer) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, topic, - time.timer(timeoutMs), + timer, retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index 078fc2137c103..c40f3c22e7e4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -24,7 +24,7 @@ public abstract class ApplicationEvent { public enum Type { - COMMIT_SYNC, COMMIT_ASYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + COMMIT, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, COMMIT_ON_CLOSE, LEAVE_ON_CLOSE @@ -60,9 +60,7 @@ protected String toStringBase() { } @Override - public String toString() { - return "ApplicationEvent{" + - toStringBase() + - '}'; + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } } \ No newline at end of file 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 f7ac2e30481be..e90c453010e40 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 @@ -17,11 +17,13 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -29,6 +31,7 @@ import java.time.Duration; import java.util.Objects; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -40,6 +43,7 @@ public class ApplicationEventHandler implements Closeable { private final Logger log; + private final Time time; private final BlockingQueue applicationEventQueue; private final ConsumerNetworkThread networkThread; private final IdempotentCloser closer = new IdempotentCloser(); @@ -51,6 +55,7 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier) { this.log = logContext.logger(ApplicationEventHandler.class); + this.time = time; this.applicationEventQueue = applicationEventQueue; this.networkThread = new ConsumerNetworkThread(logContext, time, @@ -98,7 +103,8 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#get()} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link CompletableApplicationEvent#future()}, {@link CompletableApplicationEvent#deadlineMs()}, and + * {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread * @return Value that is the result of the event @@ -107,7 +113,9 @@ public long maximumTimeToWait() { public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); add(event); - return event.get(); + CompletableFuture future = event.future(); + Timer timer = time.timer(event.deadlineMs() - time.milliseconds()); + return ConsumerUtils.getResult(future, timer); } @Override 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 67c2ce1703b10..7dd836e1a0833 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 @@ -29,6 +29,8 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; 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.List; @@ -45,15 +47,18 @@ public class ApplicationEventProcessor extends EventProcessor { private final Logger log; + private final Time time; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; public ApplicationEventProcessor(final LogContext logContext, + final Time time, final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); + this.time = time; this.requestManagers = requestManagers; this.metadata = metadata; } @@ -70,12 +75,8 @@ public boolean process() { @Override public void process(ApplicationEvent event) { switch (event.type()) { - case COMMIT_SYNC: - process((CommitSyncApplicationEvent) event); - return; - - case COMMIT_ASYNC: - process((CommitAsyncApplicationEvent) event); + case COMMIT: + process((CommitApplicationEvent) event); return; case POLL: @@ -144,7 +145,7 @@ private void process(final PollApplicationEvent event) { requestManagers.heartbeatRequestManager.ifPresent(hrm -> hrm.resetPollTimer(event.pollTimeMs())); } - private void process(final CommitSyncApplicationEvent event) { + private void process(final CommitApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id // upfront, so we should never get to this block. @@ -153,43 +154,24 @@ private void process(final CommitSyncApplicationEvent event) { return; } + Timer timer = timer(event); CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture future = manager.addOffsetCommitRequest( - event.offsets(), - event.remainingMs(), - false - ); + CompletableFuture future = manager.addOffsetCommitRequest(event.offsets(), timer, false); chain(event, future); } - private void process(final CommitAsyncApplicationEvent event) { - if (!requestManagers.commitRequestManager.isPresent()) { - // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id - // upfront, so we should never get to this block. - Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set"); - event.future().completeExceptionally(exception); - return; - } - - CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture future = manager.addOffsetCommitRequest( - event.offsets(), - Long.MAX_VALUE, - false - ); - ConsumerUtils.chain(future, event.future()); - } - private void process(final FetchCommittedOffsetsApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { event.future().completeExceptionally(new KafkaException("Unable to fetch committed " + "offset because the CommittedRequestManager is not available. Check if group.id was set correctly")); return; } + + Timer timer = timer(event); CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture> future = manager.addOffsetFetchRequest( event.partitions(), - event.remainingMs() + timer ); chain(event, future); } @@ -213,10 +195,11 @@ private void process(final AssignmentChangeApplicationEvent event) { private void process(final ListOffsetsApplicationEvent event) { final CompletableFuture> future; + final Timer timer = timer(event); future = requestManagers.offsetsRequestManager.fetchOffsets( event.timestampsToSearch(), event.requireTimestamps(), - event.remainingMs() + timer ); chain(event, future); } @@ -250,28 +233,31 @@ private void process(final UnsubscribeApplicationEvent event) { return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - CompletableFuture result = membershipManager.leaveGroup(event.remainingMs()); + Timer timer = timer(event); + CompletableFuture result = membershipManager.leaveGroup(timer); chain(event, result); } private void process(final ResetPositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(event.remainingMs()); + Timer timer = timer(event); + CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(timer); chain(event, result); } private void process(final ValidatePositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(event.remainingMs()); + Timer timer = timer(event); + CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(timer); chain(event, result); } private void process(final TopicMetadataApplicationEvent event) { - final long timeoutMs = event.remainingMs(); + final Timer timer = timer(event); final CompletableFuture>> future; if (event.isAllTopics()) { - future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(timeoutMs); + future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(timer); } else { - future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), timeoutMs); + future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), timer); } chain(event, future); @@ -305,7 +291,8 @@ private void process(final LeaveOnCloseApplicationEvent event) { Objects.requireNonNull(requestManagers.heartbeatRequestManager.get().membershipManager(), "Expecting " + "membership manager to be non-null"); log.debug("Leaving group before closing"); - CompletableFuture future = membershipManager.leaveGroup(event.remainingMs()); + final Timer timer = timer(event); + CompletableFuture future = membershipManager.leaveGroup(timer); // The future will be completed on heartbeat sent chain(event, future); } @@ -314,11 +301,16 @@ private void chain(CompletableApplicationEvent event, CompletableFuture event) { + return time.timer(event.deadlineMs() - time.milliseconds()); + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, + final Time time, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, final Supplier requestManagersSupplier) { @@ -328,6 +320,7 @@ protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); return new ApplicationEventProcessor( logContext, + time, applicationEventQueue, requestManagers, metadata diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java index ccf7199f2606b..e5b14f7c0ccb9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java @@ -63,11 +63,7 @@ public int hashCode() { } @Override - public String toString() { - return "AssignmentChangeApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - ", currentTimeMs=" + currentTimeMs + - '}'; + protected String toStringBase() { + return toStringBase() + ", offsets=" + offsets + ", currentTimeMs=" + currentTimeMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index e5d522201ef0a..a61a3a592ed24 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -59,9 +59,7 @@ protected String toStringBase() { } @Override - public String toString() { - return "BackgroundEvent{" + - toStringBase() + - '}'; + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 103493d25314f..a8b669ba7db9c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -25,8 +25,7 @@ /** * An event handler that receives {@link BackgroundEvent background events} from the - * {@link ConsumerNetworkThread network thread} which are then made available to the application thread - * via the {@link BackgroundEventProcessor}. + * {@link ConsumerNetworkThread network thread} which are then made available to the application thread. */ public class BackgroundEventHandler { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java similarity index 83% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index aede924ab0c44..12ebbca030872 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitSyncApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -23,7 +23,7 @@ import java.util.Collections; import java.util.Map; -public class CommitSyncApplicationEvent extends CompletableApplicationEvent { +public class CommitApplicationEvent extends CompletableApplicationEvent { /** * Offsets to commit per partition. @@ -36,8 +36,7 @@ public class CommitSyncApplicationEvent extends CompletableApplicationEvent offsets, - final Timer timer) { + public CommitApplicationEvent(final Map offsets, final Timer timer) { super(Type.COMMIT_SYNC, timer); this.offsets = Collections.unmodifiableMap(offsets); @@ -58,7 +57,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; if (!super.equals(o)) return false; - CommitSyncApplicationEvent that = (CommitSyncApplicationEvent) o; + CommitApplicationEvent that = (CommitApplicationEvent) o; return offsets.equals(that.offsets); } @@ -71,10 +70,7 @@ public int hashCode() { } @Override - public String toString() { - return "CommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - '}'; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java deleted file mode 100644 index 6ab9530593ac9..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitAsyncApplicationEvent.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; - -import java.util.Collections; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; - -public class CommitAsyncApplicationEvent extends ApplicationEvent { - - private final CompletableFuture future; - - /** - * Offsets to commit per partition. - */ - private final Map offsets; - - /** - * Create new event to commit offsets. - */ - public CommitAsyncApplicationEvent(final Map offsets) { - super(Type.COMMIT_ASYNC); - this.offsets = Collections.unmodifiableMap(offsets); - this.future = new CompletableFuture<>(); - - for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { - if (offsetAndMetadata.offset() < 0) { - throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); - } - } - } - - public Map offsets() { - return offsets; - } - - public CompletableFuture future() { - return future; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CommitAsyncApplicationEvent that = (CommitAsyncApplicationEvent) o; - - return offsets.equals(that.offsets) && future.equals(that.future); - } - - @Override - public int hashCode() { - return 31 * super.hashCode() + Objects.hash(offsets, future); - } - - @Override - public String toString() { - return "CommitAsyncApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - ", future=" + future + - '}'; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java index 4cc07e945f9d2..66c69b8c4c236 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java @@ -21,11 +21,4 @@ public class CommitOnCloseApplicationEvent extends ApplicationEvent { public CommitOnCloseApplicationEvent() { super(Type.COMMIT_ON_CLOSE); } - - @Override - public String toString() { - return "CommitOnCloseApplicationEvent{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index b057b1db8b85d..bc64ae379d7e8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; import java.util.Objects; @@ -31,25 +30,22 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; - private final Timer timer; + private final long deadlineMs; protected CompletableApplicationEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); - this.timer = timer; + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } + @Override public CompletableFuture future() { return future; } - public long remainingMs() { - return timer.remainingMs(); - } - - public T get() { - timer.update(); - return ConsumerUtils.getResult(future, timer); + @Override + public long deadlineMs() { + return deadlineMs; } @Override @@ -60,23 +56,16 @@ public boolean equals(Object o) { CompletableApplicationEvent that = (CompletableApplicationEvent) o; - return future.equals(that.future) && timer.equals(that.timer); + return future.equals(that.future) && deadlineMs == that.deadlineMs; } @Override public int hashCode() { - return Objects.hash(future, timer); + return Objects.hash(future, deadlineMs); } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future + ", timer=" + timer; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 640ee6103af9b..3280e3d903f69 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -27,16 +30,24 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableBackgroundEvent(Type type) { + protected CompletableBackgroundEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } + @Override public CompletableFuture future() { return future; } + @Override + public long deadlineMs() { + return deadlineMs; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -45,25 +56,16 @@ public boolean equals(Object o) { CompletableBackgroundEvent that = (CompletableBackgroundEvent) o; - return future.equals(that.future); + return future.equals(that.future) && deadlineMs == that.deadlineMs; } @Override public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; + return Objects.hash(future, deadlineMs); } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 8fdcc20fa8363..56ddd07219913 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -22,4 +22,5 @@ public interface CompletableEvent { CompletableFuture future(); + long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index b260c6154ea5f..abfdf2371a0eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -28,7 +28,7 @@ * Event that signifies that the application thread has executed the {@link ConsumerRebalanceListener} callback. If * the callback execution threw an error, it is included in the event should any event listener want to know. */ -public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent { +public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent implements CompletableEvent { private final ConsumerRebalanceListenerMethodName methodName; private final CompletableFuture future; @@ -47,10 +47,16 @@ public ConsumerRebalanceListenerMethodName methodName() { return methodName; } + @Override public CompletableFuture future() { return future; } + @Override + public long deadlineMs() { + return Long.MAX_VALUE; + } + public Optional error() { return error; } @@ -80,11 +86,4 @@ protected String toStringBase() { ", future=" + future + ", error=" + error; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 7b17c034abdbd..c4c5829048ec6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.time.Duration; import java.util.Collections; @@ -38,8 +39,9 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); + SortedSet partitions, + Timer timer) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } @@ -77,11 +79,4 @@ protected String toStringBase() { ", methodName=" + methodName + ", partitions=" + partitions; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 2945f22986b18..ccf5239b1b1c5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -18,12 +18,15 @@ import org.apache.kafka.common.KafkaException; +import java.util.Objects; + public class ErrorBackgroundEvent extends BackgroundEvent { private final RuntimeException error; public ErrorBackgroundEvent(Throwable t) { super(Type.ERROR); + Objects.requireNonNull(t); this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); } @@ -50,10 +53,7 @@ public int hashCode() { } @Override - public String toString() { - return "ErrorBackgroundEvent{" + - toStringBase() + - ", error=" + error + - '}'; + protected String toStringBase() { + return toStringBase() + ", error=" + error; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index 396ddc9338fb3..b5b7a3988383b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -60,10 +60,7 @@ public int hashCode() { } @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - ", partitions=" + partitions + - '}'; + protected String toStringBase() { + return toStringBase() + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 120e671724209..4000947841287 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -68,12 +68,4 @@ public String toStringBase() { ", memberEpoch=" + memberEpoch + ", memberId='" + memberId + '\''; } - - @Override - public String toString() { - return "GroupMetadataUpdateEvent{" + - toStringBase() + - '}'; - } - } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index 15910c609bdd9..960ea7480ee1f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -23,11 +23,4 @@ public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent commitEventCaptor = ArgumentCaptor.forClass(CommitSyncApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final CommitSyncApplicationEvent commitEvent = commitEventCaptor.getValue(); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); assertEquals(offsets, commitEvent.offsets()); assertDoesNotThrow(() -> commitEvent.future().complete(null)); assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); @@ -298,9 +298,9 @@ public void testCommitAsyncWithFencedException() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitSyncApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final CommitSyncApplicationEvent commitEvent = commitEventCaptor.getValue(); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); @@ -486,7 +486,7 @@ public void testCommitSyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } @Test @@ -520,7 +520,7 @@ public void testCommitAsyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } @Test @@ -633,7 +633,7 @@ public void testAutoCommitSyncEnabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(true, time.timer(100), null); - verify(applicationEventHandler).add(any(CommitSyncApplicationEvent.class)); + verify(applicationEventHandler).add(any(CommitApplicationEvent.class)); } @Test @@ -651,7 +651,7 @@ public void testAutoCommitSyncDisabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(false, time.timer(100), null); - verify(applicationEventHandler, never()).add(any(CommitSyncApplicationEvent.class)); + verify(applicationEventHandler, never()).add(any(CommitApplicationEvent.class)); } private void assertMockCommitCallbackInvoked(final Executable task, @@ -1427,18 +1427,18 @@ private HashMap mockTimestampToSearch() { private void completeCommitApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - CommitSyncApplicationEvent event = invocation.getArgument(0); + CommitApplicationEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } private void completeCommitApplicationEventExceptionally() { doAnswer(invocation -> { - CommitSyncApplicationEvent event = invocation.getArgument(0); + CommitApplicationEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitSyncApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { 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 cb9e54228148e..91fed714a983d 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 @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitSyncApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -135,7 +135,7 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testApplicationEvent() { - ApplicationEvent e = new CommitSyncApplicationEvent(new HashMap<>(), Optional.empty()); + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor, times(1)).process(e); @@ -151,10 +151,10 @@ public void testMetadataUpdateEvent() { @Test public void testCommitEvent() { - ApplicationEvent e = new CommitSyncApplicationEvent(new HashMap<>(), Optional.empty()); + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(CommitSyncApplicationEvent.class)); + verify(applicationEventProcessor).process(any(CommitApplicationEvent.class)); } @Test @@ -274,8 +274,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new CommitSyncApplicationEvent(Collections.emptyMap(), Optional.empty())); - ApplicationEvent event2 = new CommitSyncApplicationEvent(Collections.emptyMap(), Optional.empty()); + CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap(), Optional.empty())); + ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap(), Optional.empty()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); From bbbfec749f9591faae5b5849f9df93e1e3429593 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:11:12 -0800 Subject: [PATCH 003/130] Updates --- .../internals/MembershipManagerImpl.java | 16 +++++++++++----- .../internals/OffsetsRequestManager.java | 7 +++++-- .../internals/TopicMetadataRequestManager.java | 4 +--- 3 files changed, 17 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index fc4323feb0031..45219d1462309 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -202,7 +203,7 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource * requests in cases where a currently assigned topic is in the target assignment (new * partition assigned, or revoked), but it is not present the Metadata cache at that moment. * The cache is cleared when the subscription changes ({@link #transitionToJoining()}, the - * member fails ({@link #transitionToFatal()} or leaves the group ({@link MembershipManager#leaveGroup(long)}). + * member fails ({@link #transitionToFatal()} or leaves the group ({@link #leaveGroup(Timer)}). */ private final Map assignedTopicNamesCache; @@ -239,7 +240,7 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource private int memberEpochOnReconciliationStart; /** - * If the member is currently leaving the group after a call to {@link MembershipManager#leaveGroup(long)}}, this + * If the member is currently leaving the group after a call to {@link #leaveGroup(Timer)}}, this * will have a future that will complete when the ongoing leave operation completes * (callbacks executed and heartbeat request to leave is sent out). This will be empty is the * member is not leaving. @@ -579,7 +580,7 @@ private void registerForMetadataUpdates() { * {@inheritDoc} */ @Override - public CompletableFuture leaveGroup(long timeoutMs) { + public CompletableFuture leaveGroup(Timer timer) { if (state == MemberState.UNSUBSCRIBED || state == MemberState.FATAL) { // Member is not part of the group. No-op and return completed future to avoid // unnecessary transitions. @@ -1198,10 +1199,15 @@ CompletableFuture invokeOnPartitionsLostCallback(Set parti * @return Future that will be chained within the rest of the reconciliation logic */ private CompletableFuture enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, - Set partitions) { + Set partitions, + Timer timer) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent( + methodName, + sortedPartitions, + timer + ); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); 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 ba54a0f146a3a..b1c2766be51db 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 @@ -82,6 +82,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis private final Set requestsToRetry; private final List requestsToSend; + private final long requestTimeoutMs; private final Time time; private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; @@ -93,6 +94,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, + final long requestTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final BackgroundEventHandler backgroundEventHandler, @@ -113,6 +115,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; this.time = time; + this.requestTimeoutMs = requestTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; @@ -259,7 +262,7 @@ public void onUpdate(ClusterResource clusterResource) { // fetchOffsetsByTimes call if any of the requests being retried fails List requestsToProcess = new ArrayList<>(requestsToRetry); requestsToRetry.clear(); - Timer timer = time.timer(55); + Timer timer = time.timer(requestTimeoutMs); requestsToProcess.forEach(requestState -> { Map timestampsToSearch = new HashMap<>(requestState.remainingToSearch); @@ -403,7 +406,7 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( timestampsToSearchByNode.forEach((node, resetTimestamps) -> { subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), - time.milliseconds() + timer.remainingMs()); + time.milliseconds() + requestTimeoutMs); CompletableFuture partialResult = buildListOffsetRequestToNode( node, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 71fcb4c93fba9..9f217477a1e08 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -69,16 +69,14 @@ public class TopicMetadataRequestManager implements RequestManager { private final long retryBackoffMaxMs; private final Logger log; private final LogContext logContext; - private final Time time; - public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config, final Time time) { + public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config) { logContext = context; log = logContext.logger(getClass()); inflightRequests = new LinkedList<>(); retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); allowAutoTopicCreation = config.getBoolean(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG); - this.time = time; } @Override From a0f6fc58fb5ec0ae09e606e80efdb662e70ce883 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:12:33 -0800 Subject: [PATCH 004/130] Reverting --- .../consumer/internals/events/ApplicationEvent.java | 6 ++++-- .../events/AssignmentChangeApplicationEvent.java | 8 ++++++-- .../consumer/internals/events/BackgroundEvent.java | 6 ++++-- .../consumer/internals/events/BackgroundEventHandler.java | 3 ++- 4 files changed, 16 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index c40f3c22e7e4b..4396df2785368 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -60,7 +60,9 @@ protected String toStringBase() { } @Override - public final String toString() { - return getClass().getSimpleName() + "{" + toStringBase() + '}'; + public String toString() { + return "ApplicationEvent{" + + toStringBase() + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java index e5b14f7c0ccb9..ccf7199f2606b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java @@ -63,7 +63,11 @@ public int hashCode() { } @Override - protected String toStringBase() { - return toStringBase() + ", offsets=" + offsets + ", currentTimeMs=" + currentTimeMs; + public String toString() { + return "AssignmentChangeApplicationEvent{" + + toStringBase() + + ", offsets=" + offsets + + ", currentTimeMs=" + currentTimeMs + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index a61a3a592ed24..e5d522201ef0a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -59,7 +59,9 @@ protected String toStringBase() { } @Override - public final String toString() { - return getClass().getSimpleName() + "{" + toStringBase() + '}'; + public String toString() { + return "BackgroundEvent{" + + toStringBase() + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index a8b669ba7db9c..103493d25314f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -25,7 +25,8 @@ /** * An event handler that receives {@link BackgroundEvent background events} from the - * {@link ConsumerNetworkThread network thread} which are then made available to the application thread. + * {@link ConsumerNetworkThread network thread} which are then made available to the application thread + * via the {@link BackgroundEventProcessor}. */ public class BackgroundEventHandler { From 72b96b5cb5e3939d27307a7ee099b433d7121b43 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:14:02 -0800 Subject: [PATCH 005/130] Reverting toString() changes --- .../internals/events/CommitOnCloseApplicationEvent.java | 7 +++++++ .../internals/events/CompletableApplicationEvent.java | 9 ++++++++- .../internals/events/CompletableBackgroundEvent.java | 9 ++++++++- .../ConsumerRebalanceListenerCallbackCompletedEvent.java | 7 +++++++ 4 files changed, 30 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java index 66c69b8c4c236..4cc07e945f9d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java @@ -21,4 +21,11 @@ public class CommitOnCloseApplicationEvent extends ApplicationEvent { public CommitOnCloseApplicationEvent() { super(Type.COMMIT_ON_CLOSE); } + + @Override + public String toString() { + return "CommitOnCloseApplicationEvent{" + + toStringBase() + + '}'; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index bc64ae379d7e8..37f4b308dd25b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -66,6 +66,13 @@ public int hashCode() { @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; + return super.toStringBase() + ", future=" + future; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 3280e3d903f69..f701523d6d2e7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -66,6 +66,13 @@ public int hashCode() { @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; + return super.toStringBase() + ", future=" + future; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index abfdf2371a0eb..a5b43ba500270 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -86,4 +86,11 @@ protected String toStringBase() { ", future=" + future + ", error=" + error; } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } } From ca1f16df5c0759e502617c88fc240722a36ade60 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:16:55 -0800 Subject: [PATCH 006/130] Revert toString() changes --- .../consumer/internals/events/CommitApplicationEvent.java | 7 +++++-- .../ConsumerRebalanceListenerCallbackNeededEvent.java | 7 +++++++ .../consumer/internals/events/ErrorBackgroundEvent.java | 7 +++++-- .../events/FetchCommittedOffsetsApplicationEvent.java | 7 +++++-- .../internals/events/GroupMetadataUpdateEvent.java | 7 +++++++ .../internals/events/LeaveOnCloseApplicationEvent.java | 7 +++++++ .../internals/events/ListOffsetsApplicationEvent.java | 7 ++++--- .../events/NewTopicsMetadataUpdateRequestEvent.java | 7 +++++++ 8 files changed, 47 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index 12ebbca030872..b50bb0af4d507 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -70,7 +70,10 @@ public int hashCode() { } @Override - protected String toStringBase() { - return super.toStringBase() + ", offsets=" + offsets; + public String toString() { + return "CommitApplicationEvent{" + + toStringBase() + + ", offsets=" + offsets + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index c4c5829048ec6..4644ceca01042 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -79,4 +79,11 @@ protected String toStringBase() { ", methodName=" + methodName + ", partitions=" + partitions; } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index ccf5239b1b1c5..1b0d3581728be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -53,7 +53,10 @@ public int hashCode() { } @Override - protected String toStringBase() { - return toStringBase() + ", error=" + error; + public String toString() { + return "ErrorBackgroundEvent{" + + toStringBase() + + ", error=" + error + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index b5b7a3988383b..396ddc9338fb3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -60,7 +60,10 @@ public int hashCode() { } @Override - protected String toStringBase() { - return toStringBase() + ", partitions=" + partitions; + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + ", partitions=" + partitions + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 4000947841287..21ea7a8cf931d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -68,4 +68,11 @@ public String toStringBase() { ", memberEpoch=" + memberEpoch + ", memberId='" + memberId + '\''; } + + @Override + public String toString() { + return "GroupMetadataUpdateEvent{" + + toStringBase() + + '}'; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index 960ea7480ee1f..e03cd7a963855 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -23,4 +23,11 @@ public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent Date: Tue, 23 Jan 2024 16:22:31 -0800 Subject: [PATCH 007/130] More reverting --- .../clients/consumer/internals/ConsumerUtils.java | 14 ++++++++++++++ .../internals/events/ErrorBackgroundEvent.java | 6 +++--- .../events/LeaveOnCloseApplicationEvent.java | 4 ++-- .../events/ListOffsetsApplicationEvent.java | 14 ++++++++++++++ .../internals/events/PollApplicationEvent.java | 7 +++++-- .../events/TopicMetadataApplicationEvent.java | 12 +++++++----- 6 files changed, 45 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 7c68fc8d07535..15458d1bf20d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -252,4 +252,18 @@ public static KafkaException maybeWrapAsKafkaException(Throwable t, String messa else return new KafkaException(message, t); } + + /** + * @return Expiration time in milliseconds calculated with the current time plus the given + * timeout. Returns Long.MAX_VALUE if the expiration overflows it. + * Visible for testing. + */ + public static long getExpirationTimeForTimeout(final long timeoutMs) { + // TODO: should use Time.milliseconds(), not System.currentTimeMillis() + long expiration = System.currentTimeMillis() + timeoutMs; + if (expiration < 0) { + return Long.MAX_VALUE; + } + return expiration; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 1b0d3581728be..587532dfc292a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -55,8 +55,8 @@ public int hashCode() { @Override public String toString() { return "ErrorBackgroundEvent{" + - toStringBase() + - ", error=" + error + - '}'; + toStringBase() + + ", error=" + error + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index e03cd7a963855..15910c609bdd9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -27,7 +27,7 @@ public LeaveOnCloseApplicationEvent(Timer timer) { @Override public String toString() { return "LeaveOnCloseApplicationEvent{" + - toStringBase() + - '}'; + toStringBase() + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index ab5a73a2bf89c..e8daeb7d1497a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.utils.Timer; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -44,6 +45,19 @@ public ListOffsetsApplicationEvent(Map timestampToSearch, this.requireTimestamps = requireTimestamps; } + /** + * Build result representing that no offsets were found as part of the current event. + * + * @return Map containing all the partitions the event was trying to get offsets for, and + * null {@link OffsetAndTimestamp} as value + */ + public Map emptyResult() { + HashMap offsetsByTimes = new HashMap<>(timestampsToSearch.size()); + for (Map.Entry entry : timestampsToSearch.entrySet()) + offsetsByTimes.put(entry.getKey(), null); + return offsetsByTimes; + } + public Map timestampsToSearch() { return timestampsToSearch; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java index 4e41da92cc2c5..b958f0ec41703 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java @@ -48,7 +48,10 @@ public int hashCode() { } @Override - public String toStringBase() { - return toStringBase() + ", pollTimeMs=" + pollTimeMs; + public String toString() { + return "PollApplicationEvent{" + + toStringBase() + + ", pollTimeMs=" + pollTimeMs + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index e569d5fc79c57..b22a5b8424f75 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -24,7 +24,6 @@ import java.util.Objects; public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { - private final String topic; private final boolean allTopics; @@ -48,6 +47,13 @@ public boolean isAllTopics() { return allTopics; } + @Override + public String toString() { + return getClass().getSimpleName() + " {" + toStringBase() + + ", topic=" + topic + + ", allTopics=" + allTopics + "}"; + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), topic, allTopics); @@ -65,8 +71,4 @@ public boolean equals(Object o) { return topic.equals(that.topic) && (allTopics == that.allTopics); } - @Override - public String toStringBase() { - return toStringBase() + ", topic=" + topic + ", allTopics=" + allTopics + "}"; - } } From 3b5bc623826ff5412715f70a611b323e1f3b6deb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:25:58 -0800 Subject: [PATCH 008/130] Reverts --- .../clients/consumer/internals/ConsumerUtils.java | 11 ----------- .../events/CompletableApplicationEvent.java | 14 ++++++++++++++ .../internals/events/GroupMetadataUpdateEvent.java | 4 ++-- .../events/LeaveOnCloseApplicationEvent.java | 1 - .../events/TopicMetadataApplicationEvent.java | 9 ++++----- 5 files changed, 20 insertions(+), 19 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 15458d1bf20d1..d190348ef2484 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -50,7 +50,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -207,16 +206,6 @@ public static void refreshCommittedOffsets(final Map void chain(final CompletableFuture a, final CompletableFuture b) { - a.whenComplete((value, exception) -> { - if (exception != null) { - b.completeExceptionally(exception); - } else { - b.complete(value); - } - }); - } - public static T getResult(Future future, Timer timer) { try { return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 37f4b308dd25b..95c089ec507fa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; import java.util.Objects; @@ -43,6 +44,19 @@ public CompletableFuture future() { return future; } + public T get(Timer timer) { + return ConsumerUtils.getResult(future, timer); + } + + public void chain(final CompletableFuture providedFuture) { + providedFuture.whenComplete((value, exception) -> { + if (exception != null) { + this.future.completeExceptionally(exception); + } else { + this.future.complete(value); + } + }); + } @Override public long deadlineMs() { return deadlineMs; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 21ea7a8cf931d..73dd6c5145d58 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -72,7 +72,7 @@ public String toStringBase() { @Override public String toString() { return "GroupMetadataUpdateEvent{" + - toStringBase() + - '}'; + toStringBase() + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index 15910c609bdd9..ab91166e4c539 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.Timer; public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent(Timer timer) { super(Type.LEAVE_ON_CLOSE, timer); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index b22a5b8424f75..8f02bf159492e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -54,11 +54,6 @@ public String toString() { ", allTopics=" + allTopics + "}"; } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics); - } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -71,4 +66,8 @@ public boolean equals(Object o) { return topic.equals(that.topic) && (allTopics == that.allTopics); } + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), topic, allTopics); + } } From 46af018a76ca75119384c23a0948c367bbb81319 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:33:12 -0800 Subject: [PATCH 009/130] Reverting --- .../consumer/internals/RequestManagers.java | 9 ++---- .../events/ApplicationEventProcessor.java | 28 ++++++++++--------- .../events/GroupMetadataUpdateEvent.java | 1 + .../internals/ConsumerTestBuilder.java | 7 +++-- .../events/ApplicationEventProcessorTest.java | 22 ++++++++++----- 5 files changed, 38 insertions(+), 29 deletions(-) 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 1cbb53704159d..4c43acf6230db 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 @@ -131,6 +131,7 @@ protected RequestManagers create() { fetchConfig.isolationLevel, time, retryBackoffMs, + requestTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, @@ -145,10 +146,7 @@ protected RequestManagers create() { networkClientDelegate, apiVersions, requestTimeoutMs); - final TopicMetadataRequestManager topic = new TopicMetadataRequestManager( - logContext, - config, - time); + final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(logContext, config); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; CoordinatorRequestManager coordinator = null; @@ -182,8 +180,7 @@ protected RequestManagers create() { metadata, logContext, clientTelemetryReporter, - backgroundEventHandler, - time); + backgroundEventHandler); membershipManager.registerStateListener(commit); heartbeatRequestManager = new HeartbeatRequestManager( 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 7dd836e1a0833..0a9fa73990d85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.MembershipManager; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; @@ -157,7 +156,7 @@ private void process(final CommitApplicationEvent event) { Timer timer = timer(event); CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture future = manager.addOffsetCommitRequest(event.offsets(), timer, false); - chain(event, future); + event.chain(future); } private void process(final FetchCommittedOffsetsApplicationEvent event) { @@ -173,7 +172,7 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { event.partitions(), timer ); - chain(event, future); + event.chain(future); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { @@ -201,7 +200,7 @@ private void process(final ListOffsetsApplicationEvent event) { event.requireTimestamps(), timer ); - chain(event, future); + event.chain(future); } /** @@ -235,19 +234,19 @@ private void process(final UnsubscribeApplicationEvent event) { MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); Timer timer = timer(event); CompletableFuture result = membershipManager.leaveGroup(timer); - chain(event, result); + event.chain(result); } private void process(final ResetPositionsApplicationEvent event) { Timer timer = timer(event); CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(timer); - chain(event, result); + event.chain(result); } private void process(final ValidatePositionsApplicationEvent event) { Timer timer = timer(event); CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(timer); - chain(event, result); + event.chain(result); } private void process(final TopicMetadataApplicationEvent event) { @@ -260,7 +259,7 @@ private void process(final TopicMetadataApplicationEvent event) { future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), timer); } - chain(event, future); + event.chain(future); } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { @@ -294,13 +293,16 @@ private void process(final LeaveOnCloseApplicationEvent event) { final Timer timer = timer(event); CompletableFuture future = membershipManager.leaveGroup(timer); // The future will be completed on heartbeat sent - chain(event, future); - } - - private void chain(CompletableApplicationEvent event, CompletableFuture future) { - ConsumerUtils.chain(future, event.future()); + event.chain(future); } + /** + * Creates a {@link Timer time} for the network I/O thread that is separate from the timer for the + * application thread. + * + * @param event + * @return + */ private Timer timer(CompletableEvent event) { return time.timer(event.deadlineMs() - time.milliseconds()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 73dd6c5145d58..120e671724209 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -75,4 +75,5 @@ public String toString() { toStringBase() + '}'; } + } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index c4888c030bcf4..5bbeea74b7aaf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -165,12 +165,13 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA fetchConfig.isolationLevel, time, retryBackoffMs, + requestTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, logContext)); - this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config, time)); + this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config)); if (groupInfo.isPresent()) { GroupInformation gi = groupInfo.get(); @@ -202,8 +203,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA metadata, logContext, Optional.empty(), - backgroundEventHandler, - time + backgroundEventHandler ) ); HeartbeatRequestManager.HeartbeatState heartbeatState = spy(new HeartbeatRequestManager.HeartbeatState( @@ -262,6 +262,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA heartbeatRequestManager); this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, + time, applicationEventQueue, requestManagers, metadata diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 4d6c72e1f4b24..cad3f06d5b00b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.CoordinatorRequestManager; import org.apache.kafka.clients.consumer.internals.FetchRequestManager; import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager; @@ -27,6 +28,9 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -44,6 +48,7 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { + private Time time; private ApplicationEventProcessor processor; private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); private RequestManagers requestManagers; @@ -61,6 +66,7 @@ public class ApplicationEventProcessorTest { @BeforeEach @SuppressWarnings("unchecked") public void setup() { + time = new MockTime(); LogContext logContext = new LogContext(); offsetRequestManager = mock(OffsetsRequestManager.class); offsetsRequestManager = mock(OffsetsRequestManager.class); @@ -79,7 +85,8 @@ public void setup() { Optional.of(commitRequestManager), Optional.of(heartbeatRequestManager)); processor = new ApplicationEventProcessor( - new LogContext(), + logContext, + time, applicationEventQueue, requestManagers, metadata @@ -96,20 +103,21 @@ public void testPrepClosingCommitEvents() { @Test public void testExpirationCalculation() { - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE)); - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); - long timeout = processor.getExpirationTimeForTimeout(1000); + assertEquals(Long.MAX_VALUE, ConsumerUtils.getExpirationTimeForTimeout(Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, ConsumerUtils.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); + long timeout = ConsumerUtils.getExpirationTimeForTimeout(1000); assertTrue(timeout > 0); assertTrue(timeout < Long.MAX_VALUE); } @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(Long.MAX_VALUE); + Timer timer = time.timer(Long.MAX_VALUE); + LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); - when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); + when(membershipManager.leaveGroup(timer)).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); - verify(membershipManager).leaveGroup(); + verify(membershipManager).leaveGroup(timer); assertTrue(event.future().isDone()); } From 4998699007f8e15c2f423f2bcf82042f62d92296 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:44:57 -0800 Subject: [PATCH 010/130] Reverts --- .../internals/AsyncKafkaConsumer.java | 296 +++++++++--------- .../consumer/internals/ConsumerUtils.java | 1 + .../events/ApplicationEventHandler.java | 7 +- .../events/CompletableApplicationEvent.java | 15 +- 4 files changed, 157 insertions(+), 162 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 849b00e45dace..8463b81f87c78 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -42,7 +42,6 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CommitAsyncApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; @@ -96,7 +95,6 @@ import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -233,20 +231,20 @@ private void process(final GroupMetadataUpdateEvent event) { if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - event.memberEpoch(), - event.memberId(), - currentGroupMetadata.groupInstanceId() + currentGroupMetadata.groupId(), + event.memberEpoch(), + event.memberId(), + currentGroupMetadata.groupInstanceId() )); } } private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ApplicationEvent invokedEvent = invokeRebalanceCallbacks( - rebalanceListenerInvoker, - event.methodName(), - event.partitions(), - event.future() + rebalanceListenerInvoker, + event.methodName(), + event.partitions(), + event.future() ); applicationEventHandler.add(invokedEvent); } @@ -297,14 +295,14 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final Deserializer keyDeserializer, final Deserializer valueDeserializer) { this( - config, - keyDeserializer, - valueDeserializer, - Time.SYSTEM, - ApplicationEventHandler::new, - FetchCollector::new, - ConsumerMetadata::new, - new LinkedBlockingQueue<>() + config, + keyDeserializer, + valueDeserializer, + Time.SYSTEM, + ApplicationEventHandler::new, + FetchCollector::new, + ConsumerMetadata::new, + new LinkedBlockingQueue<>() ); } @@ -319,8 +317,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final LinkedBlockingQueue backgroundEventQueue) { try { GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); @@ -382,6 +380,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { clientTelemetryReporter, metrics); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, + time, metadata, applicationEventQueue, requestManagersSupplier); @@ -424,7 +423,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); if (groupMetadata.isPresent() && - GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) { + GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) { config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread } config.logUnused(); @@ -524,8 +523,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); @@ -533,38 +532,39 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( - logContext, - backgroundEventQueue + logContext, + backgroundEventQueue ); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - new RebalanceCallbackMetrics(metrics) + logContext, + subscriptions, + time, + new RebalanceCallbackMetrics(metrics) ); ApiVersions apiVersions = new ApiVersions(); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( - time, - config, - logContext, - client + time, + config, + logContext, + client ); Supplier requestManagersSupplier = RequestManagers.supplier( - time, - logContext, - backgroundEventHandler, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier, - clientTelemetryReporter, - metrics); + time, + logContext, + backgroundEventHandler, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier, + clientTelemetryReporter, + metrics); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, + time, metadata, applicationEventQueue, requestManagersSupplier @@ -587,12 +587,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { interface ApplicationEventHandlerFactory { ApplicationEventHandler build( - final LogContext logContext, - final Time time, - final BlockingQueue applicationEventQueue, - final Supplier applicationEventProcessorSupplier, - final Supplier networkClientDelegateSupplier, - final Supplier requestManagersSupplier + final LogContext logContext, + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier ); } @@ -601,13 +601,13 @@ ApplicationEventHandler build( interface FetchCollectorFactory { FetchCollector build( - final LogContext logContext, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions, - final FetchConfig fetchConfig, - final Deserializers deserializers, - final FetchMetricsManager metricsManager, - final Time time + final LogContext logContext, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final FetchConfig fetchConfig, + final Deserializers deserializers, + final FetchMetricsManager metricsManager, + final Time time ); } @@ -616,10 +616,10 @@ FetchCollector build( interface ConsumerMetadataFactory { ConsumerMetadata build( - final ConsumerConfig config, - final SubscriptionState subscriptions, - final LogContext logContext, - final ClusterResourceListeners clusterResourceListeners + final ConsumerConfig config, + final SubscriptionState subscriptions, + final LogContext logContext, + final ClusterResourceListeners clusterResourceListeners ); } @@ -627,8 +627,8 @@ ConsumerMetadata build( private Optional initializeGroupMetadata(final ConsumerConfig config, final GroupRebalanceConfig groupRebalanceConfig) { final Optional groupMetadata = initializeGroupMetadata( - groupRebalanceConfig.groupId, - groupRebalanceConfig.groupInstanceId + groupRebalanceConfig.groupId, + groupRebalanceConfig.groupInstanceId ); if (!groupMetadata.isPresent()) { config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); @@ -642,13 +642,13 @@ private Optional initializeGroupMetadata(final String gro if (groupId != null) { if (groupId.isEmpty()) { throw new InvalidGroupIdException("The configured " + ConsumerConfig.GROUP_ID_CONFIG - + " should not be an empty string or whitespace."); + + " should not be an empty string or whitespace."); } else { return Optional.of(new ConsumerGroupMetadata( - groupId, - JoinGroupRequest.UNKNOWN_GENERATION_ID, - JoinGroupRequest.UNKNOWN_MEMBER_ID, - groupInstanceId + groupId, + JoinGroupRequest.UNKNOWN_GENERATION_ID, + JoinGroupRequest.UNKNOWN_MEMBER_ID, + groupInstanceId )); } } @@ -751,19 +751,9 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - precommitCheck(offsets); - CompletableFuture future; - - if (offsets.isEmpty()) { - future = CompletableFuture.completedFuture(null); - } else { - final CommitAsyncApplicationEvent commitEvent = new CommitAsyncApplicationEvent(offsets); - applicationEventHandler.add(commitEvent); - future = commitEvent.future(); - } - // Commit without timer to indicate that the commit should be triggered without // waiting for a response. + CompletableFuture future = commit(offsets, false, Optional.empty()); future.whenComplete((r, t) -> { if (callback == null) { if (t != null) { @@ -779,13 +769,30 @@ public void commitAsync(Map offsets, OffsetCo } } - void precommitCheck(final Map offsets) { + // Visible for testing + CompletableFuture commit(final Map offsets, + final boolean isWakeupable, + final Optional retryTimeoutMs) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); maybeThrowInvalidGroupIdException(); log.debug("Committing offsets: {}", offsets); offsets.forEach(this::updateLastSeenEpochIfNewer); + + if (offsets.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + + final long timeout = retryTimeoutMs.orElse(Long.MAX_VALUE); + final Timer timer = time.timer(timeout); + final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, timer); + if (isWakeupable) { + // the task can only be woken up if the top level API call is commitSync + wakeupTrigger.setActiveTask(commitEvent.future()); + } + applicationEventHandler.add(commitEvent); + return commitEvent.future(); } @Override @@ -797,9 +804,9 @@ public void seek(TopicPartition partition, long offset) { try { log.info("Seeking to offset {} for partition {}", offset, partition); SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( - offset, - Optional.empty(), // This will ensure we skip validation - metadata.currentLeader(partition)); + offset, + Optional.empty(), // This will ensure we skip validation + metadata.currentLeader(partition)); subscriptions.seekUnvalidated(partition, newPosition); } finally { release(); @@ -817,15 +824,15 @@ public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) try { if (offsetAndMetadata.leaderEpoch().isPresent()) { log.info("Seeking to offset {} for partition {} with epoch {}", - offset, partition, offsetAndMetadata.leaderEpoch().get()); + offset, partition, offsetAndMetadata.leaderEpoch().get()); } else { log.info("Seeking to offset {} for partition {}", offset, partition); } Metadata.LeaderAndEpoch currentLeaderAndEpoch = metadata.currentLeader(partition); SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( - offsetAndMetadata.offset(), - offsetAndMetadata.leaderEpoch(), - currentLeaderAndEpoch); + offsetAndMetadata.offset(), + offsetAndMetadata.leaderEpoch(), + currentLeaderAndEpoch); updateLastSeenEpochIfNewer(partition, offsetAndMetadata); subscriptions.seekUnvalidated(partition, newPosition); } finally { @@ -883,7 +890,7 @@ public long position(TopicPartition partition, Duration timeout) { } while (timer.notExpired()); throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the position " + - "for partition " + partition + " could be determined"); + "for partition " + partition + " could be determined"); } finally { release(); } @@ -917,21 +924,19 @@ public Map committed(final Set committedOffsets = event.get(); + final Map committedOffsets = applicationEventHandler.addAndGet(event); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the last " + - "committed offset for partitions " + partitions + " could be determined. Try tuning " + - ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG + " larger to relax the threshold."); + "committed offset for partitions " + partitions + " could be determined. Try tuning " + + ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG + " larger to relax the threshold."); } finally { wakeupTrigger.clearTask(); } @@ -944,7 +949,7 @@ public Map committed(final Set partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } + final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, time.timer(timeout)); + new TopicMetadataApplicationEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = @@ -1000,8 +1006,9 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } + final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(time.timer(timeout)); + new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); @@ -1066,27 +1073,22 @@ public Map offsetsForTimes(Map offsetsByTimes = new HashMap<>(timestampsToSearch.size()); - for (Map.Entry entry : timestampsToSearch.entrySet()) - offsetsByTimes.put(entry.getKey(), null); - return offsetsByTimes; - } - + final Timer timer = time.timer(timeout); final ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( timestampsToSearch, true, - time.timer(timeout) - ); + timer); + + // If timeout is set to zero return empty immediately; otherwise try to get the results + // and throw timeout exception if it cannot complete in time. + if (timeout.toMillis() == 0L) + return listOffsetsEvent.emptyResult(); return applicationEventHandler.addAndGet(listOffsetsEvent); } finally { @@ -1127,21 +1129,19 @@ private Map beginningOrEndOffset(Collection timestampToSearch = partitions - .stream() - .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + .stream() + .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); Timer timer = time.timer(timeout); ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( - timestampToSearch, - false, - timer - ); + timestampToSearch, + false, + timer); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent - ); + listOffsetsEvent); return offsetAndTimestampMap - .entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + .entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); } finally { release(); } @@ -1160,7 +1160,7 @@ public OptionalLong currentLag(TopicPartition topicPartition) { // since we would not try to poll the network client synchronously if (lag == null) { if (subscriptions.partitionEndOffset(topicPartition, isolationLevel) == null && - !subscriptions.partitionEndOffsetRequested(topicPartition)) { + !subscriptions.partitionEndOffsetRequested(topicPartition)) { log.info("Requesting the log end offset for {} in order to compute lag", topicPartition); subscriptions.requestPartitionEndOffset(topicPartition); endOffsets(Collections.singleton(topicPartition), Duration.ofMillis(0)); @@ -1261,13 +1261,13 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx maybeAutoCommitSync(autoCommitEnabled, timer, firstException); applicationEventHandler.add(new CommitOnCloseApplicationEvent()); completeQuietly( - () -> { - maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); - }, - "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); + () -> { + maybeRevokePartitions(); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); + }, + "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", this::maybeInvokeCommitCallbacks, - firstException); + firstException); } // Visible for testing @@ -1342,16 +1342,11 @@ public void commitSync(Map offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - precommitCheck(offsets); - - if (!offsets.isEmpty()) { - Timer timer = time.timer(timeout); - final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, timer); - - // the task can only be woken up if the top level API call is commitSync - wakeupTrigger.setActiveTask(commitEvent.future()); - applicationEventHandler.addAndGet(commitEvent); - } + Timer requestTimer = time.timer(timeout.toMillis()); + // Commit with a timer to control how long the request should be retried until it + // gets a successful response or non-retriable error. + CompletableFuture commitFuture = commit(offsets, true, Optional.of(timeout.toMillis())); + ConsumerUtils.getResult(commitFuture, requestTimer); } finally { wakeupTrigger.clearTask(); kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart); @@ -1483,7 +1478,7 @@ public void unsubscribe() { @Deprecated public ConsumerRecords poll(final long timeoutMs) { throw new UnsupportedOperationException("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is deprecated and will be removed in the next major release."); + "This method is deprecated and will be removed in the next major release."); } // Visible for testing @@ -1616,10 +1611,9 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); try { final FetchCommittedOffsetsApplicationEvent event = - new FetchCommittedOffsetsApplicationEvent( - initializingPartitions, - timer - ); + new FetchCommittedOffsetsApplicationEvent( + initializingPartitions, + timer); final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; @@ -1703,8 +1697,8 @@ private void acquire() { final long threadId = thread.getId(); if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId)) throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access. " + - "currentThread(name: " + thread.getName() + ", id: " + threadId + ")" + - " otherThread(id: " + currentThread.get() + ")" + "currentThread(name: " + thread.getName() + ", id: " + threadId + ")" + + " otherThread(id: " + currentThread.get() + ")" ); refCount.incrementAndGet(); } @@ -1723,7 +1717,7 @@ private void subscribeInternal(Pattern pattern, Optional T getResult(Future future, Timer timer) { try { + timer.update(); return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); } catch (ExecutionException e) { throw maybeWrapAsKafkaException(e.getCause()); 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 e90c453010e40..3f3ec41b671da 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 @@ -103,8 +103,7 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#future()}, {@link CompletableApplicationEvent#deadlineMs()}, and - * {@link Future#get(long, TimeUnit)} for more details. + * See {@link CompletableApplicationEvent#get()} and {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread * @return Value that is the result of the event @@ -113,9 +112,7 @@ public long maximumTimeToWait() { public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); add(event); - CompletableFuture future = event.future(); - Timer timer = time.timer(event.deadlineMs() - time.milliseconds()); - return ConsumerUtils.getResult(future, timer); + return event.get(); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 95c089ec507fa..f76485da28a49 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -31,11 +31,13 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; + private final Timer timer; private final long deadlineMs; protected CompletableApplicationEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); + this.timer = timer; this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } @@ -44,7 +46,12 @@ public CompletableFuture future() { return future; } - public T get(Timer timer) { + @Override + public long deadlineMs() { + return deadlineMs; + } + + public T get() { return ConsumerUtils.getResult(future, timer); } @@ -57,10 +64,6 @@ public void chain(final CompletableFuture providedFuture) { } }); } - @Override - public long deadlineMs() { - return deadlineMs; - } @Override public boolean equals(Object o) { @@ -80,7 +83,7 @@ public int hashCode() { @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } @Override From 1366f08774815d2e4c29c6ebfe8cba282ddb2bb6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:46:04 -0800 Subject: [PATCH 011/130] Indent SNAFU --- .../internals/AsyncKafkaConsumer.java | 380 +++++++++--------- 1 file changed, 190 insertions(+), 190 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 8463b81f87c78..cb4e988a95471 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 @@ -231,20 +231,20 @@ private void process(final GroupMetadataUpdateEvent event) { if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - event.memberEpoch(), - event.memberId(), - currentGroupMetadata.groupInstanceId() + currentGroupMetadata.groupId(), + event.memberEpoch(), + event.memberId(), + currentGroupMetadata.groupInstanceId() )); } } private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ApplicationEvent invokedEvent = invokeRebalanceCallbacks( - rebalanceListenerInvoker, - event.methodName(), - event.partitions(), - event.future() + rebalanceListenerInvoker, + event.methodName(), + event.partitions(), + event.future() ); applicationEventHandler.add(invokedEvent); } @@ -295,14 +295,14 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final Deserializer keyDeserializer, final Deserializer valueDeserializer) { this( - config, - keyDeserializer, - valueDeserializer, - Time.SYSTEM, - ApplicationEventHandler::new, - FetchCollector::new, - ConsumerMetadata::new, - new LinkedBlockingQueue<>() + config, + keyDeserializer, + valueDeserializer, + Time.SYSTEM, + ApplicationEventHandler::new, + FetchCollector::new, + ConsumerMetadata::new, + new LinkedBlockingQueue<>() ); } @@ -317,8 +317,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final LinkedBlockingQueue backgroundEventQueue) { try { GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); @@ -339,8 +339,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer); this.subscriptions = createSubscriptionState(config, logContext); ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(), - interceptorList, - Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); + interceptorList, + Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners); final List addresses = ClientUtils.parseAndValidateAddresses(config); metadata.bootstrap(addresses); @@ -352,78 +352,78 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ApiVersions apiVersions = new ApiVersions(); final BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( - logContext, - backgroundEventQueue + logContext, + backgroundEventQueue ); // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier(time, - logContext, - metadata, - config, - apiVersions, - metrics, - fetchMetricsManager, - clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); + logContext, + metadata, + config, + apiVersions, + metrics, + fetchMetricsManager, + clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); final Supplier requestManagersSupplier = RequestManagers.supplier(time, - logContext, - backgroundEventHandler, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier, - clientTelemetryReporter, - metrics); + logContext, + backgroundEventHandler, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier, + clientTelemetryReporter, + metrics); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, - time, - metadata, - applicationEventQueue, - requestManagersSupplier); + time, + metadata, + applicationEventQueue, + requestManagersSupplier); this.applicationEventHandler = applicationEventHandlerFactory.build( - logContext, - time, - applicationEventQueue, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); + logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - new RebalanceCallbackMetrics(metrics) + logContext, + subscriptions, + time, + new RebalanceCallbackMetrics(metrics) ); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( - config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) + config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) ); this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, - metadata, - subscriptions, - fetchConfig, - deserializers, - fetchMetricsManager, - time); + metadata, + subscriptions, + fetchConfig, + deserializers, + fetchMetricsManager, + time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); if (groupMetadata.isPresent() && - GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) { + GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) { config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread } config.logUnused(); @@ -468,10 +468,10 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.interceptors = Objects.requireNonNull(interceptors); this.time = time; this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); this.metrics = metrics; this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty()); @@ -514,17 +514,17 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX); FetchMetricsManager fetchMetricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics); this.fetchCollector = new FetchCollector<>(logContext, - metadata, - subscriptions, - new FetchConfig(config), - deserializers, - fetchMetricsManager, - time); + metadata, + subscriptions, + new FetchConfig(config), + deserializers, + fetchMetricsManager, + time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); @@ -532,54 +532,54 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( - logContext, - backgroundEventQueue + logContext, + backgroundEventQueue ); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - new RebalanceCallbackMetrics(metrics) + logContext, + subscriptions, + time, + new RebalanceCallbackMetrics(metrics) ); ApiVersions apiVersions = new ApiVersions(); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( - time, - config, - logContext, - client + time, + config, + logContext, + client ); Supplier requestManagersSupplier = RequestManagers.supplier( - time, - logContext, - backgroundEventHandler, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier, - clientTelemetryReporter, - metrics); + time, + logContext, + backgroundEventHandler, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier, + clientTelemetryReporter, + metrics); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( - logContext, - time, - metadata, - applicationEventQueue, - requestManagersSupplier + logContext, + time, + metadata, + applicationEventQueue, + requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, - time, - applicationEventQueue, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); } @@ -587,12 +587,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { interface ApplicationEventHandlerFactory { ApplicationEventHandler build( - final LogContext logContext, - final Time time, - final BlockingQueue applicationEventQueue, - final Supplier applicationEventProcessorSupplier, - final Supplier networkClientDelegateSupplier, - final Supplier requestManagersSupplier + final LogContext logContext, + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier ); } @@ -601,13 +601,13 @@ ApplicationEventHandler build( interface FetchCollectorFactory { FetchCollector build( - final LogContext logContext, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions, - final FetchConfig fetchConfig, - final Deserializers deserializers, - final FetchMetricsManager metricsManager, - final Time time + final LogContext logContext, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final FetchConfig fetchConfig, + final Deserializers deserializers, + final FetchMetricsManager metricsManager, + final Time time ); } @@ -616,10 +616,10 @@ FetchCollector build( interface ConsumerMetadataFactory { ConsumerMetadata build( - final ConsumerConfig config, - final SubscriptionState subscriptions, - final LogContext logContext, - final ClusterResourceListeners clusterResourceListeners + final ConsumerConfig config, + final SubscriptionState subscriptions, + final LogContext logContext, + final ClusterResourceListeners clusterResourceListeners ); } @@ -627,8 +627,8 @@ ConsumerMetadata build( private Optional initializeGroupMetadata(final ConsumerConfig config, final GroupRebalanceConfig groupRebalanceConfig) { final Optional groupMetadata = initializeGroupMetadata( - groupRebalanceConfig.groupId, - groupRebalanceConfig.groupInstanceId + groupRebalanceConfig.groupId, + groupRebalanceConfig.groupInstanceId ); if (!groupMetadata.isPresent()) { config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); @@ -642,13 +642,13 @@ private Optional initializeGroupMetadata(final String gro if (groupId != null) { if (groupId.isEmpty()) { throw new InvalidGroupIdException("The configured " + ConsumerConfig.GROUP_ID_CONFIG - + " should not be an empty string or whitespace."); + + " should not be an empty string or whitespace."); } else { return Optional.of(new ConsumerGroupMetadata( - groupId, - JoinGroupRequest.UNKNOWN_GENERATION_ID, - JoinGroupRequest.UNKNOWN_MEMBER_ID, - groupInstanceId + groupId, + JoinGroupRequest.UNKNOWN_GENERATION_ID, + JoinGroupRequest.UNKNOWN_MEMBER_ID, + groupInstanceId )); } } @@ -709,7 +709,7 @@ public ConsumerRecords poll(final Duration timeout) { if (!fetch.isEmpty()) { if (fetch.records().isEmpty()) { log.trace("Returning empty records from `poll()` " - + "since the consumer's position has advanced for at least one topic partition"); + + "since the consumer's position has advanced for at least one topic partition"); } return interceptors.onConsume(new ConsumerRecords<>(fetch.records())); @@ -804,9 +804,9 @@ public void seek(TopicPartition partition, long offset) { try { log.info("Seeking to offset {} for partition {}", offset, partition); SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( - offset, - Optional.empty(), // This will ensure we skip validation - metadata.currentLeader(partition)); + offset, + Optional.empty(), // This will ensure we skip validation + metadata.currentLeader(partition)); subscriptions.seekUnvalidated(partition, newPosition); } finally { release(); @@ -824,15 +824,15 @@ public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) try { if (offsetAndMetadata.leaderEpoch().isPresent()) { log.info("Seeking to offset {} for partition {} with epoch {}", - offset, partition, offsetAndMetadata.leaderEpoch().get()); + offset, partition, offsetAndMetadata.leaderEpoch().get()); } else { log.info("Seeking to offset {} for partition {}", offset, partition); } Metadata.LeaderAndEpoch currentLeaderAndEpoch = metadata.currentLeader(partition); SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( - offsetAndMetadata.offset(), - offsetAndMetadata.leaderEpoch(), - currentLeaderAndEpoch); + offsetAndMetadata.offset(), + offsetAndMetadata.leaderEpoch(), + currentLeaderAndEpoch); updateLastSeenEpochIfNewer(partition, offsetAndMetadata); subscriptions.seekUnvalidated(partition, newPosition); } finally { @@ -890,7 +890,7 @@ public long position(TopicPartition partition, Duration timeout) { } while (timer.notExpired()); throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the position " + - "for partition " + partition + " could be determined"); + "for partition " + partition + " could be determined"); } finally { release(); } @@ -926,8 +926,8 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event); @@ -935,8 +935,8 @@ public Map committed(final Set committed(final Set partitionsFor(String topic, Duration timeout) { final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timer); + new TopicMetadataApplicationEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1008,7 +1008,7 @@ public Map> listTopics(Duration timeout) { final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timer); + new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); @@ -1073,7 +1073,7 @@ public Map offsetsForTimes(Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions - .stream() - .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + .stream() + .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); Timer timer = time.timer(timeout); ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( - timestampToSearch, - false, - timer); + timestampToSearch, + false, + timer); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent); + listOffsetsEvent); return offsetAndTimestampMap - .entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + .entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); } finally { release(); } @@ -1160,7 +1160,7 @@ public OptionalLong currentLag(TopicPartition topicPartition) { // since we would not try to poll the network client synchronously if (lag == null) { if (subscriptions.partitionEndOffset(topicPartition, isolationLevel) == null && - !subscriptions.partitionEndOffsetRequested(topicPartition)) { + !subscriptions.partitionEndOffsetRequested(topicPartition)) { log.info("Requesting the log end offset for {} in order to compute lag", topicPartition); subscriptions.requestPartitionEndOffset(topicPartition); endOffsets(Collections.singleton(topicPartition), Duration.ofMillis(0)); @@ -1261,13 +1261,13 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx maybeAutoCommitSync(autoCommitEnabled, timer, firstException); applicationEventHandler.add(new CommitOnCloseApplicationEvent()); completeQuietly( - () -> { - maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); - }, - "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); + () -> { + maybeRevokePartitions(); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); + }, + "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", this::maybeInvokeCommitCallbacks, - firstException); + firstException); } // Visible for testing @@ -1444,8 +1444,8 @@ public void assign(Collection partitions) { */ private void updatePatternSubscription(Cluster cluster) { final Set topicsToSubscribe = cluster.topics().stream() - .filter(subscriptions::matchesSubscribedPattern) - .collect(Collectors.toSet()); + .filter(subscriptions::matchesSubscribedPattern) + .collect(Collectors.toSet()); if (subscriptions.subscribeFromPattern(topicsToSubscribe)) metadata.requestUpdateForNewTopics(); } @@ -1478,7 +1478,7 @@ public void unsubscribe() { @Deprecated public ConsumerRecords poll(final long timeoutMs) { throw new UnsupportedOperationException("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is deprecated and will be removed in the next major release."); + "This method is deprecated and will be removed in the next major release."); } // Visible for testing @@ -1488,8 +1488,8 @@ WakeupTrigger wakeupTrigger() { private Fetch pollForFetches(Timer timer) { long pollTimeout = isCommittedOffsetsManagementEnabled() - ? Math.min(applicationEventHandler.maximumTimeToWait(), timer.remainingMs()) - : timer.remainingMs(); + ? Math.min(applicationEventHandler.maximumTimeToWait(), timer.remainingMs()) + : timer.remainingMs(); // if data is available already, return it immediately final Fetch fetch = collectFetch(); @@ -1611,9 +1611,9 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); try { final FetchCommittedOffsetsApplicationEvent event = - new FetchCommittedOffsetsApplicationEvent( - initializingPartitions, - timer); + new FetchCommittedOffsetsApplicationEvent( + initializingPartitions, + timer); final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; @@ -1626,7 +1626,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { private void throwIfNoAssignorsConfigured() { if (assignors.isEmpty()) throw new IllegalStateException("Must configure at least one partition assigner class name to " + - ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); } private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) { @@ -1697,8 +1697,8 @@ private void acquire() { final long threadId = thread.getId(); if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId)) throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access. " + - "currentThread(name: " + thread.getName() + ", id: " + threadId + ")" + - " otherThread(id: " + currentThread.get() + ")" + "currentThread(name: " + thread.getName() + ", id: " + threadId + ")" + + " otherThread(id: " + currentThread.get() + ")" ); refCount.incrementAndGet(); } @@ -1717,7 +1717,7 @@ private void subscribeInternal(Pattern pattern, Optional Date: Tue, 23 Jan 2024 16:48:19 -0800 Subject: [PATCH 012/130] Indentation reverts --- .../internals/AsyncKafkaConsumer.java | 146 +++++++++--------- 1 file changed, 73 insertions(+), 73 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 cb4e988a95471..0240f416488e0 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 @@ -339,8 +339,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer); this.subscriptions = createSubscriptionState(config, logContext); ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(), - interceptorList, - Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); + interceptorList, + Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners); final List addresses = ClientUtils.parseAndValidateAddresses(config); metadata.bootstrap(addresses); @@ -352,73 +352,73 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ApiVersions apiVersions = new ApiVersions(); final BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( - logContext, - backgroundEventQueue + logContext, + backgroundEventQueue ); // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier(time, - logContext, - metadata, - config, - apiVersions, - metrics, - fetchMetricsManager, - clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); + logContext, + metadata, + config, + apiVersions, + metrics, + fetchMetricsManager, + clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); final Supplier requestManagersSupplier = RequestManagers.supplier(time, - logContext, - backgroundEventHandler, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier, - clientTelemetryReporter, - metrics); + logContext, + backgroundEventHandler, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier, + clientTelemetryReporter, + metrics); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, - time, - metadata, - applicationEventQueue, - requestManagersSupplier); + time, + metadata, + applicationEventQueue, + requestManagersSupplier); this.applicationEventHandler = applicationEventHandlerFactory.build( - logContext, - time, - applicationEventQueue, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); + logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - new RebalanceCallbackMetrics(metrics) + logContext, + subscriptions, + time, + new RebalanceCallbackMetrics(metrics) ); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( - config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) + config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) ); this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, - metadata, - subscriptions, - fetchConfig, - deserializers, - fetchMetricsManager, - time); + metadata, + subscriptions, + fetchConfig, + deserializers, + fetchMetricsManager, + time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); @@ -468,10 +468,10 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.interceptors = Objects.requireNonNull(interceptors); this.time = time; this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); this.metrics = metrics; this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty()); @@ -514,12 +514,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX); FetchMetricsManager fetchMetricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics); this.fetchCollector = new FetchCollector<>(logContext, - metadata, - subscriptions, - new FetchConfig(config), - deserializers, - fetchMetricsManager, - time); + metadata, + subscriptions, + new FetchConfig(config), + deserializers, + fetchMetricsManager, + time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( @@ -563,23 +563,23 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { clientTelemetryReporter, metrics); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( - logContext, - time, - metadata, - applicationEventQueue, - requestManagersSupplier + logContext, + time, + metadata, + applicationEventQueue, + requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, - time, - applicationEventQueue, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker ); } From 6030c8ec33e0ebca8ea23f2480708c6f1582ee7e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 23 Jan 2024 16:49:49 -0800 Subject: [PATCH 013/130] Update AsyncKafkaConsumer.java --- .../consumer/internals/AsyncKafkaConsumer.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 0240f416488e0..23f394f5aecc5 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 @@ -709,7 +709,7 @@ public ConsumerRecords poll(final Duration timeout) { if (!fetch.isEmpty()) { if (fetch.records().isEmpty()) { log.trace("Returning empty records from `poll()` " - + "since the consumer's position has advanced for at least one topic partition"); + + "since the consumer's position has advanced for at least one topic partition"); } return interceptors.onConsume(new ConsumerRecords<>(fetch.records())); @@ -978,11 +978,11 @@ public List partitionsFor(String topic, Duration timeout) { final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timer); + new TopicMetadataApplicationEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1444,8 +1444,8 @@ public void assign(Collection partitions) { */ private void updatePatternSubscription(Cluster cluster) { final Set topicsToSubscribe = cluster.topics().stream() - .filter(subscriptions::matchesSubscribedPattern) - .collect(Collectors.toSet()); + .filter(subscriptions::matchesSubscribedPattern) + .collect(Collectors.toSet()); if (subscriptions.subscribeFromPattern(topicsToSubscribe)) metadata.requestUpdateForNewTopics(); } @@ -1478,7 +1478,7 @@ public void unsubscribe() { @Deprecated public ConsumerRecords poll(final long timeoutMs) { throw new UnsupportedOperationException("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is deprecated and will be removed in the next major release."); + "This method is deprecated and will be removed in the next major release."); } // Visible for testing @@ -1488,8 +1488,8 @@ WakeupTrigger wakeupTrigger() { private Fetch pollForFetches(Timer timer) { long pollTimeout = isCommittedOffsetsManagementEnabled() - ? Math.min(applicationEventHandler.maximumTimeToWait(), timer.remainingMs()) - : timer.remainingMs(); + ? Math.min(applicationEventHandler.maximumTimeToWait(), timer.remainingMs()) + : timer.remainingMs(); // if data is available already, return it immediately final Fetch fetch = collectFetch(); From 72497564a2d13fbdd1424c4bde9d0240882954f9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 13:14:55 -0800 Subject: [PATCH 014/130] Everything compiles (for now) --- .../internals/AsyncKafkaConsumer.java | 15 +- .../internals/CommitRequestManager.java | 59 ++-- .../internals/CoordinatorRequestManager.java | 10 +- .../internals/FetchRequestManager.java | 2 +- .../internals/MembershipManagerImpl.java | 25 +- .../internals/NetworkClientDelegate.java | 7 +- .../consumer/internals/RequestManagers.java | 3 +- .../events/ApplicationEventHandler.java | 5 - .../events/ApplicationEventProcessor.java | 65 +++- .../events/CommitApplicationEvent.java | 27 +- .../events/CompletableBackgroundEvent.java | 18 +- ...rRebalanceListenerCallbackNeededEvent.java | 6 +- .../internals/AsyncKafkaConsumerTest.java | 64 ++-- .../internals/CommitRequestManagerTest.java | 330 ++++++++++-------- .../internals/ConsumerNetworkThreadTest.java | 38 +- .../internals/ConsumerTestBuilder.java | 5 +- .../CoordinatorRequestManagerTest.java | 67 ++-- .../internals/FetchRequestManagerTest.java | 12 +- .../internals/MembershipManagerImplTest.java | 40 ++- .../internals/OffsetsRequestManagerTest.java | 150 +++++--- .../TopicMetadataRequestManagerTest.java | 105 +++--- 21 files changed, 615 insertions(+), 438 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 23f394f5aecc5..3c5a138765812 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 @@ -784,9 +784,18 @@ CompletableFuture commit(final Map offs return CompletableFuture.completedFuture(null); } - final long timeout = retryTimeoutMs.orElse(Long.MAX_VALUE); - final Timer timer = time.timer(timeout); - final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, timer); + final Timer timer; + final boolean allowsRetries; + + if (retryTimeoutMs.isPresent()) { + timer = time.timer(retryTimeoutMs.get()); + allowsRetries = true; + } else { + timer = time.timer(defaultApiTimeoutMs); + allowsRetries = false; + } + + final CommitApplicationEvent commitEvent = new CommitApplicationEvent(offsets, timer, allowsRetries); if (isWakeupable) { // the task can only be woken up if the top level API call is commitSync wakeupTrigger.setActiveTask(commitEvent.future()); 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 99b246508d181..6dd9423f0d34b 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 @@ -85,6 +85,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener private final long retryBackoffMaxMs; // For testing only private final OptionalDouble jitter; + private final int defaultApiTimeoutMs; private final boolean throwOnFetchStableOffsetUnsupported; final PendingRequests pendingRequests; private boolean closing = false; @@ -117,6 +118,7 @@ public CommitRequestManager( config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG), OptionalDouble.empty(), + config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG), CONSUMER_METRIC_GROUP_PREFIX, metrics); } @@ -133,6 +135,7 @@ public CommitRequestManager( final long retryBackoffMs, final long retryBackoffMaxMs, final OptionalDouble jitter, + final int defaultApiTimeoutMs, final String metricGroupPrefix, final Metrics metrics) { this.time = time; @@ -154,6 +157,7 @@ public CommitRequestManager( this.retryBackoffMs = retryBackoffMs; this.retryBackoffMaxMs = retryBackoffMaxMs; this.jitter = jitter; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.throwOnFetchStableOffsetUnsupported = config.getBoolean(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED); this.memberInfo = new MemberInfo(); this.commitSensor = addCommitSensor(metrics, metricGroupPrefix); @@ -229,7 +233,7 @@ private static long findMinTime(final Collection request * completed future if no request is generated. */ private CompletableFuture maybeAutoCommit(final Map offsets, - final Timer timer, + final Optional timer, boolean checkInterval, boolean retryOnStaleEpoch) { if (!autoCommitEnabled()) { @@ -264,9 +268,8 @@ public CompletableFuture maybeAutoCommitAllConsumedAsync() { // Early return to ensure that no action/logging is performed. return CompletableFuture.completedFuture(null); } - Timer timer = time.timer(requestTimeoutMs); Map offsets = subscriptions.allConsumed(); - CompletableFuture result = maybeAutoCommit(offsets, timer, true, true); + CompletableFuture result = maybeAutoCommit(offsets, Optional.empty(), true, true); result.whenComplete((__, error) -> { if (error != null) { if (error instanceof RetriableCommitFailedException) { @@ -288,9 +291,8 @@ public CompletableFuture maybeAutoCommitAllConsumedAsync() { * until the request succeeds or fails with a fatal error. */ public CompletableFuture maybeAutoCommitAllConsumedNow( - final long timeoutMs, + final Optional timer, final boolean retryOnStaleEpoch) { - Timer timer = time.timer(timeoutMs); return maybeAutoCommit(subscriptions.allConsumed(), timer, false, retryOnStaleEpoch); } @@ -313,22 +315,7 @@ public CompletableFuture maybeAutoCommitAllConsumedNow( * {@link OffsetCommitRequestState} and enqueue it to send later. */ public CompletableFuture addOffsetCommitRequest(final Map offsets, - final Optional deadlineMs, - final boolean retryOnStaleEpoch) { - - if (offsets.isEmpty()) { - log.debug("Skipping commit of empty offsets"); - return CompletableFuture.completedFuture(null); - } - return pendingRequests.addOffsetCommitRequest(offsets, timer, retryOnStaleEpoch).future; - } - - /** - * Handles {@link CommitApplicationEvent}. It creates an - * {@link OffsetCommitRequestState} and enqueue it to send later. - */ - public CompletableFuture addOffsetCommitRequest(final Map offsets, - final Timer timer, + final Optional timer, final boolean retryOnStaleEpoch) { if (offsets.isEmpty()) { log.debug("Skipping commit of empty offsets"); @@ -437,12 +424,12 @@ private class OffsetCommitRequestState extends RetriableRequestState { private final CompletableFuture future; - private final Timer timer; + private final Optional timer; OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Timer timer, + final Optional timer, final long retryBackoffMs, final long retryBackoffMaxMs, final MemberInfo memberInfo, @@ -460,7 +447,7 @@ private class OffsetCommitRequestState extends RetriableRequestState { OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Timer timer, + final Optional timer, final long retryBackoffMs, final long retryBackoffMaxMs, final double jitter, @@ -509,10 +496,12 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data); + // Timer will be "empty" for async/auto-commit, so create a dummy timer using default.api.timeout.ms. + Timer requestTimer = timer.orElseGet(() -> time.timer(defaultApiTimeoutMs)); NetworkClientDelegate.UnsentRequest resp = new NetworkClientDelegate.UnsentRequest( builder, coordinatorRequestManager.coordinator(), - timer + requestTimer ); resp.whenComplete( (response, throwable) -> { @@ -615,7 +604,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { future.completeExceptionally(commitExceptionForRetriableError(throwable)); return; } - if (timer.isExpired()) { + if (timer.isPresent() && timer.get().isExpired()) { // Fail requests that allowed retries (sync requests), but expired. future.completeExceptionally(throwable); return; @@ -627,6 +616,10 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { pendingRequests.addOffsetCommitRequest(this); } + private boolean isExpired() { + return timer.isPresent() && timer.get().isExpired(); + } + /** * @return True if the requests allows to be retried (sync requests that provide an * expiration time to bound the retries). False if the request does not allow to be @@ -634,7 +627,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { * for retries) */ private boolean allowsRetries() { - return expirationTimeMs.isPresent(); + return timer.isPresent(); } /** @@ -644,7 +637,7 @@ private boolean allowsRetries() { * @return True if the request expired. */ private boolean maybeExpire() { - if (timer.isExpired()) { + if (isExpired()) { future.completeExceptionally(new TimeoutException("OffsetCommit could not complete " + "before timeout expired.")); return true; @@ -845,7 +838,7 @@ private void onFailure(final long currentTimeMs, */ @Override void maybeRetry(long currentTimeMs, Throwable throwable) { - if (isExpired(currentTimeMs)) { + if (timer.isExpired()) { future.completeExceptionally(throwable); return; } @@ -970,7 +963,7 @@ boolean hasUnsentRequests() { OffsetCommitRequestState addOffsetCommitRequest( final Map offsets, - final Timer timer, + final Optional timer, final boolean retryOnStaleEpoch) { // TODO: Dedupe committing the same offsets to the same partitions OffsetCommitRequestState requestState = createOffsetCommitRequest( @@ -989,7 +982,7 @@ OffsetCommitRequestState addOffsetCommitRequest(OffsetCommitRequestState request OffsetCommitRequestState createOffsetCommitRequest(final Map offsets, final OptionalDouble jitter, - final Timer timer, + final Optional timer, final boolean retryOnStaleEpoch) { return jitter.isPresent() ? new OffsetCommitRequestState( @@ -1075,7 +1068,7 @@ List drain(final long currentTimeMs) { .filter(request -> !request.canSendRequest(currentTimeMs)) .collect(Collectors.toList()); - failAndRemoveExpiredCommitRequests(currentTimeMs); + failAndRemoveExpiredCommitRequests(); // Add all unsent offset commit requests to the unsentRequests list unsentRequests.addAll( @@ -1090,7 +1083,7 @@ List drain(final long currentTimeMs) { unsentOffsetFetches.stream() .collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs))); - failAndRemoveExpiredFetchRequests(currentTimeMs); + failAndRemoveExpiredFetchRequests(); // Add all sendable offset fetch requests to the unsentRequests list and to the inflightOffsetFetches list for (OffsetFetchRequestState request : partitionedBySendability.get(true)) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index 447ddd1ba91d5..6b0c335b6b28a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -53,7 +53,7 @@ public class CoordinatorRequestManager implements RequestManager { private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000; private final Time time; - private final long requestTimeoutMs; + private final long defaultApiTimeoutMs; private final Logger log; private final BackgroundEventHandler backgroundEventHandler; private final String groupId; @@ -66,7 +66,7 @@ public class CoordinatorRequestManager implements RequestManager { public CoordinatorRequestManager( final Time time, final LogContext logContext, - final long requestTimeoutMs, + final long defaultApiTimeoutMs, final long retryBackoffMs, final long retryBackoffMaxMs, final BackgroundEventHandler backgroundEventHandler, @@ -75,7 +75,7 @@ public CoordinatorRequestManager( Objects.requireNonNull(groupId); this.time = time; this.log = logContext.logger(this.getClass()); - this.requestTimeoutMs = requestTimeoutMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.backgroundEventHandler = backgroundEventHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( @@ -114,11 +114,11 @@ NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long curren FindCoordinatorRequestData data = new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) .setKey(this.groupId); - Timer timer = time.timer(requestTimeoutMs); + Timer timer = time.timer(defaultApiTimeoutMs); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder(data), Optional.empty(), - timer + timer ); return unsentRequest.whenComplete((clientResponse, throwable) -> { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index c68953956edde..50e835978b99a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -53,7 +53,7 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager final FetchMetricsManager metricsManager, final NetworkClientDelegate networkClientDelegate, final ApiVersions apiVersions, - final long requestTimeoutMs) { + final int requestTimeoutMs) { super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time, apiVersions); this.networkClientDelegate = networkClientDelegate; this.requestTimeoutMs = requestTimeoutMs; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 45219d1462309..44a2c3e39216c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -190,6 +191,11 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private final Logger log; + /** + * Time instance used to create {@link Timer timers}. + */ + private final Time time; + /** * Manager to perform commit requests needed before revoking partitions (if auto-commit is * enabled) @@ -285,7 +291,8 @@ public MembershipManagerImpl(String groupId, ConsumerMetadata metadata, LogContext logContext, Optional clientTelemetryReporter, - BackgroundEventHandler backgroundEventHandler) { + BackgroundEventHandler backgroundEventHandler, + Time time) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -302,6 +309,7 @@ public MembershipManagerImpl(String groupId, this.clientTelemetryReporter = clientTelemetryReporter; this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.backgroundEventHandler = backgroundEventHandler; + this.time = time; } /** @@ -597,7 +605,7 @@ public CompletableFuture leaveGroup(Timer timer) { CompletableFuture leaveResult = new CompletableFuture<>(); leaveGroupInProgress = Optional.of(leaveResult); - CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); + CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(timer); callbackResult.whenComplete((result, error) -> { // Clear the subscription, no matter if the callback execution failed or succeeded. updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); @@ -627,7 +635,7 @@ public CompletableFuture leaveGroup(Timer timer) { * * @return Future that will complete when the callback execution completes. */ - private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignment() { + private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignment(Timer timer) { SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); droppedPartitions.addAll(subscriptions.assignedPartitions()); @@ -833,7 +841,8 @@ boolean reconcile() { // best effort to commit the offsets in the case where the epoch might have changed while // the current reconciliation is in process. Note this is using the rebalance timeout as // it is the limit enforced by the broker to complete the reconciliation process. - commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow(rebalanceTimeoutMs, true); + Timer timer = time.timer(rebalanceTimeoutMs); + commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow(Optional.of(timer), true); // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. commitResult.whenComplete((commitReqResult, commitReqError) -> { @@ -1199,14 +1208,12 @@ CompletableFuture invokeOnPartitionsLostCallback(Set parti * @return Future that will be chained within the rest of the reconciliation logic */ private CompletableFuture enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, - Set partitions, - Timer timer) { + Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent( - methodName, - sortedPartitions, - timer + methodName, + sortedPartitions ); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); 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 b0cad1d4523c5..28c92b9bcad39 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 @@ -232,7 +232,8 @@ public void addAll(final List requests) { } public void add(final UnsentRequest r) { - unsentRequests.add(Objects.requireNonNull(r)); + Objects.requireNonNull(r); + unsentRequests.add(r); } public static class PollResult { @@ -275,6 +276,10 @@ public UnsentRequest(final AbstractRequest.Builder requestBuilder, this.timer = timer; } + Timer timer() { + return timer; + } + CompletableFuture future() { return handler.future; } 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 4c43acf6230db..022c03b2ae523 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 @@ -180,7 +180,8 @@ protected RequestManagers create() { metadata, logContext, clientTelemetryReporter, - backgroundEventHandler); + backgroundEventHandler, + time); membershipManager.registerStateListener(commit); heartbeatRequestManager = new HeartbeatRequestManager( logContext, 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 3f3ec41b671da..f7ac2e30481be 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 @@ -17,13 +17,11 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -31,7 +29,6 @@ import java.time.Duration; import java.util.Objects; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -43,7 +40,6 @@ public class ApplicationEventHandler implements Closeable { private final Logger log; - private final Time time; private final BlockingQueue applicationEventQueue; private final ConsumerNetworkThread networkThread; private final IdempotentCloser closer = new IdempotentCloser(); @@ -55,7 +51,6 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier) { this.log = logContext.logger(ApplicationEventHandler.class); - this.time = time; this.applicationEventQueue = applicationEventQueue; this.networkThread = new ConsumerNetworkThread(logContext, time, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 0a9fa73990d85..bfdf2bf38873e 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 @@ -27,6 +27,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -35,6 +36,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; @@ -153,9 +155,25 @@ private void process(final CommitApplicationEvent event) { return; } - Timer timer = timer(event); + final Optional timer; + + if (event.allowsRetries()) { + Timer t = timer(event); + + if (maybeTimeout(event, t, "Unable to commit offset due to exceeding timeout")) + return; + + timer = Optional.of(t); + } else { + timer = Optional.empty(); + } + CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture future = manager.addOffsetCommitRequest(event.offsets(), timer, false); + CompletableFuture future = manager.addOffsetCommitRequest( + event.offsets(), + timer, + false + ); event.chain(future); } @@ -167,6 +185,10 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { } Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to fetch committed offset due to exceeding timeout")) + return; + CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture> future = manager.addOffsetFetchRequest( event.partitions(), @@ -195,6 +217,10 @@ private void process(final AssignmentChangeApplicationEvent event) { private void process(final ListOffsetsApplicationEvent event) { final CompletableFuture> future; final Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to list offsets due to exceeding timeout")) + return; + future = requestManagers.offsetsRequestManager.fetchOffsets( event.timestampsToSearch(), event.requireTimestamps(), @@ -233,24 +259,40 @@ private void process(final UnsubscribeApplicationEvent event) { } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to unsubscribe due to exceeding timeout")) + return; + CompletableFuture result = membershipManager.leaveGroup(timer); event.chain(result); } private void process(final ResetPositionsApplicationEvent event) { Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to reset positions due to exceeding timeout")) + return; + CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(timer); event.chain(result); } private void process(final ValidatePositionsApplicationEvent event) { Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to validate positions due to exceeding timeout")) + return; + CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(timer); event.chain(result); } private void process(final TopicMetadataApplicationEvent event) { - final Timer timer = timer(event); + Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to retrieve topic metadata due to exceeding timeout")) + return; + final CompletableFuture>> future; if (event.isAllTopics()) { @@ -291,6 +333,10 @@ private void process(final LeaveOnCloseApplicationEvent event) { "membership manager to be non-null"); log.debug("Leaving group before closing"); final Timer timer = timer(event); + + if (maybeTimeout(event, timer, "Unable to leave group due to exceeding timeout")) + return; + CompletableFuture future = membershipManager.leaveGroup(timer); // The future will be completed on heartbeat sent event.chain(future); @@ -299,14 +345,21 @@ private void process(final LeaveOnCloseApplicationEvent event) { /** * Creates a {@link Timer time} for the network I/O thread that is separate from the timer for the * application thread. - * - * @param event - * @return */ private Timer timer(CompletableEvent event) { return time.timer(event.deadlineMs() - time.milliseconds()); } + private boolean maybeTimeout(CompletableEvent event, Timer timer, String timeoutMessage) { + if (timer.isExpired()) { + Exception exception = new TimeoutException(timeoutMessage); + event.future().completeExceptionally(exception); + return true; + } else { + return false; + } + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index b50bb0af4d507..59a674c9e512f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Objects; public class CommitApplicationEvent extends CompletableApplicationEvent { @@ -30,15 +31,26 @@ public class CommitApplicationEvent extends CompletableApplicationEvent { */ private final Map offsets; + /** + * The value of this flag is true if the requests allows to be retried + * (sync requests that provide an expiration time to bound the retries). The value + * of false means that we do not allow retries on RetriableErrors (async + * requests that does not provide an expiration time for retries). + */ + private final boolean allowsRetries; + /** * Create new event to commit offsets. If timer is present, the request will be retried on * retriable errors until the timer expires (sync commit offsets request). If the timer is * not present, the request will be sent without waiting for a response of retrying (async * commit offsets request). */ - public CommitApplicationEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_SYNC, timer); + public CommitApplicationEvent(final Map offsets, + final Timer timer, + final boolean allowsRetries) { + super(Type.COMMIT, timer); this.offsets = Collections.unmodifiableMap(offsets); + this.allowsRetries = allowsRetries; for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { @@ -51,6 +63,10 @@ public Map offsets() { return offsets; } + public boolean allowsRetries() { + return allowsRetries; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -59,14 +75,12 @@ public boolean equals(Object o) { CommitApplicationEvent that = (CommitApplicationEvent) o; - return offsets.equals(that.offsets); + return offsets.equals(that.offsets) && allowsRetries == that.allowsRetries; } @Override public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + return 31 * super.hashCode() + Objects.hash(offsets, allowsRetries); } @Override @@ -74,6 +88,7 @@ public String toString() { return "CommitApplicationEvent{" + toStringBase() + ", offsets=" + offsets + + ", allowsRetries=" + allowsRetries + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index f701523d6d2e7..323e23e3b274e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - import java.util.Objects; import java.util.concurrent.CompletableFuture; @@ -27,27 +25,19 @@ * * @param */ -public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { +public abstract class CompletableBackgroundEvent extends BackgroundEvent { private final CompletableFuture future; - private final long deadlineMs; - protected CompletableBackgroundEvent(Type type, Timer timer) { + protected CompletableBackgroundEvent(Type type) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } - @Override public CompletableFuture future() { return future; } - @Override - public long deadlineMs() { - return deadlineMs; - } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -56,12 +46,12 @@ public boolean equals(Object o) { CompletableBackgroundEvent that = (CompletableBackgroundEvent) o; - return future.equals(that.future) && deadlineMs == that.deadlineMs; + return future.equals(that.future); } @Override public int hashCode() { - return Objects.hash(future, deadlineMs); + return Objects.hash(future); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 4644ceca01042..7b17c034abdbd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.time.Duration; import java.util.Collections; @@ -39,9 +38,8 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions, - Timer timer) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); + SortedSet partitions) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } 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 f6fb2a8928631..cb8cf9479acae 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 @@ -313,7 +313,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -335,7 +335,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); } @Test @@ -343,7 +343,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsApplicationEvent.class))).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); @@ -552,9 +552,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(null).when(applicationEventHandler).addAndGet(any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); } @@ -597,7 +597,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -730,15 +730,14 @@ public void testBeginningOffsets() { Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -748,26 +747,23 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class), - any()); + any(ListOffsetsApplicationEvent.class)); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -802,12 +798,11 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -825,8 +820,7 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -841,7 +835,7 @@ public void testWakeupCommitted() { return ConsumerUtils.getResult(event.future(), timer); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1041,9 +1035,13 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); + Timer timer = time.timer(defaultApiTimeoutMs); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( + methodName, + partitions + ); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. @@ -1247,20 +1245,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } } @@ -1275,11 +1273,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } @Test @@ -1444,13 +1442,13 @@ private void completeCommitApplicationEventExceptionally() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { 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 1ac58248c97be..d57098feb6afb 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 @@ -43,6 +43,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -101,7 +102,6 @@ public class CommitRequestManagerTest { private CoordinatorRequestManager coordinatorRequestManager; private Metrics metrics = new Metrics(); private Properties props; - private final int defaultApiTimeoutMs = 60000; @@ -120,23 +120,25 @@ public void setup() { @Test public void testPollSkipIfCoordinatorUnknown() { CommitRequestManager commitRequestManger = create(false, 0); - assertPoll(false, 0, commitRequestManger); + Timer timer = time.timer(defaultApiTimeoutMs); + assertPoll(false, 0, commitRequestManger, timer); Map offsets = new HashMap<>(); offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0)); commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); - assertPoll(false, 0, commitRequestManger); + assertPoll(false, 0, commitRequestManger, timer); } @Test public void testPollEnsureManualCommitSent() { CommitRequestManager commitRequestManger = create(false, 0); - assertPoll(0, commitRequestManger); + Timer timer = time.timer(defaultApiTimeoutMs); + assertPoll(0, commitRequestManger, timer); Map offsets = new HashMap<>(); offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0)); commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); - assertPoll(1, commitRequestManger); + assertPoll(1, commitRequestManger, timer); } @Test @@ -145,17 +147,19 @@ public void testPollEnsureAutocommitSent() { subscriptionState.assignFromUser(Collections.singleton(tp)); subscriptionState.seek(tp, 100); CommitRequestManager commitRequestManger = create(true, 100); - assertPoll(0, commitRequestManger); + Timer timer = time.timer(defaultApiTimeoutMs); + assertPoll(0, commitRequestManger, timer); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - time.sleep(100); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - List pollResults = assertPoll(1, commitRequestManger); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + timer.sleep(100); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + List pollResults = assertPoll(1, commitRequestManger, timer); pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse( "t1", 1, (short) 1, - Errors.NONE))); + Errors.NONE, + timer))); } @Test @@ -174,14 +178,14 @@ public void testPollEnsureCorrectInflightRequestBufferSize() { // Add the requests to the CommitRequestManager and store their futures ArrayList> commitFutures = new ArrayList<>(); ArrayList>> fetchFutures = new ArrayList<>(); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitFutures.add(commitManager.addOffsetCommitRequest(offsets1, Optional.of(expirationTimeMs), false)); - fetchFutures.add(commitManager.addOffsetFetchRequest(Collections.singleton(new TopicPartition("test", 0)), expirationTimeMs)); - commitFutures.add(commitManager.addOffsetCommitRequest(offsets2, Optional.of(expirationTimeMs), false)); - fetchFutures.add(commitManager.addOffsetFetchRequest(Collections.singleton(new TopicPartition("test", 1)), expirationTimeMs)); + Timer timer = time.timer(defaultApiTimeoutMs); + commitFutures.add(commitManager.addOffsetCommitRequest(offsets1, Optional.of(timer), false)); + fetchFutures.add(commitManager.addOffsetFetchRequest(Collections.singleton(new TopicPartition("test", 0)), timer)); + commitFutures.add(commitManager.addOffsetCommitRequest(offsets2, Optional.of(timer), false)); + fetchFutures.add(commitManager.addOffsetFetchRequest(Collections.singleton(new TopicPartition("test", 1)), timer)); // Poll the CommitRequestManager and verify that the inflightOffsetFetches size is correct - NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult result = commitManager.poll(timer.currentTimeMs()); assertEquals(4, result.unsentRequests.size()); assertTrue(result.unsentRequests .stream().anyMatch(r -> r.requestBuilder() instanceof OffsetCommitRequest.Builder)); @@ -205,7 +209,8 @@ public void testPollEnsureEmptyPendingRequestAfterPoll() { new OffsetAndMetadata(0)); commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); assertEquals(1, commitRequestManger.unsentOffsetCommitRequests().size()); - assertEquals(1, commitRequestManger.poll(time.milliseconds()).unsentRequests.size()); + Timer timer = time.timer(defaultApiTimeoutMs); + assertEquals(1, commitRequestManger.poll(timer.currentTimeMs()).unsentRequests.size()); assertTrue(commitRequestManger.unsentOffsetCommitRequests().isEmpty()); assertEmptyPendingRequests(commitRequestManger); } @@ -219,35 +224,38 @@ public void testAsyncAutocommitNotRetriedAfterException() { TopicPartition tp = new TopicPartition("topic", 1); subscriptionState.assignFromUser(Collections.singleton(tp)); subscriptionState.seek(tp, 100); - time.sleep(commitInterval); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - List futures = assertPoll(1, commitRequestManger); + Timer timer = time.timer(defaultApiTimeoutMs); + timer.sleep(commitInterval); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + List futures = assertPoll(1, commitRequestManger, timer); // Complete the autocommit request exceptionally. It should fail right away, without retry. futures.get(0).onComplete(mockOffsetCommitResponse( "topic", 1, (short) 1, - Errors.COORDINATOR_LOAD_IN_PROGRESS)); + Errors.COORDINATOR_LOAD_IN_PROGRESS, + timer)); // When polling again before the auto-commit interval no request should be generated // (making sure we wait for the backoff, to check that the failed request is not being // retried). - time.sleep(retryBackoffMs); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - assertPoll(0, commitRequestManger); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); + timer.sleep(retryBackoffMs); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + assertPoll(0, commitRequestManger, timer); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); // Only when polling after the auto-commit interval, a new auto-commit request should be // generated. - time.sleep(commitInterval); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - futures = assertPoll(1, commitRequestManger); + timer.sleep(commitInterval); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + futures = assertPoll(1, commitRequestManger, timer); assertEmptyPendingRequests(commitRequestManger); futures.get(0).onComplete(mockOffsetCommitResponse( "topic", 1, (short) 1, - Errors.NONE)); + Errors.NONE, + timer)); } // This is the case of the sync auto commit sent when the consumer is being closed (sync commit @@ -261,17 +269,17 @@ public void testSyncAutocommitRetriedAfterRetriableException(Errors error) { subscriptionState.assignFromUser(Collections.singleton(tp)); subscriptionState.seek(tp, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); - time.sleep(commitInterval); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + timer.sleep(commitInterval); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); // Auto-commit all consume sync (ex. triggered when the consumer is closed). - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture commitResult = - commitRequestManger.maybeAutoCommitAllConsumedNow(Optional.of(expirationTimeMs), false); - sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManger, error, commitResult); + commitRequestManger.maybeAutoCommitAllConsumedNow(Optional.of(timer), false); + sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManger, error, commitResult, timer); // We expect that request should have been retried on this sync commit. - assertExceptionHandling(commitRequestManger, error, true); + assertExceptionHandling(commitRequestManger, error, true, timer); assertCoordinatorDisconnect(error); } @@ -283,16 +291,16 @@ public void testOffsetCommitFailsWithCommitFailedExceptionIfUnknownMemberId() { subscriptionState.assignFromUser(Collections.singleton(tp)); subscriptionState.seek(tp, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); - time.sleep(commitInterval); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + timer.sleep(commitInterval); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); // Auto-commit all consume sync (ex. triggered when the consumer is closed). - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture commitResult = - commitRequestManger.maybeAutoCommitAllConsumedNow(Optional.of(expirationTimeMs), false); + commitRequestManger.maybeAutoCommitAllConsumedNow(Optional.of(timer), false); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.UNKNOWN_MEMBER_ID); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.UNKNOWN_MEMBER_ID, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); // Commit should fail with CommitFailedException assertTrue(commitResult.isDone()); @@ -314,10 +322,11 @@ public void testOffsetCommitFailsWithCommitFailedExceptionIfStaleMemberEpochNotR new OffsetAndMetadata(0)); // Send commit request expected to be retried on retriable errors + Timer timer = time.timer(defaultApiTimeoutMs); CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest( - offsets, Optional.of(time.milliseconds() + defaultApiTimeoutMs), false); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.STALE_MEMBER_EPOCH); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + offsets, Optional.of(timer), false); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.STALE_MEMBER_EPOCH, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); // Commit should fail with CommitFailedException @@ -342,8 +351,9 @@ public void testOffsetAutoCommitOnIntervalFailsWithRetriableOnStaleMemberEpoch() // Async auto-commit that won't be retried. CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest( offsets, Optional.empty(), true); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.STALE_MEMBER_EPOCH); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.STALE_MEMBER_EPOCH, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); // Commit should fail with RetriableCommitFailedException. @@ -358,20 +368,21 @@ public void testAutocommitEnsureOnlyOneInflightRequest() { subscriptionState.seek(t1p, 100); CommitRequestManager commitRequestManger = create(true, 100); - time.sleep(100); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); - List futures = assertPoll(1, commitRequestManger); + Timer timer = time.timer(defaultApiTimeoutMs); + timer.sleep(100); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); + List futures = assertPoll(1, commitRequestManger, timer); - time.sleep(100); - commitRequestManger.updateAutoCommitTimer(time.milliseconds()); + timer.sleep(100); + commitRequestManger.updateAutoCommitTimer(timer.currentTimeMs()); // We want to make sure we don't resend autocommit if the previous request has not been // completed, even if the interval expired - assertPoll(0, commitRequestManger); + assertPoll(0, commitRequestManger, timer); assertEmptyPendingRequests(commitRequestManger); // complete the unsent request and re-poll - futures.get(0).onComplete(buildOffsetCommitClientResponse(new OffsetCommitResponse(0, new HashMap<>()))); - assertPoll(1, commitRequestManger); + futures.get(0).onComplete(buildOffsetCommitClientResponse(new OffsetCommitResponse(0, new HashMap<>()), timer)); + assertPoll(1, commitRequestManger, timer); } @Test @@ -408,9 +419,10 @@ public void testOffsetFetchRequestErroredRequests(final Errors error, final bool 5, error); // we only want to make sure to purge the outbound buffer for non-retriables, so retriable will be re-queued. - if (isRetriable) - testRetriable(commitRequestManger, futures); - else { + if (isRetriable) { + Timer timer = time.timer(defaultApiTimeoutMs); + testRetriable(commitRequestManger, futures, timer); + } else { testNonRetriable(futures); assertEmptyPendingRequests(commitRequestManger); } @@ -429,8 +441,9 @@ public void testOffsetCommitRequestErroredRequestsNotRetriedForAsyncCommit(final // Send commit request without expiration (async commit not expected to be retried). CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); - completeOffsetCommitRequestWithError(commitRequestManger, error); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + completeOffsetCommitRequestWithError(commitRequestManger, error, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); assertTrue(commitResult.isCompletedExceptionally()); @@ -439,7 +452,7 @@ public void testOffsetCommitRequestErroredRequestsNotRetriedForAsyncCommit(final } // We expect that the request should not have been retried on this async commit. - assertExceptionHandling(commitRequestManger, error, false); + assertExceptionHandling(commitRequestManger, error, false, timer); assertCoordinatorDisconnect(error); } @@ -456,8 +469,9 @@ public void testAsyncOffsetCommitThrowsRetriableCommitExceptionForUnhandledRetri // network exception that has no specific handling. Should fail with // RetriableCommitException. CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.NETWORK_EXCEPTION); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.NETWORK_EXCEPTION, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); assertTrue(commitResult.isCompletedExceptionally()); @@ -474,19 +488,19 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() { new OffsetAndMetadata(0)); // Send sync offset commit request that fails with retriable error. - Long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; - CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(expirationTimeMs), false); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.REQUEST_TIMED_OUT); + Timer timer = time.timer(retryBackoffMs * 2); + CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(timer), false); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.REQUEST_TIMED_OUT, timer); // Request retried after backoff, and fails with retriable again. Should not complete yet // given that the request timeout hasn't expired. - time.sleep(retryBackoffMs); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.REQUEST_TIMED_OUT); + timer.sleep(retryBackoffMs); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.REQUEST_TIMED_OUT, timer); assertFalse(commitResult.isDone()); // Sleep to expire the request timeout. Request should fail on the next poll. - time.sleep(retryBackoffMs); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + timer.sleep(retryBackoffMs); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); assertTrue(commitResult.isCompletedExceptionally()); @@ -506,14 +520,14 @@ public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExp new OffsetAndMetadata(0)); // Send offset commit request that fails with retriable error. - Long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; - CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(expirationTimeMs), false); - completeOffsetCommitRequestWithError(commitRequestManger, Errors.COORDINATOR_NOT_AVAILABLE); + Timer timer = time.timer(retryBackoffMs * 2); + CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(timer), false); + completeOffsetCommitRequestWithError(commitRequestManger, Errors.COORDINATOR_NOT_AVAILABLE, timer); // Sleep to expire the request timeout. Request should fail on the next poll with a // TimeoutException. - time.sleep(expirationTimeMs); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + timer.sleep(timer.remainingMs()); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); assertTrue(commitResult.isCompletedExceptionally()); @@ -536,14 +550,15 @@ public void testOffsetCommitAsyncFailedWithRetriableThrowsRetriableCommitExcepti // Send async commit request that fails with retriable error (not expected to be retried). Errors retriableError = Errors.COORDINATOR_NOT_AVAILABLE; CompletableFuture commitResult = commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); - completeOffsetCommitRequestWithError(commitRequestManger, retriableError); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + completeOffsetCommitRequestWithError(commitRequestManger, retriableError, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); assertTrue(commitResult.isCompletedExceptionally()); // We expect that the request should not have been retried on this async commit. - assertExceptionHandling(commitRequestManger, retriableError, false); + assertExceptionHandling(commitRequestManger, retriableError, false, timer); // Request should have been completed with a RetriableCommitException Throwable t = assertThrows(ExecutionException.class, () -> commitResult.get()); @@ -560,15 +575,15 @@ public void testEnsureBackoffRetryOnOffsetCommitRequestTimeout() { Map offsets = Collections.singletonMap(new TopicPartition("topic", 1), new OffsetAndMetadata(0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(expirationTimeMs), false); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + commitRequestManger.addOffsetCommitRequest(offsets, Optional.of(timer), false); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); - res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); + res.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException()); assertTrue(commitRequestManger.pendingRequests.hasUnsentRequests()); assertEquals(1, commitRequestManger.unsentOffsetCommitRequests().size()); - assertRetryBackOff(commitRequestManger, retryBackoffMs); + assertRetryBackOff(commitRequestManger, retryBackoffMs, timer); } private void assertCoordinatorDisconnect(final Errors error) { @@ -578,7 +593,7 @@ private void assertCoordinatorDisconnect(final Errors error) { } private void assertExceptionHandling(CommitRequestManager commitRequestManger, Errors errors, - boolean requestShouldBeRetried) { + boolean requestShouldBeRetried, Timer timer) { long remainBackoffMs; if (requestShouldBeRetried) { remainBackoffMs = retryBackoffMs; @@ -590,12 +605,12 @@ private void assertExceptionHandling(CommitRequestManager commitRequestManger, E case COORDINATOR_NOT_AVAILABLE: case REQUEST_TIMED_OUT: verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong()); - assertPollDoesNotReturn(commitRequestManger, remainBackoffMs); + assertPollDoesNotReturn(commitRequestManger, remainBackoffMs, timer); break; case UNKNOWN_TOPIC_OR_PARTITION: case COORDINATOR_LOAD_IN_PROGRESS: if (requestShouldBeRetried) { - assertRetryBackOff(commitRequestManger, remainBackoffMs); + assertRetryBackOff(commitRequestManger, remainBackoffMs, timer); } break; case GROUP_AUTHORIZATION_FAILED: @@ -604,33 +619,33 @@ private void assertExceptionHandling(CommitRequestManager commitRequestManger, E case TOPIC_AUTHORIZATION_FAILED: case OFFSET_METADATA_TOO_LARGE: case INVALID_COMMIT_OFFSET_SIZE: - assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE); + assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE, timer); break; case FENCED_INSTANCE_ID: // This is a fatal failure, so we should not retry - assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE); + assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE, timer); break; default: if (errors.exception() instanceof RetriableException && requestShouldBeRetried) { - assertRetryBackOff(commitRequestManger, remainBackoffMs); + assertRetryBackOff(commitRequestManger, remainBackoffMs, timer); } else { - assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE); + assertPollDoesNotReturn(commitRequestManger, Long.MAX_VALUE, timer); } } } - private void assertPollDoesNotReturn(CommitRequestManager commitRequestManager, long assertNextPollMs) { - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + private void assertPollDoesNotReturn(CommitRequestManager commitRequestManager, long assertNextPollMs, Timer timer) { + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertEquals(assertNextPollMs, res.timeUntilNextPollMs); } - private void assertRetryBackOff(CommitRequestManager commitRequestManager, long retryBackoffMs) { - assertPollDoesNotReturn(commitRequestManager, retryBackoffMs); - time.sleep(retryBackoffMs - 1); - assertPollDoesNotReturn(commitRequestManager, 1); - time.sleep(1); - assertPoll(1, commitRequestManager); + private void assertRetryBackOff(CommitRequestManager commitRequestManager, long retryBackoffMs, Timer timer) { + assertPollDoesNotReturn(commitRequestManager, retryBackoffMs, timer); + timer.sleep(retryBackoffMs - 1); + assertPollDoesNotReturn(commitRequestManager, 1, timer); + timer.sleep(1); + assertPoll(1, commitRequestManager, timer); } // This should be the case where the OffsetFetch fails with invalid member epoch and the @@ -644,8 +659,8 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndRetriesWithNewEpoch() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); // Send request that is expected to fail with invalid epoch. - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitRequestManager.addOffsetFetchRequest(partitions, expirationTimeMs); + Timer timer = time.timer(defaultApiTimeoutMs); + commitRequestManager.addOffsetFetchRequest(partitions, timer); // Mock member has new a valid epoch. int newEpoch = 8; @@ -653,17 +668,17 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndRetriesWithNewEpoch() { commitRequestManager.onMemberEpochUpdated(Optional.of(newEpoch), Optional.of(memberId)); // Receive error when member already has a newer member epoch. Request should be retried. - completeOffsetFetchRequestWithError(commitRequestManager, partitions, Errors.STALE_MEMBER_EPOCH); + completeOffsetFetchRequestWithError(commitRequestManager, partitions, Errors.STALE_MEMBER_EPOCH, timer); // Check that the request that failed was removed from the inflight requests buffer. assertEquals(0, commitRequestManager.pendingRequests.inflightOffsetFetches.size()); assertEquals(1, commitRequestManager.pendingRequests.unsentOffsetFetches.size()); // Request should be retried with backoff. - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); - time.sleep(retryBackoffMs); - res = commitRequestManager.poll(time.milliseconds()); + timer.sleep(retryBackoffMs); + res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); // The retried request should include the latest member ID and epoch. @@ -683,21 +698,21 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndNotRetriedIfMemberNotInGrou when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); // Send request that is expected to fail with invalid epoch. - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + Timer timer = time.timer(defaultApiTimeoutMs); CompletableFuture> requestResult = - commitRequestManager.addOffsetFetchRequest(partitions, expirationTimeMs); + commitRequestManager.addOffsetFetchRequest(partitions, timer); // Mock member not having a valid epoch anymore (left/failed/fenced). commitRequestManager.onMemberEpochUpdated(Optional.empty(), Optional.empty()); // Receive error when member is not in the group anymore. Request should fail. - completeOffsetFetchRequestWithError(commitRequestManager, partitions, Errors.STALE_MEMBER_EPOCH); + completeOffsetFetchRequestWithError(commitRequestManager, partitions, Errors.STALE_MEMBER_EPOCH, timer); assertTrue(requestResult.isDone()); assertTrue(requestResult.isCompletedExceptionally()); // No new request should be generated on the next poll. - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); } @@ -715,25 +730,25 @@ public void testOffsetCommitFailsWithStaleEpochAndRetriesWithNewEpoch() { // Send commit request expected to be retried on STALE_MEMBER_EPOCH. // errors while it does not expire). - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitRequestManager.addOffsetCommitRequest(offsets, Optional.of(expirationTimeMs), true); + Timer timer = time.timer(defaultApiTimeoutMs); + commitRequestManager.addOffsetCommitRequest(offsets, Optional.of(timer), true); // Mock member has new a valid epoch. int newEpoch = 8; String memberId = "member1"; commitRequestManager.onMemberEpochUpdated(Optional.of(newEpoch), Optional.of(memberId)); - completeOffsetCommitRequestWithError(commitRequestManager, Errors.STALE_MEMBER_EPOCH); + completeOffsetCommitRequestWithError(commitRequestManager, Errors.STALE_MEMBER_EPOCH, timer); // Check that the request that failed was removed from the inflight requests buffer. assertEquals(0, commitRequestManager.pendingRequests.inflightOffsetFetches.size()); assertEquals(1, commitRequestManager.pendingRequests.unsentOffsetCommits.size()); // Request should be retried with backoff. - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); - time.sleep(retryBackoffMs); - res = commitRequestManager.poll(time.milliseconds()); + timer.sleep(retryBackoffMs); + res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); // The retried request should include the latest member ID and epoch. @@ -747,8 +762,9 @@ public void testEnsureCommitSensorRecordsMetric() { CommitRequestManager commitRequestManager = create(true, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); - commitOffsetWithAssertedLatency(commitRequestManager, 100); - commitOffsetWithAssertedLatency(commitRequestManager, 101); + Timer timer = time.timer(defaultApiTimeoutMs); + commitOffsetWithAssertedLatency(commitRequestManager, 100, timer); + commitOffsetWithAssertedLatency(commitRequestManager, 101, timer); assertEquals(100.5, getMetric("commit-latency-avg").metricValue()); // 201 / 2 assertEquals(101.0, getMetric("commit-latency-max").metricValue()); // Math.max(101, 100) @@ -756,21 +772,21 @@ public void testEnsureCommitSensorRecordsMetric() { assertEquals(2.0, getMetric("commit-total").metricValue()); } - private void commitOffsetWithAssertedLatency(CommitRequestManager commitRequestManager, long latencyMs) { + private void commitOffsetWithAssertedLatency(CommitRequestManager commitRequestManager, long latencyMs, Timer timer) { final String topic = "topic"; final int partition = 1; Map offsets = Collections.singletonMap( new TopicPartition(topic, partition), new OffsetAndMetadata(0)); - long commitCreationTimeMs = time.milliseconds(); + long commitCreationTimeMs = timer.currentTimeMs(); commitRequestManager.addOffsetCommitRequest(offsets, Optional.empty(), true); - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); - time.sleep(latencyMs); - long commitReceivedTimeMs = time.milliseconds(); + timer.sleep(latencyMs); + long commitReceivedTimeMs = timer.currentTimeMs(); res.unsentRequests.get(0).future().complete(mockOffsetCommitResponse( topic, partition, @@ -782,26 +798,29 @@ private void commitOffsetWithAssertedLatency(CommitRequestManager commitRequestM private void completeOffsetFetchRequestWithError(CommitRequestManager commitRequestManager, Set partitions, - Errors error) { - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + Errors error, + Timer timer) { + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().complete(buildOffsetFetchClientResponse(res.unsentRequests.get(0), partitions, error)); } private void completeOffsetCommitRequestWithError(CommitRequestManager commitRequestManager, - Errors error) { - NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); + Errors error, + Timer timer) { + NetworkClientDelegate.PollResult res = commitRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); - res.unsentRequests.get(0).future().complete(mockOffsetCommitResponse("topic", 1, (short) 1, error)); + res.unsentRequests.get(0).future().complete(mockOffsetCommitResponse("topic", 1, (short) 1, error, timer)); } private void testRetriable(final CommitRequestManager commitRequestManger, - final List>> futures) { + final List>> futures, + final Timer timer) { futures.forEach(f -> assertFalse(f.isDone())); // The manager should backoff for 100ms - time.sleep(100); - commitRequestManger.poll(time.milliseconds()); + timer.sleep(100); + commitRequestManger.poll(timer.currentTimeMs()); futures.forEach(f -> assertFalse(f.isDone())); } @@ -861,11 +880,11 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b TopicPartition tp2 = new TopicPartition("t2", 3); partitions.add(tp1); partitions.add(tp2); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + Timer timer = time.timer(defaultApiTimeoutMs); CompletableFuture> future = - commitRequestManger.addOffsetFetchRequest(partitions, expirationTimeMs); + commitRequestManger.addOffsetFetchRequest(partitions, timer); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); // Setting 1 partition with error @@ -878,7 +897,7 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b topicPartitionData, Errors.NONE)); if (isRetriable) - testRetriable(commitRequestManger, Collections.singletonList(future)); + testRetriable(commitRequestManger, Collections.singletonList(future), timer); else testNonRetriable(Collections.singletonList(future)); } @@ -893,7 +912,8 @@ public void testSignalClose() { commitRequestManger.addOffsetCommitRequest(offsets, Optional.empty(), false); commitRequestManger.signalClose(); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); OffsetCommitRequestData data = (OffsetCommitRequestData) res.unsentRequests.get(0).requestBuilder().build().data(); assertEquals("topic", data.topics().get(0).name()); @@ -920,16 +940,16 @@ private List>> sendAndV int numRequest, final Errors error) { List>> futures = new ArrayList<>(); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + Timer timer = time.timer(defaultApiTimeoutMs); for (int i = 0; i < numRequest; i++) { - futures.add(commitRequestManger.addOffsetFetchRequest(partitions, expirationTimeMs)); + futures.add(commitRequestManger.addOffsetFetchRequest(partitions, timer)); } - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).handler().onComplete(buildOffsetFetchClientResponse(res.unsentRequests.get(0), partitions, error)); - res = commitRequestManger.poll(time.milliseconds()); + res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); return futures; } @@ -937,9 +957,10 @@ private List>> sendAndV private void sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried( final CommitRequestManager commitRequestManger, final Errors error, - final CompletableFuture commitResult) { - completeOffsetCommitRequestWithError(commitRequestManger, error); - NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); + final CompletableFuture commitResult, + final Timer timer) { + completeOffsetCommitRequestWithError(commitRequestManger, error, timer); + NetworkClientDelegate.PollResult res = commitRequestManger.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); if (error.exception() instanceof RetriableException) { // Commit should not complete if the timer is still valid and the error is retriable. @@ -953,20 +974,22 @@ private void sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried( private List assertPoll( final int numRes, - final CommitRequestManager manager) { - return assertPoll(true, numRes, manager); + final CommitRequestManager manager, + final Timer timer) { + return assertPoll(true, numRes, manager, timer); } private List assertPoll( final boolean coordinatorDiscovered, final int numRes, - final CommitRequestManager manager) { + final CommitRequestManager manager, + final Timer timer) { if (coordinatorDiscovered) { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); } else { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); } - NetworkClientDelegate.PollResult res = manager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = manager.poll(timer.currentTimeMs()); assertEquals(numRes, res.unsentRequests.size()); return res.unsentRequests.stream().map(NetworkClientDelegate.UnsentRequest::handler).collect(Collectors.toList()); @@ -990,6 +1013,7 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long retryBackoffMs, retryBackoffMaxMs, OptionalDouble.of(0), + defaultApiTimeoutMs, consumerMetricGroupPrefix, metrics)); } @@ -1007,14 +1031,15 @@ private ClientResponse buildOffsetFetchClientResponse( return buildOffsetFetchClientResponse(request, topicPartitionData, error); } - private ClientResponse buildOffsetCommitClientResponse(final OffsetCommitResponse commitResponse) { + private ClientResponse buildOffsetCommitClientResponse(final OffsetCommitResponse commitResponse, + final Timer timer) { short apiVersion = 1; return new ClientResponse( new RequestHeader(ApiKeys.OFFSET_COMMIT, apiVersion, "", 1), null, "-1", - time.milliseconds(), - time.milliseconds(), + timer.currentTimeMs(), + timer.currentTimeMs(), false, null, null, @@ -1026,8 +1051,9 @@ private ClientResponse buildOffsetCommitClientResponse(final OffsetCommitRespons public ClientResponse mockOffsetCommitResponse(String topic, int partition, short apiKeyVersion, - Errors error) { - return mockOffsetCommitResponse(topic, partition, apiKeyVersion, time.milliseconds(), time.milliseconds(), error); + Errors error, + Timer timer) { + return mockOffsetCommitResponse(topic, partition, apiKeyVersion, timer.currentTimeMs(), timer.currentTimeMs(), error); } public ClientResponse mockOffsetCommitResponse(String topic, int partition, @@ -1071,8 +1097,8 @@ private ClientResponse buildOffsetFetchClientResponse( new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), false, null, null, 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 91fed714a983d..1a68b218e334a 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; @@ -39,6 +40,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -56,7 +58,6 @@ import java.util.concurrent.CompletableFuture; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -88,6 +89,7 @@ public class ConsumerNetworkThreadTest { private ConsumerNetworkThread consumerNetworkThread; private MockClient client; private SubscriptionState subscriptions; + private int defaultApiTimeoutMs; @BeforeEach public void setup() { @@ -106,6 +108,7 @@ public void setup() { consumerNetworkThread = testBuilder.consumerNetworkThread; subscriptions = testBuilder.subscriptions; consumerNetworkThread.initializeResources(); + defaultApiTimeoutMs = testBuilder.config.getInt(CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG); } @AfterEach @@ -135,7 +138,8 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testApplicationEvent() { - ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); + Timer timer = time.timer(defaultApiTimeoutMs); + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), timer, false); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor, times(1)).process(e); @@ -151,7 +155,8 @@ public void testMetadataUpdateEvent() { @Test public void testCommitEvent() { - ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), Optional.empty()); + Timer timer = time.timer(defaultApiTimeoutMs); + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>(), timer, false); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(CommitApplicationEvent.class)); @@ -159,8 +164,9 @@ public void testCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { + Timer timer = time.timer(defaultApiTimeoutMs); Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, time.timer(Long.MAX_VALUE)); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); @@ -178,9 +184,10 @@ public void testResetPositionsEventIsProcessed() { @Test public void testResetPositionsProcessFailureIsIgnored() { - doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); + Timer timer = time.timer(defaultApiTimeoutMs); + doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(timer); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(time.timer(Long.MAX_VALUE)); + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(timer); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -189,7 +196,8 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(time.timer(Long.MAX_VALUE)); + Timer timer = time.timer(defaultApiTimeoutMs); + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); @@ -198,9 +206,10 @@ public void testValidatePositionsEventIsProcessed() { @Test public void testAssignmentChangeEvent() { + Timer timer = time.timer(defaultApiTimeoutMs); HashMap offset = mockTopicPartitionOffset(); - final long currentTimeMs = time.milliseconds(); + final long currentTimeMs = timer.currentTimeMs(); ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs); applicationEventsQueue.add(e); @@ -214,20 +223,22 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", time.timer(Long.MAX_VALUE))); + Timer timer = time.timer(defaultApiTimeoutMs); + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", timer)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); } @Test void testPollResultTimer() { + Timer timer = time.timer(defaultApiTimeoutMs); NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) .setKey("foobar")), Optional.empty(), - time.timer(DEFAULT_REQUEST_TIMEOUT_MS) + timer ); // purposely setting a non-MAX time to ensure it is returning Long.MAX_VALUE upon success @@ -270,12 +281,13 @@ void testEnsureMetadataUpdateOnPoll() { @Test void testEnsureEventsAreCompleted() { + Timer timer = time.timer(defaultApiTimeoutMs); Node node = metadata.fetch().nodes().get(0); - coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); + coordinatorRequestManager.markCoordinatorUnknown("test", timer.currentTimeMs()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap(), Optional.empty())); - ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap(), Optional.empty()); + CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap(), timer, false)); + ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap(), timer, false); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 5bbeea74b7aaf..9911861899d9f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -137,7 +137,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.fetchConfig = new FetchConfig(config); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.metrics = createMetrics(config, time); this.subscriptions = spy(createSubscriptionState(config, logContext)); @@ -203,7 +203,8 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA metadata, logContext, Optional.empty(), - backgroundEventHandler + backgroundEventHandler, + time ) ); HeartbeatRequestManager.HeartbeatState heartbeatState = spy(new HeartbeatRequestManager.HeartbeatState( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index d7ad1b55738c4..d12c1e1df7a45 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -45,6 +46,7 @@ import static org.mockito.Mockito.verifyNoInteractions; public class CoordinatorRequestManagerTest { + private static final int DEFAULT_API_TIMEOUT_MS = 60000; private static final int RETRY_BACKOFF_MS = 500; private static final String GROUP_ID = "group-1"; private MockTime time; @@ -61,7 +63,8 @@ public void setup() { @Test public void testSuccessfulResponse() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); Optional coordinatorOpt = coordinatorManager.coordinator(); assertTrue(coordinatorOpt.isPresent()); @@ -69,7 +72,7 @@ public void testSuccessfulResponse() { assertEquals(node.host(), coordinatorOpt.get().host()); assertEquals(node.port(), coordinatorOpt.get().port()); - NetworkClientDelegate.PollResult pollResult = coordinatorManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = coordinatorManager.poll(timer.currentTimeMs()); assertEquals(Collections.emptyList(), pollResult.unsentRequests); } @@ -77,41 +80,44 @@ public void testSuccessfulResponse() { public void testMarkCoordinatorUnknown() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); assertTrue(coordinatorManager.coordinator().isPresent()); // It may take time for metadata to converge between after a coordinator has // been demoted. This can cause a tight loop in which FindCoordinator continues to // return node X while that node continues to reply with NOT_COORDINATOR. Hence we // still want to ensure a backoff after successfully finding the coordinator. - coordinatorManager.markCoordinatorUnknown("coordinator changed", time.milliseconds()); - assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); + coordinatorManager.markCoordinatorUnknown("coordinator changed", timer.currentTimeMs()); + assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); - time.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); + timer.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); - time.sleep(RETRY_BACKOFF_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); + timer.sleep(RETRY_BACKOFF_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); assertTrue(coordinatorManager.coordinator().isPresent()); } @Test public void testBackoffAfterRetriableFailure() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS, timer); verifyNoInteractions(backgroundEventHandler); - time.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); + timer.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); - time.sleep(1); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); + timer.sleep(1); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); } @Test public void testPropagateAndBackoffAfterFatalError() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED, timer); verify(backgroundEventHandler).add(argThat(backgroundEvent -> { if (!(backgroundEvent instanceof ErrorBackgroundEvent)) @@ -126,11 +132,11 @@ public void testPropagateAndBackoffAfterFatalError() { return groupAuthException.groupId().equals(GROUP_ID); })); - time.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); + timer.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); - time.sleep(1); - assertEquals(1, coordinatorManager.poll(time.milliseconds()).unsentRequests.size()); + timer.sleep(1); + assertEquals(1, coordinatorManager.poll(timer.currentTimeMs()).unsentRequests.size()); assertEquals(Optional.empty(), coordinatorManager.coordinator()); } @@ -156,27 +162,29 @@ public void testFindCoordinatorResponseVersions() { @Test public void testNetworkTimeout() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - NetworkClientDelegate.PollResult res = coordinatorManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult res = coordinatorManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); // Mimic a network timeout - res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); + res.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException()); // Sleep for exponential backoff - 1ms - time.sleep(RETRY_BACKOFF_MS - 1); - NetworkClientDelegate.PollResult res2 = coordinatorManager.poll(this.time.milliseconds()); + timer.sleep(RETRY_BACKOFF_MS - 1); + NetworkClientDelegate.PollResult res2 = coordinatorManager.poll(timer.currentTimeMs()); assertEquals(0, res2.unsentRequests.size()); - time.sleep(1); - res2 = coordinatorManager.poll(time.milliseconds()); + timer.sleep(1); + res2 = coordinatorManager.poll(timer.currentTimeMs()); assertEquals(1, res2.unsentRequests.size()); } private void expectFindCoordinatorRequest( CoordinatorRequestManager coordinatorManager, - Errors error + Errors error, + Timer timer ) { - NetworkClientDelegate.PollResult res = coordinatorManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = coordinatorManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); @@ -190,6 +198,7 @@ private CoordinatorRequestManager setupCoordinatorManager(String groupId) { return new CoordinatorRequestManager( time, new LogContext(), + DEFAULT_API_TIMEOUT_MS, RETRY_BACKOFF_MS, RETRY_BACKOFF_MS, this.backgroundEventHandler, @@ -211,8 +220,8 @@ private ClientResponse buildResponse( new RequestHeader(ApiKeys.FIND_COORDINATOR, findCoordinatorRequest.version(), "", 1), request.handler(), node.idString(), - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), false, null, null, 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 c3da9157acfa3..09f66e67c3fad 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 @@ -177,7 +177,7 @@ public class FetchRequestManagerTest { private int maxWaitMs = 0; private int fetchSize = 1000; private long retryBackoffMs = 100; - private long requestTimeoutMs = 30000; + private int requestTimeoutMs = 30000; private MockTime time = new MockTime(1); private SubscriptionState subscriptions; private ConsumerMetadata metadata; @@ -3530,14 +3530,15 @@ private void buildFetcher(MetricConfig metricConfig, metricsManager, networkClientDelegate, fetchCollector, - apiVersions)); + apiVersions, + requestTimeoutMs)); ConsumerNetworkClient consumerNetworkClient = new ConsumerNetworkClient( logContext, client, metadata, time, retryBackoffMs, - (int) requestTimeoutMs, + requestTimeoutMs, Integer.MAX_VALUE); offsetFetcher = new OffsetFetcher(logContext, consumerNetworkClient, @@ -3589,8 +3590,9 @@ public TestableFetchRequestManager(LogContext logContext, FetchMetricsManager metricsManager, NetworkClientDelegate networkClientDelegate, FetchCollector fetchCollector, - ApiVersions apiVersions) { - super(logContext, time, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, networkClientDelegate, apiVersions); + ApiVersions apiVersions, + int requestTimeoutMs) { + super(logContext, time, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, networkClientDelegate, apiVersions, requestTimeoutMs); this.fetchCollector = fetchCollector; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java index 7683a521e0d6a..abb712abc31b7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; @@ -33,6 +34,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -91,6 +93,7 @@ public class MembershipManagerImplTest { private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private Time time; + private int defaultApiTimeoutMs; @BeforeEach public void setup() { @@ -101,6 +104,7 @@ public void setup() { backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; time = testBuilder.time; + defaultApiTimeoutMs = testBuilder.config.getInt(CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG); } @AfterEach @@ -167,7 +171,8 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() // Following joins should not register again. receiveEmptyAssignment(manager); mockLeaveGroup(); - manager.leaveGroup(); + Timer timer = time.timer(defaultApiTimeoutMs); + manager.leaveGroup(timer); assertEquals(MemberState.LEAVING, manager.state()); manager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, manager.state()); @@ -279,7 +284,8 @@ public void testListenersGetNotifiedOnTransitionsToLeavingGroup() { clearInvocations(listener); mockLeaveGroup(); - membershipManager.leaveGroup(); + Timer timer = time.timer(defaultApiTimeoutMs); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); verify(listener).onMemberEpochUpdated(Optional.empty(), Optional.empty()); } @@ -383,7 +389,8 @@ public void testFencingWhenStateIsLeaving() { // Start leaving group. mockLeaveGroup(); - membershipManager.leaveGroup(); + Timer timer = time.timer(defaultApiTimeoutMs); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); // Get fenced while leaving. Member should not trigger any callback or try to @@ -409,7 +416,8 @@ public void testLeaveGroupEpoch() { // Static member should leave the group with epoch -2. MembershipManagerImpl membershipManager = createMemberInStableState("instance1"); mockLeaveGroup(); - membershipManager.leaveGroup(); + Timer timer = time.timer(defaultApiTimeoutMs); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -417,7 +425,7 @@ public void testLeaveGroupEpoch() { // Dynamic member should leave the group with epoch -1. membershipManager = createMemberInStableState(null); mockLeaveGroup(); - membershipManager.leaveGroup(); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -593,11 +601,12 @@ public void testLeaveGroupWhenMemberOwnsAssignment() { @Test public void testLeaveGroupWhenMemberAlreadyLeaving() { MembershipManager membershipManager = createMemberInStableState(); + Timer timer = time.timer(defaultApiTimeoutMs); // First leave attempt. Should trigger the callbacks and stay LEAVING until // callbacks complete and the heartbeat is sent out. mockLeaveGroup(); - CompletableFuture leaveResult1 = membershipManager.leaveGroup(); + CompletableFuture leaveResult1 = membershipManager.leaveGroup(timer); assertFalse(leaveResult1.isDone()); assertEquals(MemberState.LEAVING, membershipManager.state()); verify(subscriptionState).assignFromSubscribed(Collections.emptySet()); @@ -607,7 +616,7 @@ public void testLeaveGroupWhenMemberAlreadyLeaving() { // trigger any callbacks, and return a future that will complete when the ongoing first // leave operation completes. mockLeaveGroup(); - CompletableFuture leaveResult2 = membershipManager.leaveGroup(); + CompletableFuture leaveResult2 = membershipManager.leaveGroup(timer); verify(subscriptionState, never()).rebalanceListener(); assertFalse(leaveResult2.isDone()); @@ -624,11 +633,12 @@ public void testLeaveGroupWhenMemberAlreadyLeaving() { @Test public void testLeaveGroupWhenMemberAlreadyLeft() { + Timer timer = time.timer(defaultApiTimeoutMs); MembershipManager membershipManager = createMemberInStableState(); // Leave group triggered and completed mockLeaveGroup(); - CompletableFuture leaveResult1 = membershipManager.leaveGroup(); + CompletableFuture leaveResult1 = membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); @@ -640,7 +650,7 @@ public void testLeaveGroupWhenMemberAlreadyLeft() { // Call to leave group again, when member already left. Should be no-op (no callbacks, // no assignment updated) mockLeaveGroup(); - CompletableFuture leaveResult2 = membershipManager.leaveGroup(); + CompletableFuture leaveResult2 = membershipManager.leaveGroup(timer); assertTrue(leaveResult2.isDone()); assertFalse(leaveResult2.isCompletedExceptionally()); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); @@ -683,11 +693,12 @@ public void testFatalFailureWhenStateIsPrepareLeaving() { @Test public void testFatalFailureWhenStateIsLeaving() { + Timer timer = time.timer(defaultApiTimeoutMs); MembershipManagerImpl membershipManager = createMemberInStableState(); // Start leaving group. mockLeaveGroup(); - membershipManager.leaveGroup(); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); // Get fatal failure while waiting to send the heartbeat to leave. Member should @@ -702,11 +713,12 @@ public void testFatalFailureWhenStateIsLeaving() { @Test public void testFatalFailureWhenMemberAlreadyLeft() { + Timer timer = time.timer(defaultApiTimeoutMs); MembershipManagerImpl membershipManager = createMemberInStableState(); // Start leaving group. mockLeaveGroup(); - membershipManager.leaveGroup(); + membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); // Last heartbeat sent. @@ -1726,9 +1738,10 @@ private void testFencedMemberReleasesAssignmentAndTransitionsToJoining(Membershi * transition to {@link MemberState#UNSUBSCRIBED} */ private void testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(MembershipManager membershipManager) { + Timer timer = time.timer(defaultApiTimeoutMs); mockLeaveGroup(); - CompletableFuture leaveResult = membershipManager.leaveGroup(); + CompletableFuture leaveResult = membershipManager.leaveGroup(timer); assertEquals(MemberState.LEAVING, membershipManager.state()); assertFalse(leaveResult.isDone(), "Leave group result should not complete until the " + @@ -1753,6 +1766,7 @@ private void mockLeaveGroup() { private ConsumerRebalanceListenerCallbackCompletedEvent mockPrepareLeavingStuckOnUserCallback( MembershipManagerImpl membershipManager, ConsumerRebalanceListenerInvoker invoker) { + Timer timer = time.timer(defaultApiTimeoutMs); String topicName = "topic1"; TopicPartition ownedPartition = new TopicPartition(topicName, 0); @@ -1763,7 +1777,7 @@ private ConsumerRebalanceListenerCallbackCompletedEvent mockPrepareLeavingStuckO when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); doNothing().when(subscriptionState).markPendingRevocation(anySet()); when(commitRequestManager.autoCommitEnabled()).thenReturn(false); - membershipManager.leaveGroup(); + membershipManager.leaveGroup(timer); return performCallback( membershipManager, invoker, 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 5ca034d636029..6573ca50b31bc 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 @@ -45,6 +45,7 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -101,6 +102,7 @@ public class OffsetsRequestManagerTest { private static final IsolationLevel DEFAULT_ISOLATION_LEVEL = IsolationLevel.READ_COMMITTED; private static final int RETRY_BACKOFF_MS = 500; private static final int REQUEST_TIMEOUT_MS = 500; + private static final int DEFAULT_API_TIMEOUT_MS = 60000; @BeforeEach public void setup() { @@ -131,14 +133,16 @@ public void testListOffsetsRequest_Success() throws ExecutionException, Interrup ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); Map expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(result, expectedOffsets); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets, timer); } @Test @@ -148,12 +152,13 @@ public void testListOffsetsWaitingForMetadataUpdate_Timeout() { // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets(timestampsToSearch, false); + requestManager.fetchOffsets(timestampsToSearch, false, timer); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); // Metadata update not happening within the time boundaries of the request future, so // future should time out. @@ -172,26 +177,30 @@ public void testListOffsetsRequestMultiplePartitions() throws ExecutionException partitionLeaders.put(TEST_PARTITION_1, LEADER_1); partitionLeaders.put(TEST_PARTITION_2, LEADER_1); mockSuccessfulRequest(partitionLeaders); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); Map expectedOffsets = timestampsToSearch.entrySet().stream() .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); - verifySuccessfulPollAndResponseReceived(result, expectedOffsets); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets, timer); } @Test public void testListOffsetsRequestEmpty() throws ExecutionException, InterruptedException { + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( Collections.emptyMap(), - false); + false, + timer); assertEquals(0, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); assertTrue(pollResult.unsentRequests.isEmpty()); assertEquals(0, requestManager.requestsToRetry()); @@ -209,16 +218,18 @@ public void testListOffsetsRequestUnknownOffset() throws ExecutionException, ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); List topicResponses = Collections.singletonList( mockUnknownOffsetResponse(TEST_PARTITION_1)); - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, topicResponses); @@ -237,14 +248,16 @@ public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws Execu // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets(timestampsToSearch, - false); + false, + timer); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertFalse(fetchOffsetsFuture.isDone()); @@ -256,7 +269,7 @@ public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws Execu Map expectedOffsets = Collections.singletonMap( TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets, timer); } @ParameterizedTest @@ -267,14 +280,16 @@ public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throw // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent to single broker - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res); assertFalse(fetchOffsetsFuture.isDone()); @@ -294,7 +309,7 @@ public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throw assertEquals(1, requestManager.requestsToSend()); Map expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets, timer); } @Test @@ -315,14 +330,16 @@ private void testResponseWithErrorCodeAndUnknownOffsets(Errors error) throws Exe // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent to single broker - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res); assertFalse(fetchOffsetsFuture.isDone()); @@ -352,14 +369,16 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E partitionLeaders.put(TEST_PARTITION_1, LEADER_1); partitionLeaders.put(TEST_PARTITION_2, LEADER_2); mockSuccessfulRequest(partitionLeaders); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(2, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Requests successfully sent to both brokers - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res, 2); assertFalse(fetchOffsetsFuture.isDone()); @@ -386,7 +405,7 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E assertEquals(1, requestManager.requestsToSend()); // Following poll should send the request and get a successful response - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, @@ -405,15 +424,17 @@ public void testRequestFailedResponse_NonRetriableAuthError() { // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res); // Response received with non-retriable auth error @@ -434,15 +455,17 @@ public void testRequestFailedResponse_NonRetriableErrorTimeout() { // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res); // Response received @@ -466,15 +489,17 @@ public void testRequestFails_AuthenticationException() { // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( timestampsToSearch, - false); + false, + timer); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(res); // Response received with auth error @@ -491,17 +516,19 @@ public void testRequestFails_AuthenticationException() { @Test public void testResetPositionsSendNoRequestIfNoPartitionsNeedingReset() { - when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.emptySet()); - requestManager.resetPositionsIfNeeded(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.emptySet()); + requestManager.resetPositionsIfNeeded(timer); assertEquals(0, requestManager.requestsToSend()); } @Test public void testResetPositionsMissingLeader() { mockFailedRequest_MissingLeader(); - when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST); - requestManager.resetPositionsIfNeeded(); + requestManager.resetPositionsIfNeeded(timer); verify(metadata).requestUpdate(true); assertEquals(0, requestManager.requestsToSend()); } @@ -522,14 +549,15 @@ public void testResetPositionsSuccess_LeaderEpochInResponse() { @Test public void testResetOffsetsAuthorizationFailure() { - when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - requestManager.resetPositionsIfNeeded(); + requestManager.resetPositionsIfNeeded(timer); // Reset positions response with TopicAuthorizationException - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); Errors topicAuthorizationFailedError = Errors.TOPIC_AUTHORIZATION_FAILED; ClientResponse clientResponse = buildClientResponseWithErrors( @@ -544,7 +572,7 @@ public void testResetOffsetsAuthorizationFailure() { // Following resetPositions should enqueue the previous exception in the background event queue // without performing any request - assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded()); + assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded(timer)); assertEquals(0, requestManager.requestsToSend()); // Check that the event was enqueued during resetPositionsIfNeeded @@ -573,14 +601,15 @@ public void testValidatePositionsSuccess() { mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); - requestManager.validatePositionsIfNeeded(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + requestManager.validatePositionsIfNeeded(timer); assertEquals(1, requestManager.requestsToSend(), "Invalid request count"); verify(subscriptionState).setNextAllowedRetry(any(), anyLong()); // Validate positions response with end offsets when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(LEADER_1, leaderAndEpoch.epoch)); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); NetworkClientDelegate.UnsentRequest unsentRequest = pollResult.unsentRequests.get(0); ClientResponse clientResponse = buildOffsetsForLeaderEpochResponse(unsentRequest, Collections.singletonList(tp), expectedEndOffset); @@ -592,15 +621,16 @@ public void testValidatePositionsSuccess() { @Test public void testValidatePositionsMissingLeader() { + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(Node.noNode()), Optional.of(5)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(5L, Optional.of(10), leaderAndEpoch); - when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); NodeApiVersions nodeApiVersions = NodeApiVersions.create(); when(apiVersions.get(LEADER_1.idString())).thenReturn(nodeApiVersions); - requestManager.validatePositionsIfNeeded(); + requestManager.validatePositionsIfNeeded(timer); verify(metadata).requestUpdate(true); assertEquals(0, requestManager.requestsToSend()); } @@ -613,10 +643,11 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { Optional.of(10), leaderAndEpoch); mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); - requestManager.validatePositionsIfNeeded(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + requestManager.validatePositionsIfNeeded(timer); // Validate positions response with TopicAuthorizationException - NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildOffsetsForLeaderEpochResponseWithErrors(unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); @@ -627,7 +658,7 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { // Following validatePositions should raise the previous exception without performing any // request - assertThrows(TopicAuthorizationException.class, () -> requestManager.validatePositionsIfNeeded()); + assertThrows(TopicAuthorizationException.class, () -> requestManager.validatePositionsIfNeeded(timer)); assertEquals(0, requestManager.requestsToSend()); } @@ -639,26 +670,28 @@ public void testValidatePositionsAbortIfNoApiVersionsToCheckAgainstThenRecovers( SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(currentOffset, Optional.of(10), leaderAndEpoch); - when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); // No api version info initially available when(apiVersions.get(LEADER_1.idString())).thenReturn(null); - requestManager.validatePositionsIfNeeded(); + requestManager.validatePositionsIfNeeded(timer); assertEquals(0, requestManager.requestsToSend(), "Invalid request count"); verify(subscriptionState, never()).completeValidation(TEST_PARTITION_1); verify(subscriptionState, never()).setNextAllowedRetry(any(), anyLong()); // Api version updated, next validate positions should successfully build the request when(apiVersions.get(LEADER_1.idString())).thenReturn(NodeApiVersions.create()); - when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); - requestManager.validatePositionsIfNeeded(); + requestManager.validatePositionsIfNeeded(timer); assertEquals(1, requestManager.requestsToSend(), "Invalid request count"); } private void mockSuccessfulBuildRequestForValidatingPositions(SubscriptionState.FetchPosition position, Node leader) { - when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); NodeApiVersions nodeApiVersions = NodeApiVersions.create(); when(apiVersions.get(leader.idString())).thenReturn(nodeApiVersions); @@ -671,16 +704,17 @@ private void testResetPositionsSuccessWithLeaderEpoch(Metadata.LeaderAndEpoch le long offset = 5L; Map expectedOffsets = Collections.singletonMap(tp, new OffsetAndTimestamp(offset, 1L, leaderAndEpoch.epoch)); - when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(tp)); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(tp)); when(subscriptionState.resetStrategy(any())).thenReturn(strategy); mockSuccessfulRequest(Collections.singletonMap(tp, leader)); - requestManager.resetPositionsIfNeeded(); + requestManager.resetPositionsIfNeeded(timer); assertEquals(1, requestManager.requestsToSend()); // Reset positions response with offsets when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(leader, leaderAndEpoch.epoch)); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); NetworkClientDelegate.UnsentRequest unsentRequest = pollResult.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, expectedOffsets); clientResponse.onComplete(); @@ -715,10 +749,10 @@ private static Stream retriableErrors() { private void verifySuccessfulPollAndResponseReceived( CompletableFuture> actualResult, - Map expectedResult) throws ExecutionException, + Map expectedResult, Timer timer) throws ExecutionException, InterruptedException { // Following poll should send the request and get a response - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, expectedResult); @@ -856,8 +890,8 @@ private ClientResponse buildOffsetsForLeaderEpochResponse( new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), false, null, null, @@ -889,8 +923,8 @@ private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), false, null, null, @@ -938,8 +972,8 @@ private ClientResponse buildClientResponse( new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), disconnected, null, authenticationException, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index f759770036389..e2013bfa1e511 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -35,6 +36,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -62,6 +64,7 @@ public class TopicMetadataRequestManagerTest { private MockTime time; private TopicMetadataRequestManager topicMetadataRequestManager; + private int defaultApiTimeoutMs; @BeforeEach public void setup() { @@ -71,26 +74,29 @@ public void setup() { props.put(ALLOW_AUTO_CREATE_TOPICS_CONFIG, false); props.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + ConsumerConfig config = new ConsumerConfig(props); this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager( new LogContext(), - new ConsumerConfig(props), - time)); + config)); + this.defaultApiTimeoutMs = config.getInt(CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG); } @Test public void testPoll_SuccessfulRequestTopicMetadata() { String topic = "hello"; - this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); - this.time.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); + timer.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); } @Test public void testPoll_SuccessfulRequestAllTopicsMetadata() { - this.topicMetadataRequestManager.requestAllTopicsMetadata(Long.MAX_VALUE); - this.time.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + this.topicMetadataRequestManager.requestAllTopicsMetadata(timer); + timer.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); } @@ -98,13 +104,15 @@ public void testPoll_SuccessfulRequestAllTopicsMetadata() { @MethodSource("exceptionProvider") public void testTopicExceptionAndInflightRequests(final Errors error, final boolean shouldRetry) { String topic = "hello"; - this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); - this.time.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); + timer.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - error)); + error, + timer)); List inflights = this.topicMetadataRequestManager.inflightRequests(); if (shouldRetry) { @@ -118,12 +126,14 @@ public void testTopicExceptionAndInflightRequests(final Errors error, final bool @ParameterizedTest @MethodSource("exceptionProvider") public void testAllTopicsExceptionAndInflightRequests(final Errors error, final boolean shouldRetry) { - this.topicMetadataRequestManager.requestAllTopicsMetadata(Long.MAX_VALUE); - this.time.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + this.topicMetadataRequestManager.requestAllTopicsMetadata(timer); + timer.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); res.unsentRequests.get(0).future().complete(buildAllTopicsMetadataClientResponse( res.unsentRequests.get(0), - error)); + error, + timer)); List inflights = this.topicMetadataRequestManager.inflightRequests(); if (shouldRetry) { @@ -138,33 +148,35 @@ public void testExpiringRequest() { String topic = "hello"; // Request topic metadata with 1000ms expiration - long now = this.time.milliseconds(); + Timer timer = time.timer(1000); CompletableFuture>> future = - this.topicMetadataRequestManager.requestTopicMetadata(topic, now + 1000L); + this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); assertEquals(1, this.topicMetadataRequestManager.inflightRequests().size()); // Poll the request manager to get the list of requests to send // - fail the request with a RetriableException - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - Errors.REQUEST_TIMED_OUT)); + Errors.REQUEST_TIMED_OUT, + timer)); // Sleep for long enough to exceed the backoff delay but still within the expiration // - fail the request again with a RetriableException - this.time.sleep(500); - res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + timer.sleep(500); + res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - Errors.REQUEST_TIMED_OUT)); + Errors.REQUEST_TIMED_OUT, + timer)); // Sleep for long enough to expire the request which should fail - this.time.sleep(1000); - res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + timer.sleep(1000); + res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res.unsentRequests.size()); assertEquals(0, this.topicMetadataRequestManager.inflightRequests().size()); assertTrue(future.isCompletedExceptionally()); @@ -174,9 +186,9 @@ public void testExpiringRequest() { @MethodSource("hardFailureExceptionProvider") public void testHardFailures(Exception exception) { String topic = "hello"; - - this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().completeExceptionally(exception); @@ -191,37 +203,39 @@ public void testHardFailures(Exception exception) { @Test public void testNetworkTimeout() { String topic = "hello"; - - topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + Timer timer = time.timer(defaultApiTimeoutMs); + topicMetadataRequestManager.requestTopicMetadata(topic, timer); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res.unsentRequests.size()); - NetworkClientDelegate.PollResult res2 = this.topicMetadataRequestManager.poll(this.time.milliseconds()); + NetworkClientDelegate.PollResult res2 = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res2.unsentRequests.size()); // Mimic a network timeout - res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); + res.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException()); - long backoffMs = topicMetadataRequestManager.inflightRequests().get(0).remainingBackoffMs(time.milliseconds()); + long backoffMs = topicMetadataRequestManager.inflightRequests().get(0).remainingBackoffMs(timer.currentTimeMs()); // Sleep for exponential backoff - 1ms - time.sleep(backoffMs - 1); - res2 = topicMetadataRequestManager.poll(this.time.milliseconds()); + timer.sleep(backoffMs - 1); + res2 = topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(0, res2.unsentRequests.size()); - time.sleep(1); - res2 = topicMetadataRequestManager.poll(this.time.milliseconds()); + timer.sleep(1); + res2 = topicMetadataRequestManager.poll(timer.currentTimeMs()); assertEquals(1, res2.unsentRequests.size()); res2.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res2.unsentRequests.get(0), topic, - Errors.NONE)); + Errors.NONE, + timer)); assertTrue(topicMetadataRequestManager.inflightRequests().isEmpty()); } private ClientResponse buildTopicMetadataClientResponse( final NetworkClientDelegate.UnsentRequest request, final String topic, - final Errors error) { + final Errors error, + final Timer timer) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof MetadataRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; @@ -237,8 +251,8 @@ private ClientResponse buildTopicMetadataClientResponse( new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + timer.currentTimeMs(), + timer.currentTimeMs(), false, null, null, @@ -247,7 +261,8 @@ private ClientResponse buildTopicMetadataClientResponse( private ClientResponse buildAllTopicsMetadataClientResponse( final NetworkClientDelegate.UnsentRequest request, - final Errors error) { + final Errors error, + final Timer timer) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof MetadataRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; @@ -265,8 +280,8 @@ private ClientResponse buildAllTopicsMetadataClientResponse( new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), request.handler(), "-1", - time.milliseconds(), - time.milliseconds(), + timer.currentTimeMs(), + timer.currentTimeMs(), false, null, null, From 338440e86d76c952e8a43b3fc1ce6e6665ba2a50 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 13:30:06 -0800 Subject: [PATCH 015/130] Update HeartbeatRequestManagerTest.java --- .../HeartbeatRequestManagerTest.java | 126 ++++++++++-------- 1 file changed, 72 insertions(+), 54 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index e44414b0677b9..5b91075ca3d00 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -78,6 +78,7 @@ public class HeartbeatRequestManagerTest { private int heartbeatIntervalMs = DEFAULT_HEARTBEAT_INTERVAL_MS; private int maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL_MS; private long retryBackoffMaxMs = DEFAULT_RETRY_BACKOFF_MAX_MS; + private final int DEFAULT_API_TIMEOUT_MS = 60000; private static final String DEFAULT_GROUP_ID = "groupId"; private ConsumerTestBuilder testBuilder; @@ -141,17 +142,18 @@ public void cleanup() { @Test public void testHeartbeatOnStartup() { - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(0, result.unsentRequests.size()); resetWithZeroHeartbeatInterval(Optional.empty()); mockStableMember(); - assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, heartbeatRequestManager.maximumTimeToWait(timer.currentTimeMs())); + result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); // Ensure we do not resend the request without the first request being completed - NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(0, result2.unsentRequests.size()); } @@ -163,9 +165,11 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); membershipManager.onSubscriptionUpdated(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + // Create a ConsumerHeartbeatRequest and verify the payload - assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, heartbeatRequestManager.maximumTimeToWait(timer.currentTimeMs())); + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); @@ -194,7 +198,8 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); if (!shouldSkipHeartbeat) { assertEquals(1, result.unsentRequests.size()); @@ -209,16 +214,17 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { @Test public void testTimerNotDue() { mockStableMember(); - time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + timer.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(timer.currentTimeMs())); // Member in state where it should not send Heartbeat anymore when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); membershipManager.transitionToFatal(); - result = heartbeatRequestManager.poll(time.milliseconds()); + result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); } @@ -226,13 +232,14 @@ public void testTimerNotDue() { public void testHeartbeatOutsideInterval() { when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); // Heartbeat should be sent assertEquals(1, result.unsentRequests.size()); // Interval timer reset assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, result.timeUntilNextPollMs); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(timer.currentTimeMs())); // Membership manager updated (to transition out of the heartbeating state) verify(membershipManager).onHeartbeatRequestSent(); } @@ -243,18 +250,19 @@ public void testNetworkTimeout() { resetWithZeroHeartbeatInterval(Optional.empty()); mockStableMember(); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout - result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout")); + result.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException("timeout")); // Assure the manager will backoff on timeout - time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); - result = heartbeatRequestManager.poll(time.milliseconds()); + timer.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); + result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(0, result.unsentRequests.size()); - time.sleep(1); - result = heartbeatRequestManager.poll(time.milliseconds()); + timer.sleep(1); + result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); } @@ -265,9 +273,10 @@ public void testFailureOnFatalException() { mockStableMember(); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); - result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new KafkaException("fatal")); + result.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new KafkaException("fatal")); verify(membershipManager).transitionToFatal(); verify(backgroundEventHandler).add(any()); } @@ -275,10 +284,11 @@ public void testFailureOnFatalException() { @Test public void testNoCoordinator() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(timer.currentTimeMs())); assertEquals(0, result.unsentRequests.size()); } @@ -300,7 +310,8 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatResponseReceived(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload - NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); @@ -331,8 +342,9 @@ public void testConsumerGroupMetadataFirstUpdate() { public void testConsumerGroupMetadataUpdateWithSameUpdate() { makeFirstGroupMetadataUpdate(memberId, memberEpoch); - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + timer.sleep(2000); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); @@ -345,8 +357,9 @@ public void testConsumerGroupMetadataUpdateWithSameUpdate() { public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated() { makeFirstGroupMetadataUpdate(memberId, memberEpoch); - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + timer.sleep(2000); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); @@ -372,8 +385,9 @@ public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame() { makeFirstGroupMetadataUpdate(memberId, memberEpoch); - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + timer.sleep(2000); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); @@ -400,7 +414,8 @@ private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String membe resetWithZeroHeartbeatInterval(Optional.empty()); mockStableMember(); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); ClientResponse firstResponse = createHeartbeatResponse(request, Errors.NONE, memberId, memberEpoch); @@ -416,9 +431,11 @@ private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String membe public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { mockStableMember(); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + // Handling errors on the second heartbeat - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + timer.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, result.unsentRequests.size()); // Manually completing the response to test error handling @@ -433,28 +450,28 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole case NONE: verify(backgroundEventHandler).add(any(GroupMetadataUpdateEvent.class)); verify(membershipManager, times(2)).onHeartbeatResponseReceived(mockResponse.data()); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(timer.currentTimeMs())); break; case COORDINATOR_LOAD_IN_PROGRESS: verify(backgroundEventHandler, never()).add(any()); - assertEquals(DEFAULT_RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + assertEquals(DEFAULT_RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(timer.currentTimeMs())); break; case COORDINATOR_NOT_AVAILABLE: case NOT_COORDINATOR: verify(backgroundEventHandler, never()).add(any()); verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong()); - assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + assertEquals(0, heartbeatRequestState.nextHeartbeatMs(timer.currentTimeMs())); break; default: if (isFatal) { // The memberStateManager should have stopped heartbeat at this point - ensureFatalError(); + ensureFatalError(timer); } else { verify(backgroundEventHandler, never()).add(any()); - assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + assertEquals(0, heartbeatRequestState.nextHeartbeatMs(timer.currentTimeMs())); } break; } @@ -549,29 +566,30 @@ public void testPollTimerExpiration() { when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.state()).thenReturn(MemberState.STABLE); - time.sleep(maxPollIntervalMs); - NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); + timer.sleep(maxPollIntervalMs); + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(1, pollResult.unsentRequests.size()); verify(heartbeatState).reset(); verify(heartbeatRequestState).reset(); verify(membershipManager).transitionToStale(); - assertNoHeartbeat(heartbeatRequestManager); - heartbeatRequestManager.resetPollTimer(time.milliseconds()); + assertNoHeartbeat(heartbeatRequestManager, timer); + heartbeatRequestManager.resetPollTimer(timer.currentTimeMs()); assertTrue(pollTimer.notExpired()); - assertHeartbeat(heartbeatRequestManager); + assertHeartbeat(heartbeatRequestManager, timer); } - private void assertHeartbeat(HeartbeatRequestManager hrm) { - NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); + private void assertHeartbeat(HeartbeatRequestManager hrm, Timer timer) { + NetworkClientDelegate.PollResult pollResult = hrm.poll(timer.currentTimeMs()); assertEquals(1, pollResult.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, pollResult.timeUntilNextPollMs); pollResult.unsentRequests.get(0).handler().onComplete(createHeartbeatResponse(pollResult.unsentRequests.get(0), Errors.NONE)); } - private void assertNoHeartbeat(HeartbeatRequestManager hrm) { - NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); + private void assertNoHeartbeat(HeartbeatRequestManager hrm, Timer timer) { + NetworkClientDelegate.PollResult pollResult = hrm.poll(timer.currentTimeMs()); assertEquals(0, pollResult.unsentRequests.size()); } @@ -586,16 +604,16 @@ private void mockStableMember() { assertEquals(MemberState.STABLE, membershipManager.state()); } - private void ensureFatalError() { + private void ensureFatalError(Timer timer) { verify(membershipManager).transitionToFatal(); verify(backgroundEventHandler).add(any()); - ensureHeartbeatStopped(); + ensureHeartbeatStopped(timer); } - private void ensureHeartbeatStopped() { - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + private void ensureHeartbeatStopped(Timer timer) { + timer.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(MemberState.FATAL, membershipManager.state()); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(timer.currentTimeMs()); assertEquals(0, result.unsentRequests.size()); } @@ -650,8 +668,8 @@ private ClientResponse createHeartbeatResponse( new RequestHeader(ApiKeys.CONSUMER_GROUP_HEARTBEAT, ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), "client-id", 1), request.handler(), "0", - time.milliseconds(), - time.milliseconds(), + request.timer().currentTimeMs(), + request.timer().currentTimeMs(), false, null, null, From 7ee977b57870382f49d7ddb841e14ab0c293f64a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 15:28:51 -0800 Subject: [PATCH 016/130] Updates to fix some tests --- .../clients/consumer/internals/MembershipManagerImpl.java | 2 +- .../internals/events/CompletableApplicationEvent.java | 5 +++++ .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 6 +----- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 44a2c3e39216c..f46e801f1f1ce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -598,7 +598,7 @@ public CompletableFuture leaveGroup(Timer timer) { if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) { // Member already leaving. No-op and return existing leave group future that will // complete when the ongoing leave operation completes. - return leaveGroupInProgress.get(); + return leaveGroupInProgress.orElseThrow(() -> new IllegalStateException("Member already leaving, but no Future available")); } transitionTo(MemberState.PREPARE_LEAVING); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index f76485da28a49..64423b4578466 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -56,6 +56,11 @@ public T get() { } public void chain(final CompletableFuture providedFuture) { + Objects.requireNonNull( + providedFuture, + () -> String.format("Could not chain the event's future (%s) to the provided future because the provided future was null", this.future) + ); + providedFuture.whenComplete((value, exception) -> { if (exception != null) { this.future.completeExceptionally(exception); 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 cb8cf9479acae..191a286f3c978 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 @@ -829,10 +829,9 @@ public void testWakeupCommitted() { final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future(), timer); + return event.get(); }) .when(applicationEventHandler) .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); @@ -1320,7 +1319,6 @@ public void testLongPollWaitIsLimited() { @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { consumer = newConsumer(); - Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); CountDownLatch latch = new CountDownLatch(3); @@ -1356,7 +1354,6 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { @Test public void testProcessBackgroundEventsWithoutDelay() { consumer = newConsumer(); - Time time = new MockTime(); Timer timer = time.timer(1000); // Create a future that is already completed. @@ -1378,7 +1375,6 @@ public void testProcessBackgroundEventsWithoutDelay() { @Test public void testProcessBackgroundEventsTimesOut() throws Exception { consumer = newConsumer(); - Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); From eff925d7e318f36581df33b76a7523b4e81dbbf3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 15:39:26 -0800 Subject: [PATCH 017/130] Updates --- .../consumer/internals/ConsumerUtils.java | 14 ----------- .../internals/AsyncKafkaConsumerTest.java | 25 +++++++++++++------ .../events/ApplicationEventProcessorTest.java | 9 ------- 3 files changed, 18 insertions(+), 30 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 422b0cc504410..0bd9b5f565c9c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -242,18 +242,4 @@ public static KafkaException maybeWrapAsKafkaException(Throwable t, String messa else return new KafkaException(message, t); } - - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - public static long getExpirationTimeForTimeout(final long timeoutMs) { - // TODO: should use Time.milliseconds(), not System.currentTimeMillis() - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; - } } 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 191a286f3c978..167b18c99ab8f 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 @@ -153,10 +153,14 @@ public void resetAll() { } private AsyncKafkaConsumer newConsumer() { + return newConsumer(time); + } + + private AsyncKafkaConsumer newConsumer(Time time) { final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); final ConsumerConfig config = new ConsumerConfig(props); - return newConsumer(config); + return newConsumer(config, time); } private AsyncKafkaConsumer newConsumerWithoutGroupId() { @@ -172,7 +176,7 @@ private AsyncKafkaConsumer newConsumerWithEmptyGroupId() { return newConsumer(config); } - private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { + private AsyncKafkaConsumer newConsumer(ConsumerConfig config, Time time) { return new AsyncKafkaConsumer<>( config, new StringDeserializer(), @@ -185,6 +189,10 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { ); } + private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { + return newConsumer(config, time); + } + private AsyncKafkaConsumer newConsumer( FetchBuffer fetchBuffer, ConsumerInterceptors interceptors, @@ -825,7 +833,8 @@ public void testOffsetsForTimesWithZeroTimeout() { @Test public void testWakeupCommitted() { - consumer = newConsumer(); + Time time = new MockTime(); + consumer = newConsumer(time); final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); @@ -1034,7 +1043,6 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); - Timer timer = time.timer(defaultApiTimeoutMs); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( @@ -1318,7 +1326,8 @@ public void testLongPollWaitIsLimited() { */ @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { - consumer = newConsumer(); + Time time = new MockTime(); + consumer = newConsumer(time); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); CountDownLatch latch = new CountDownLatch(3); @@ -1353,7 +1362,8 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { */ @Test public void testProcessBackgroundEventsWithoutDelay() { - consumer = newConsumer(); + Time time = new MockTime(); + consumer = newConsumer(time); Timer timer = time.timer(1000); // Create a future that is already completed. @@ -1374,7 +1384,8 @@ public void testProcessBackgroundEventsWithoutDelay() { */ @Test public void testProcessBackgroundEventsTimesOut() throws Exception { - consumer = newConsumer(); + Time time = new MockTime(); + consumer = newConsumer(time); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index cad3f06d5b00b..9276b91479988 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -101,15 +101,6 @@ public void testPrepClosingCommitEvents() { verify(commitRequestManager).signalClose(); } - @Test - public void testExpirationCalculation() { - assertEquals(Long.MAX_VALUE, ConsumerUtils.getExpirationTimeForTimeout(Long.MAX_VALUE)); - assertEquals(Long.MAX_VALUE, ConsumerUtils.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); - long timeout = ConsumerUtils.getExpirationTimeForTimeout(1000); - assertTrue(timeout > 0); - assertTrue(timeout < Long.MAX_VALUE); - } - @Test public void testPrepClosingLeaveGroupEvent() { Timer timer = time.timer(Long.MAX_VALUE); From d1dfdc644d0d4e8a1afee2e981648d2c636582b5 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 15:49:10 -0800 Subject: [PATCH 018/130] Update ApplicationEventProcessorTest.java --- .../internals/events/ApplicationEventProcessorTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 9276b91479988..bc04c958fd91f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.CoordinatorRequestManager; import org.apache.kafka.clients.consumer.internals.FetchRequestManager; import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager; @@ -40,8 +39,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -106,9 +105,9 @@ public void testPrepClosingLeaveGroupEvent() { Timer timer = time.timer(Long.MAX_VALUE); LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); - when(membershipManager.leaveGroup(timer)).thenReturn(CompletableFuture.completedFuture(null)); + when(membershipManager.leaveGroup(any())).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); - verify(membershipManager).leaveGroup(timer); + verify(membershipManager).leaveGroup(any()); assertTrue(event.future().isDone()); } From 79880a42358c39e981054ebf39b72fa4d98d70eb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:35:11 -0800 Subject: [PATCH 019/130] Updates --- .../internals/AsyncKafkaConsumer.java | 6 ++--- .../internals/CommitRequestManager.java | 24 +++++++++++++------ .../internals/CoordinatorRequestManager.java | 4 ++-- .../internals/FetchRequestManager.java | 8 +++---- .../internals/HeartbeatRequestManager.java | 13 ++++------ .../internals/MembershipManagerImpl.java | 16 +++++-------- .../internals/OffsetsRequestManager.java | 15 ++++++------ .../consumer/internals/RequestManagers.java | 8 +++---- 8 files changed, 48 insertions(+), 46 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 3c5a138765812..dee5387f3911d 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 @@ -1017,7 +1017,7 @@ public Map> listTopics(Duration timeout) { final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timer); + new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); @@ -1465,7 +1465,7 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.isPresent()) { - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(defaultApiTimeoutMs); UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(timer); applicationEventHandler.add(unsubscribeApplicationEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); @@ -1976,4 +1976,4 @@ public void invoke() { callback.onComplete(offsets, exception); } } -} \ No newline at end of file +} 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 6dd9423f0d34b..fbe41ec023569 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 @@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.DisconnectException; @@ -72,7 +71,6 @@ import static org.apache.kafka.common.protocol.Errors.COORDINATOR_LOAD_IN_PROGRESS; public class CommitRequestManager implements RequestManager, MemberStateListener { - private final Time time; private final SubscriptionState subscriptions; private final LogContext logContext; @@ -311,7 +309,7 @@ public CompletableFuture maybeAutoCommitAllConsumedNow( } /** - * Handles {@link CommitApplicationEvent}. It creates an + * Handles {@link org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent}. It creates an * {@link OffsetCommitRequestState} and enqueue it to send later. */ public CompletableFuture addOffsetCommitRequest(final Map offsets, @@ -424,6 +422,11 @@ private class OffsetCommitRequestState extends RetriableRequestState { private final CompletableFuture future; + /** + * Timer that determines the time until which the request should be retried if it fails with retriable + * errors. If not present, the request is triggered without waiting for a response or + * retrying. + */ private final Optional timer; OffsetCommitRequestState(final Map offsets, @@ -496,7 +499,7 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data); - // Timer will be "empty" for async/auto-commit, so create a dummy timer using default.api.timeout.ms. + // Timer will be "empty" for async/auto-commit, so use default.api.timeout.ms for the timeout. Timer requestTimer = timer.orElseGet(() -> time.timer(defaultApiTimeoutMs)); NetworkClientDelegate.UnsentRequest resp = new NetworkClientDelegate.UnsentRequest( builder, @@ -604,7 +607,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { future.completeExceptionally(commitExceptionForRetriableError(throwable)); return; } - if (timer.isPresent() && timer.get().isExpired()) { + if (isExpired()) { // Fail requests that allowed retries (sync requests), but expired. future.completeExceptionally(throwable); return; @@ -723,6 +726,9 @@ class OffsetFetchRequestState extends RetriableRequestState { private final CompletableFuture> future; + /** + * Used to determine the time until which the request should be retried if it fails with retriable errors. + */ private final Timer timer; public OffsetFetchRequestState(final Set partitions, @@ -838,7 +844,7 @@ private void onFailure(final long currentTimeMs, */ @Override void maybeRetry(long currentTimeMs, Throwable throwable) { - if (timer.isExpired()) { + if (isExpired()) { future.completeExceptionally(throwable); return; } @@ -847,6 +853,10 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { pendingRequests.addOffsetFetchRequest(this); } + private boolean isExpired() { + return timer.isExpired(); + } + /** * Complete the request future with a TimeoutException if the request expired. No action * taken if the request is still active. @@ -854,7 +864,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) { * @return True if the request expired. */ private boolean maybeExpire() { - if (timer.isExpired()) { + if (isExpired()) { future.completeExceptionally(new TimeoutException("OffsetFetch request could not " + "complete before timeout expired.")); return true; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index 6b0c335b6b28a..d21404555ae70 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -69,14 +69,14 @@ public CoordinatorRequestManager( final long defaultApiTimeoutMs, final long retryBackoffMs, final long retryBackoffMaxMs, - final BackgroundEventHandler backgroundEventHandler, + final BackgroundEventHandler errorHandler, final String groupId ) { Objects.requireNonNull(groupId); this.time = time; this.log = logContext.logger(this.getClass()); this.defaultApiTimeoutMs = defaultApiTimeoutMs; - this.backgroundEventHandler = backgroundEventHandler; + this.backgroundEventHandler = errorHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index 50e835978b99a..16fa012a153d9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -42,7 +42,7 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager { private final NetworkClientDelegate networkClientDelegate; - private final long requestTimeoutMs; + private final int defaultApiTimeoutMs; FetchRequestManager(final LogContext logContext, final Time time, @@ -53,10 +53,10 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager final FetchMetricsManager metricsManager, final NetworkClientDelegate networkClientDelegate, final ApiVersions apiVersions, - final int requestTimeoutMs) { + final int defaultApiTimeoutMs) { super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time, apiVersions); this.networkClientDelegate = networkClientDelegate; - this.requestTimeoutMs = requestTimeoutMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; } @Override @@ -106,7 +106,7 @@ public PollResult pollOnClose() { private PollResult pollInternal(Map fetchRequests, ResponseHandler successHandler, ResponseHandler errorHandler) { - Timer timer = time.timer(requestTimeoutMs); + Timer timer = time.timer(defaultApiTimeoutMs); List requests = fetchRequests.entrySet().stream().map(entry -> { final Node fetchTarget = entry.getKey(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index 55b2ca1e7c3ac..9df2b0e60f078 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -251,16 +251,14 @@ private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final long curr } private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final boolean ignoreResponse) { - Optional coordinator = coordinatorRequestManager.coordinator(); - ConsumerGroupHeartbeatRequest.Builder requestBuilder = new ConsumerGroupHeartbeatRequest.Builder(heartbeatState.buildRequestData()); NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( - requestBuilder, - coordinator, - pollTimer + new ConsumerGroupHeartbeatRequest.Builder(heartbeatState.buildRequestData()), + coordinatorRequestManager.coordinator(), + pollTimer ); - if (ignoreResponse) { + if (ignoreResponse) return logResponse(request); - } else { + else return request.whenComplete((response, exception) -> { if (response != null) { onResponse((ConsumerGroupHeartbeatResponse) response.responseBody(), request.handler().completionTimeMs()); @@ -268,7 +266,6 @@ private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final boolean i onFailure(exception, request.handler().completionTimeMs()); } }); - } } private NetworkClientDelegate.UnsentRequest logResponse(final NetworkClientDelegate.UnsentRequest request) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index f46e801f1f1ce..e5f1bb764a196 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -191,11 +191,6 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private final Logger log; - /** - * Time instance used to create {@link Timer timers}. - */ - private final Time time; - /** * Manager to perform commit requests needed before revoking partitions (if auto-commit is * enabled) @@ -282,6 +277,8 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private final BackgroundEventHandler backgroundEventHandler; + private final Time time; + public MembershipManagerImpl(String groupId, Optional groupInstanceId, int rebalanceTimeoutMs, @@ -842,7 +839,9 @@ boolean reconcile() { // the current reconciliation is in process. Note this is using the rebalance timeout as // it is the limit enforced by the broker to complete the reconciliation process. Timer timer = time.timer(rebalanceTimeoutMs); - commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow(Optional.of(timer), true); + commitResult = commitRequestManager.maybeAutoCommitAllConsumedNow( + Optional.of(time.timer(rebalanceTimeoutMs)), + true); // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. commitResult.whenComplete((commitReqResult, commitReqError) -> { @@ -1211,10 +1210,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent( - methodName, - sortedPartitions - ); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); 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 b1c2766be51db..59127389a0dad 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 @@ -82,7 +82,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis private final Set requestsToRetry; private final List requestsToSend; - private final long requestTimeoutMs; + private final long defaultApiTimeoutMs; private final Time time; private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; @@ -94,7 +94,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, - final long requestTimeoutMs, + final long defaultApiTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final BackgroundEventHandler backgroundEventHandler, @@ -115,7 +115,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; this.time = time; - this.requestTimeoutMs = requestTimeoutMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; @@ -262,7 +262,7 @@ public void onUpdate(ClusterResource clusterResource) { // fetchOffsetsByTimes call if any of the requests being retried fails List requestsToProcess = new ArrayList<>(requestsToRetry); requestsToRetry.clear(); - Timer timer = time.timer(requestTimeoutMs); + Timer timer = time.timer(defaultApiTimeoutMs); requestsToProcess.forEach(requestState -> { Map timestampsToSearch = new HashMap<>(requestState.remainingToSearch); @@ -403,10 +403,9 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); - + final long nextResetTimeMs = timer.currentTimeMs() + timer.remainingMs(); timestampsToSearchByNode.forEach((node, resetTimestamps) -> { - subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), - time.milliseconds() + requestTimeoutMs); + subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), nextResetTimeMs); CompletableFuture partialResult = buildListOffsetRequestToNode( node, @@ -465,7 +464,7 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi final Map> regrouped = regroupFetchPositionsByLeader(partitionsToValidate); - long nextResetTimeMs = time.milliseconds() + timer.remainingMs(); + long nextResetTimeMs = timer.currentTimeMs() + timer.remainingMs(); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); 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 022c03b2ae523..3f08274fd0f7a 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 @@ -125,13 +125,13 @@ protected RequestManagers create() { final FetchConfig fetchConfig = new FetchConfig(config); long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); - final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final int defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions, metadata, fetchConfig.isolationLevel, time, retryBackoffMs, - requestTimeoutMs, + defaultApiTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, @@ -145,7 +145,7 @@ protected RequestManagers create() { fetchMetricsManager, networkClientDelegate, apiVersions, - requestTimeoutMs); + defaultApiTimeoutMs); final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(logContext, config); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; @@ -156,7 +156,7 @@ protected RequestManagers create() { Optional serverAssignor = Optional.ofNullable(config.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); coordinator = new CoordinatorRequestManager(time, logContext, - requestTimeoutMs, + defaultApiTimeoutMs, retryBackoffMs, retryBackoffMaxMs, backgroundEventHandler, From d96e75baa33081167dda36cbb4c7fb5060227d90 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:39:23 -0800 Subject: [PATCH 020/130] Update TopicMetadataApplicationEvent.java --- .../internals/events/TopicMetadataApplicationEvent.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index 8f02bf159492e..8a71a6ea646c0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -62,8 +62,8 @@ public boolean equals(Object o) { TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - // TODO: fix this potential NPE - return topic.equals(that.topic) && (allTopics == that.allTopics); + // We use this utility method to keep this logic succinct because topic and/or that.topic can be null. + return Objects.equals(topic, that.topic) && (allTopics == that.allTopics); } @Override From 1ee795e8919bb2d95feb0243aff77034210c5f2f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:41:45 -0800 Subject: [PATCH 021/130] Updates --- .../consumer/internals/events/ErrorBackgroundEvent.java | 1 - .../internals/events/ListOffsetsApplicationEvent.java | 4 +--- .../internals/events/TopicMetadataApplicationEvent.java | 3 +-- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 587532dfc292a..26edea043e77b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -26,7 +26,6 @@ public class ErrorBackgroundEvent extends BackgroundEvent { public ErrorBackgroundEvent(Throwable t) { super(Type.ERROR); - Objects.requireNonNull(t); this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index e8daeb7d1497a..1baf966c2ecc9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -37,9 +37,7 @@ public class ListOffsetsApplicationEvent extends CompletableApplicationEvent timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, - boolean requireTimestamps, - Timer timer) { + public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps, Timer timer) { super(Type.LIST_OFFSETS, timer); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index 8a71a6ea646c0..1e72f5bb3c9bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -62,8 +62,7 @@ public boolean equals(Object o) { TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - // We use this utility method to keep this logic succinct because topic and/or that.topic can be null. - return Objects.equals(topic, that.topic) && (allTopics == that.allTopics); + return topic.equals(that.topic) && (allTopics == that.allTopics); } @Override From a685b4e0172b4236dbe3893f4043998edb57b7c2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:42:24 -0800 Subject: [PATCH 022/130] Update ErrorBackgroundEvent.java --- .../clients/consumer/internals/events/ErrorBackgroundEvent.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 26edea043e77b..2945f22986b18 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -18,8 +18,6 @@ import org.apache.kafka.common.KafkaException; -import java.util.Objects; - public class ErrorBackgroundEvent extends BackgroundEvent { private final RuntimeException error; From dcdb7676ec24cf357a2a51490c8ef0ddd5efa2b6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:45:15 -0800 Subject: [PATCH 023/130] Updates --- .../internals/events/ApplicationEventProcessor.java | 4 ++-- .../internals/events/CompletableApplicationEvent.java | 1 - .../consumer/internals/events/CompletableEvent.java | 2 -- .../ConsumerRebalanceListenerCallbackCompletedEvent.java | 8 +------- 4 files changed, 3 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index bfdf2bf38873e..b87615f45909f 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 @@ -346,11 +346,11 @@ private void process(final LeaveOnCloseApplicationEvent event) { * Creates a {@link Timer time} for the network I/O thread that is separate from the timer for the * application thread. */ - private Timer timer(CompletableEvent event) { + private Timer timer(CompletableApplicationEvent event) { return time.timer(event.deadlineMs() - time.milliseconds()); } - private boolean maybeTimeout(CompletableEvent event, Timer timer, String timeoutMessage) { + private boolean maybeTimeout(CompletableApplicationEvent event, Timer timer, String timeoutMessage) { if (timer.isExpired()) { Exception exception = new TimeoutException(timeoutMessage); event.future().completeExceptionally(exception); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 64423b4578466..b29bd87104a3a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -46,7 +46,6 @@ public CompletableFuture future() { return future; } - @Override public long deadlineMs() { return deadlineMs; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 56ddd07219913..97559d8cb9be2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,6 +21,4 @@ public interface CompletableEvent { CompletableFuture future(); - - long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index a5b43ba500270..b260c6154ea5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -28,7 +28,7 @@ * Event that signifies that the application thread has executed the {@link ConsumerRebalanceListener} callback. If * the callback execution threw an error, it is included in the event should any event listener want to know. */ -public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent implements CompletableEvent { +public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent { private final ConsumerRebalanceListenerMethodName methodName; private final CompletableFuture future; @@ -47,16 +47,10 @@ public ConsumerRebalanceListenerMethodName methodName() { return methodName; } - @Override public CompletableFuture future() { return future; } - @Override - public long deadlineMs() { - return Long.MAX_VALUE; - } - public Optional error() { return error; } From de267b7b9ed7d4a7978a69058fedefe4a8566679 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:46:47 -0800 Subject: [PATCH 024/130] Updates --- .../internals/events/CompletableBackgroundEvent.java | 8 +++++--- .../consumer/internals/events/CompletableEvent.java | 1 + 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 323e23e3b274e..3c8f9670c154d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -25,7 +24,7 @@ * * @param */ -public abstract class CompletableBackgroundEvent extends BackgroundEvent { +public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; @@ -34,6 +33,7 @@ protected CompletableBackgroundEvent(Type type) { this.future = new CompletableFuture<>(); } + @Override public CompletableFuture future() { return future; } @@ -51,7 +51,9 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(future); + int result = super.hashCode(); + result = 31 * result + future.hashCode(); + return result; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 97559d8cb9be2..8fdcc20fa8363 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,4 +21,5 @@ public interface CompletableEvent { CompletableFuture future(); + } From c94bcb92999563738a6620af2ef91d9ebae03486 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 16:49:22 -0800 Subject: [PATCH 025/130] Update ApplicationEventProcessor.java --- .../consumer/internals/events/ApplicationEventProcessor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index b87615f45909f..8999547751df3 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 @@ -215,13 +215,12 @@ private void process(final AssignmentChangeApplicationEvent event) { } private void process(final ListOffsetsApplicationEvent event) { - final CompletableFuture> future; final Timer timer = timer(event); if (maybeTimeout(event, timer, "Unable to list offsets due to exceeding timeout")) return; - future = requestManagers.offsetsRequestManager.fetchOffsets( + final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets( event.timestampsToSearch(), event.requireTimestamps(), timer From 337c30650b71fc766748785a12ddc8b449c31be1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 17:02:23 -0800 Subject: [PATCH 026/130] Update ApplicationEventProcessor.java --- .../internals/events/ApplicationEventProcessor.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 8999547751df3..92e8e79e0d193 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 @@ -169,12 +169,7 @@ private void process(final CommitApplicationEvent event) { } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture future = manager.addOffsetCommitRequest( - event.offsets(), - timer, - false - ); - event.chain(future); + event.chain(manager.addOffsetCommitRequest(event.offsets(), timer, false)); } private void process(final FetchCommittedOffsetsApplicationEvent event) { @@ -190,11 +185,7 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture> future = manager.addOffsetFetchRequest( - event.partitions(), - timer - ); - event.chain(future); + event.chain(manager.addOffsetFetchRequest(event.partitions(), timer)); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { From 422f935afb02dd1b7668c8c1f95e3199de85f1f3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 24 Jan 2024 17:03:00 -0800 Subject: [PATCH 027/130] Update ApplicationEventProcessor.java --- .../consumer/internals/events/ApplicationEventProcessor.java | 1 + 1 file changed, 1 insertion(+) 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 92e8e79e0d193..ef02bce2f9e18 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 @@ -192,6 +192,7 @@ private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { metadata.requestUpdateForNewTopics(); } + /** * Commit all consumed if auto-commit is enabled. Note this will trigger an async commit, * that will not be retried if the commit request fails. From d08d4a61dee26e94d6ee6d6fdcd6c0cec80e0e65 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 31 Jan 2024 11:39:23 -0800 Subject: [PATCH 028/130] Updates to include RelaxedCompletableFuture --- .../internals/AsyncKafkaConsumer.java | 10 +-- .../internals/ConsumerNetworkThread.java | 4 +- .../internals/RelaxedCompletableFuture.java | 75 +++++++++++++++++++ .../events/ApplicationEventProcessor.java | 18 ++++- .../events/CompletableApplicationEvent.java | 7 +- .../internals/events/EventProcessor.java | 2 - .../internals/AsyncKafkaConsumerTest.java | 32 ++++---- .../internals/ConsumerNetworkThreadTest.java | 2 +- 8 files changed, 114 insertions(+), 36 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.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 dee5387f3911d..b476659c69566 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 @@ -180,7 +180,6 @@ public BackgroundEventProcessor(final LogContext logContext, * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ - @Override public boolean process() { AtomicReference firstError = new AtomicReference<>(); @@ -1471,7 +1470,7 @@ public void unsubscribe() { log.info("Unsubscribing all topics or patterns and assigned partitions"); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); + processBackgroundEvents(unsubscribeApplicationEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1808,20 +1807,17 @@ private void subscribeInternal(Collection topics, Optional T processBackgroundEvents(EventProcessor eventProcessor, - Future future, - Timer timer) { + T processBackgroundEvents(Future future, Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = eventProcessor.process(); + boolean hadEvents = backgroundEventProcessor.process(); try { if (future.isDone()) { 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 aa352cd68a22e..dd6e968b72e42 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 @@ -128,7 +128,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - applicationEventProcessor.process(); + List> futures = applicationEventProcessor.process(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -144,6 +144,8 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); + + futures.forEach(RelaxedCompletableFuture::attempted); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java new file mode 100644 index 0000000000000..6a4c4070913ac --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java @@ -0,0 +1,75 @@ +/* + * 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.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class RelaxedCompletableFuture extends CompletableFuture { + + private final Lock lock; + private final Condition attemptedCondition; + private boolean wasAttempted; + + public RelaxedCompletableFuture() { + this.lock = new ReentrantLock(); + this.attemptedCondition = lock.newCondition(); + } + + public void attempted() { + try { + lock.lock(); + wasAttempted = true; + attemptedCondition.signalAll(); + } finally { + lock.unlock(); + } + } + + @Override + public T get() throws InterruptedException, ExecutionException { + waitForAttempt(); + return super.get(); + } + + @Override + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + waitForAttempt(); + + if (super.isDone()) { + return super.get(); + } else { + return super.get(timeout, unit); + } + } + + private void waitForAttempt() throws InterruptedException { + try { + lock.lock(); + + if (!wasAttempted) + attemptedCondition.await(); + } finally { + lock.unlock(); + } + } +} 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 ef02bce2f9e18..81c134f01385d 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 @@ -16,13 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.MembershipManager; +import org.apache.kafka.clients.consumer.internals.RelaxedCompletableFuture; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; @@ -33,6 +33,7 @@ import org.apache.kafka.common.utils.Timer; import org.slf4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -69,8 +70,19 @@ public ApplicationEventProcessor(final LogContext logContext, * an event generates an error. In such cases, the processor will log an exception, but we do not want those * errors to be propagated to the caller. */ - public boolean process() { - return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); + public List> process() { + List> futures = new ArrayList<>(); + + process((event, error) -> { + error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); + + if (event instanceof CompletableApplicationEvent) { + RelaxedCompletableFuture future = ((CompletableApplicationEvent) event).future(); + futures.add(future); + }; + }); + + return futures; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index b29bd87104a3a..61eafa88a2f9a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.clients.consumer.internals.RelaxedCompletableFuture; import org.apache.kafka.common.utils.Timer; import java.util.Objects; @@ -30,19 +31,19 @@ */ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { - private final CompletableFuture future; + private final RelaxedCompletableFuture future; private final Timer timer; private final long deadlineMs; protected CompletableApplicationEvent(Type type, Timer timer) { super(type); - this.future = new CompletableFuture<>(); + this.future = new RelaxedCompletableFuture<>(); this.timer = timer; this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } @Override - public CompletableFuture future() { + public RelaxedCompletableFuture future() { return future; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 79a987e8a7aa9..d24d4eb260555 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -48,8 +48,6 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } - public abstract boolean process(); - protected abstract void process(T event); @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 167b18c99ab8f..57abfc624b772 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 @@ -1321,7 +1321,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1348,16 +1348,14 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); - } + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1369,17 +1367,15 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); - } + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1395,12 +1391,10 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); - } + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); } private HashMap mockTopicPartitionOffset() { 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 1a68b218e334a..614a870885d75 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 @@ -288,7 +288,7 @@ void testEnsureEventsAreCompleted() { prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap(), timer, false)); ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap(), timer, false); - CompletableFuture future = new CompletableFuture<>(); + RelaxedCompletableFuture future = new RelaxedCompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); applicationEventsQueue.add(event2); From 153390123575ab60761c7dfbabd76142a11e7489 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:00:55 -0800 Subject: [PATCH 029/130] Updates --- .../internals/ConsumerNetworkThread.java | 5 +- .../internals/CoordinatorRequestManager.java | 30 +- .../internals/FetchRequestManager.java | 2 +- .../consumer/internals/MembershipManager.java | 2 +- .../internals/NetworkClientDelegate.java | 52 +- .../internals/OffsetsRequestManager.java | 186 ++++--- .../consumer/internals/RequestManagers.java | 1 - .../TopicMetadataRequestManager.java | 61 ++- .../events/AsyncCommitApplicationEvent.java | 51 +- .../events/CompletableApplicationEvent.java | 19 +- .../events/SyncCommitApplicationEvent.java | 48 +- .../internals/ConsumerTestBuilder.java | 1 - .../CoordinatorRequestManagerTest.java | 1 - .../internals/MembershipManagerImplTest.java | 330 ++++++------ .../internals/NetworkClientDelegateTest.java | 43 +- .../internals/OffsetsRequestManagerTest.java | 468 ++++++++---------- .../TopicMetadataRequestManagerTest.java | 140 +++--- .../events/ApplicationEventProcessorTest.java | 4 +- 18 files changed, 709 insertions(+), 735 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index dd6e968b72e42..d469c342956ae 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 @@ -34,6 +34,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; @@ -128,7 +129,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - List> futures = applicationEventProcessor.process(); + List> futures = applicationEventProcessor.process(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -145,7 +146,7 @@ void runOnce() { .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); - futures.forEach(RelaxedCompletableFuture::attempted); +// futures.forEach(RelaxedCompletableFuture::attempted); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index d21404555ae70..4af6ff8340e08 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.requests.FindCoordinatorResponse; 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.Objects; @@ -53,7 +52,6 @@ public class CoordinatorRequestManager implements RequestManager { private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000; private final Time time; - private final long defaultApiTimeoutMs; private final Logger log; private final BackgroundEventHandler backgroundEventHandler; private final String groupId; @@ -66,7 +64,6 @@ public class CoordinatorRequestManager implements RequestManager { public CoordinatorRequestManager( final Time time, final LogContext logContext, - final long defaultApiTimeoutMs, final long retryBackoffMs, final long retryBackoffMaxMs, final BackgroundEventHandler errorHandler, @@ -75,14 +72,13 @@ public CoordinatorRequestManager( Objects.requireNonNull(groupId); this.time = time; this.log = logContext.logger(this.getClass()); - this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.backgroundEventHandler = errorHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( - logContext, - CoordinatorRequestManager.class.getSimpleName(), - retryBackoffMs, - retryBackoffMaxMs + logContext, + CoordinatorRequestManager.class.getSimpleName(), + retryBackoffMs, + retryBackoffMaxMs ); } @@ -112,13 +108,11 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) { coordinatorRequestState.onSendAttempt(currentTimeMs); FindCoordinatorRequestData data = new FindCoordinatorRequestData() - .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) - .setKey(this.groupId); - Timer timer = time.timer(defaultApiTimeoutMs); + .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) + .setKey(this.groupId); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( new FindCoordinatorRequest.Builder(data), - Optional.empty(), - timer + Optional.empty() ); return unsentRequest.whenComplete((clientResponse, throwable) -> { @@ -140,7 +134,7 @@ NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long curren public void markCoordinatorUnknown(final String cause, final long currentTimeMs) { if (this.coordinator != null) { log.info("Group coordinator {} is unavailable or invalid due to cause: {}. " - + "Rediscovery will be attempted.", this.coordinator, cause); + + "Rediscovery will be attempted.", this.coordinator, cause); this.coordinator = null; timeMarkedUnknownMs = currentTimeMs; totalDisconnectedMin = 0; @@ -162,9 +156,9 @@ private void onSuccessfulResponse( // for the coordinator in the underlying network client layer int coordinatorConnectionId = Integer.MAX_VALUE - coordinator.nodeId(); this.coordinator = new Node( - coordinatorConnectionId, - coordinator.host(), - coordinator.port()); + coordinatorConnectionId, + coordinator.host(), + coordinator.port()); log.info("Discovered group coordinator {}", coordinator); coordinatorRequestState.onSuccessfulAttempt(currentTimeMs); } @@ -225,4 +219,4 @@ private void onResponse( public Optional coordinator() { return Optional.ofNullable(this.coordinator); } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index 16fa012a153d9..b00f79b12e61a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -119,7 +119,7 @@ private PollResult pollInternal(Map successHandler.handle(fetchTarget, data, clientResponse); }; - return new UnsentRequest(request, Optional.of(fetchTarget), timer).whenComplete(responseHandler); + return new UnsentRequest(request, Optional.of(fetchTarget)).whenComplete(responseHandler); }).collect(Collectors.toList()); return new PollResult(requests); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index 226417a80170e..1f06b52515b16 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -120,7 +120,7 @@ public interface MembershipManager extends RequestManager { * @return Future that will complete when the callback execution completes and the heartbeat * to leave the group has been sent out. */ - CompletableFuture leaveGroup(Timer timer); + CompletableFuture leaveGroup(); /** * @return True if the member should send heartbeat to the coordinator without waiting for 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 28c92b9bcad39..0637b951e0075 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 @@ -59,18 +59,20 @@ public class NetworkClientDelegate implements AutoCloseable { private final KafkaClient client; private final Time time; private final Logger log; + private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; public NetworkClientDelegate( - final Time time, - final ConsumerConfig config, - final LogContext logContext, - final KafkaClient client) { + final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client) { this.time = time; this.client = client; this.log = logContext.logger(getClass()); this.unsentRequests = new ArrayDeque<>(); + this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); } @@ -233,6 +235,7 @@ public void addAll(final List requests) { public void add(final UnsentRequest r) { Objects.requireNonNull(r); + r.setTimer(this.time, this.requestTimeoutMs); unsentRequests.add(r); } @@ -264,16 +267,19 @@ public static class UnsentRequest { private final AbstractRequest.Builder requestBuilder; private final FutureCompletionHandler handler; private final Optional node; // empty if random node can be chosen - private final Timer timer; + + private Timer timer; public UnsentRequest(final AbstractRequest.Builder requestBuilder, - final Optional node, - final Timer timer) { + final Optional node) { Objects.requireNonNull(requestBuilder); this.requestBuilder = requestBuilder; this.node = node; this.handler = new FutureCompletionHandler(); - this.timer = timer; + } + + void setTimer(final Time time, final long requestTimeoutMs) { + this.timer = time.timer(requestTimeoutMs); } Timer timer() { @@ -304,11 +310,11 @@ Optional node() { @Override public String toString() { return "UnsentRequest{" + - "requestBuilder=" + requestBuilder + - ", handler=" + handler + - ", node=" + node + - ", timer=" + timer + - '}'; + "requestBuilder=" + requestBuilder + + ", handler=" + handler + + ", node=" + node + + ", timer=" + timer + + '}'; } } @@ -370,17 +376,17 @@ public static Supplier supplier(final Time time, @Override protected NetworkClientDelegate create() { KafkaClient client = ClientUtils.createNetworkClient(config, - metrics, - CONSUMER_METRIC_GROUP_PREFIX, - logContext, - apiVersions, - time, - CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, - metadata, - fetchMetricsManager.throttleTimeSensor(), - clientTelemetrySender); + metrics, + CONSUMER_METRIC_GROUP_PREFIX, + logContext, + apiVersions, + time, + CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, + metadata, + fetchMetricsManager.throttleTimeSensor(), + clientTelemetrySender); return new NetworkClientDelegate(time, config, logContext, client); } }; } -} +} \ No newline at end of file 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 59127389a0dad..bcc66005a0af5 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 @@ -39,7 +39,6 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; 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.ArrayList; @@ -82,7 +81,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis private final Set requestsToRetry; private final List requestsToSend; - private final long defaultApiTimeoutMs; + private final long requestTimeoutMs; private final Time time; private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; @@ -94,7 +93,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, final IsolationLevel isolationLevel, final Time time, final long retryBackoffMs, - final long defaultApiTimeoutMs, + final long requestTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, final BackgroundEventHandler backgroundEventHandler, @@ -115,12 +114,12 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.requestsToSend = new ArrayList<>(); this.subscriptionState = subscriptionState; this.time = time; - this.defaultApiTimeoutMs = defaultApiTimeoutMs; + this.requestTimeoutMs = requestTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState, - time, retryBackoffMs, apiVersions); + 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. @@ -153,34 +152,33 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * processed, following a call to {@link #poll(long)} */ public CompletableFuture> fetchOffsets( - final Map timestampsToSearch, - final boolean requireTimestamps, - final Timer timer) { + final Map timestampsToSearch, + final boolean requireTimestamps) { if (timestampsToSearch.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyMap()); } metadata.addTransientTopics(OffsetFetcherUtils.topicsForPartitions(timestampsToSearch.keySet())); ListOffsetsRequestState listOffsetsRequestState = new ListOffsetsRequestState( - timestampsToSearch, - requireTimestamps, - offsetFetcherUtils, - isolationLevel); + timestampsToSearch, + requireTimestamps, + offsetFetcherUtils, + isolationLevel); listOffsetsRequestState.globalResult.whenComplete((result, error) -> { metadata.clearTransientTopics(); if (error != null) { log.debug("Fetch offsets completed with error for partitions and timestamps {}.", - timestampsToSearch, error); + timestampsToSearch, error); } else { log.debug("Fetch offsets completed successfully for partitions and timestamps {}." + - " Result {}", timestampsToSearch, result); + " Result {}", timestampsToSearch, result); } }); - fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState, timer); + fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState); return listOffsetsRequestState.globalResult.thenApply(result -> - OffsetFetcherUtils.buildOffsetsForTimesResult(timestampsToSearch, result.fetchedOffsets)); + OffsetFetcherUtils.buildOffsetsForTimesResult(timestampsToSearch, result.fetchedOffsets)); } /** @@ -195,7 +193,7 @@ public CompletableFuture> fetchOffsets( * an error is received in the response, it will be saved to be thrown on the next call to * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ - public CompletableFuture resetPositionsIfNeeded(Timer timer) { + public CompletableFuture resetPositionsIfNeeded() { Map offsetResetTimestamps; try { @@ -208,7 +206,7 @@ public CompletableFuture resetPositionsIfNeeded(Timer timer) { if (offsetResetTimestamps.isEmpty()) return CompletableFuture.completedFuture(null); - return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps, timer); + return sendListOffsetsRequestsAndResetPositions(offsetResetTimestamps); } /** @@ -223,14 +221,14 @@ public CompletableFuture resetPositionsIfNeeded(Timer timer) { * detected, a {@link LogTruncationException} will be saved in memory, to be thrown on the * next call to this function. */ - public CompletableFuture validatePositionsIfNeeded(Timer timer) { + public CompletableFuture validatePositionsIfNeeded() { Map partitionsToValidate = - offsetFetcherUtils.getPartitionsToValidate(); + offsetFetcherUtils.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return CompletableFuture.completedFuture(null); } - return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate, timer); + return sendOffsetsForLeaderEpochRequestsAndValidatePositions(partitionsToValidate); } /** @@ -239,8 +237,7 @@ public CompletableFuture validatePositionsIfNeeded(Timer timer) { */ private void fetchOffsetsByTimes(final Map timestampsToSearch, final boolean requireTimestamps, - final ListOffsetsRequestState listOffsetsRequestState, - final Timer timer) { + final ListOffsetsRequestState listOffsetsRequestState) { if (timestampsToSearch.isEmpty()) { // Early return if empty map to avoid wrongfully raising StaleMetadataException on // empty grouping @@ -248,7 +245,7 @@ private void fetchOffsetsByTimes(final Map timestampsToSea } try { List unsentRequests = buildListOffsetsRequests( - timestampsToSearch, requireTimestamps, listOffsetsRequestState, timer); + timestampsToSearch, requireTimestamps, listOffsetsRequestState); requestsToSend.addAll(unsentRequests); } catch (StaleMetadataException e) { requestsToRetry.add(listOffsetsRequestState); @@ -262,12 +259,11 @@ public void onUpdate(ClusterResource clusterResource) { // fetchOffsetsByTimes call if any of the requests being retried fails List requestsToProcess = new ArrayList<>(requestsToRetry); requestsToRetry.clear(); - Timer timer = time.timer(defaultApiTimeoutMs); requestsToProcess.forEach(requestState -> { Map timestampsToSearch = - new HashMap<>(requestState.remainingToSearch); + new HashMap<>(requestState.remainingToSearch); requestState.remainingToSearch.clear(); - fetchOffsetsByTimes(timestampsToSearch, requestState.requireTimestamps, requestState, timer); + fetchOffsetsByTimes(timestampsToSearch, requestState.requireTimestamps, requestState); }); } @@ -282,13 +278,12 @@ public void onUpdate(ClusterResource clusterResource) { * that can be polled to obtain the corresponding timestamps and offsets. */ private List buildListOffsetsRequests( - final Map timestampsToSearch, - final boolean requireTimestamps, - final ListOffsetsRequestState listOffsetsRequestState, - final Timer timer) { + final Map timestampsToSearch, + final boolean requireTimestamps, + final ListOffsetsRequestState listOffsetsRequestState) { log.debug("Building ListOffsets request for partitions {}", timestampsToSearch); Map> timestampsToSearchByNode = - groupListOffsetRequests(timestampsToSearch, Optional.of(listOffsetsRequestState)); + groupListOffsetRequests(timestampsToSearch, Optional.of(listOffsetsRequestState)); if (timestampsToSearchByNode.isEmpty()) { throw new StaleMetadataException(); } @@ -301,12 +296,12 @@ private List buildListOffsetsRequests( listOffsetsRequestState.fetchedOffsets.putAll(multiNodeResult.fetchedOffsets); listOffsetsRequestState.addPartitionsToRetry(multiNodeResult.partitionsToRetry); offsetFetcherUtils.updateSubscriptionState(multiNodeResult.fetchedOffsets, - isolationLevel); + isolationLevel); if (listOffsetsRequestState.remainingToSearch.size() == 0) { ListOffsetResult listOffsetResult = - new ListOffsetResult(listOffsetsRequestState.fetchedOffsets, - listOffsetsRequestState.remainingToSearch.keySet()); + new ListOffsetResult(listOffsetsRequestState.fetchedOffsets, + listOffsetsRequestState.remainingToSearch.keySet()); listOffsetsRequestState.globalResult.complete(listOffsetResult); } else { requestsToRetry.add(listOffsetsRequestState); @@ -321,12 +316,10 @@ private List buildListOffsetsRequests( Node node = entry.getKey(); CompletableFuture partialResult = buildListOffsetRequestToNode( - node, - entry.getValue(), - requireTimestamps, - unsentRequests, - timer - ); + node, + entry.getValue(), + requireTimestamps, + unsentRequests); partialResult.whenComplete((result, error) -> { if (error != null) { @@ -344,38 +337,35 @@ private List buildListOffsetsRequests( * target timestamps. This also adds the request to the list of unsentRequests. */ private CompletableFuture buildListOffsetRequestToNode( - Node node, - Map targetTimes, - boolean requireTimestamps, - List unsentRequests, - Timer timer) { + Node node, + Map targetTimes, + boolean requireTimestamps, + List unsentRequests) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamps, isolationLevel, false) - .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); + .forConsumer(requireTimestamps, isolationLevel, false) + .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); log.debug("Creating ListOffset request {} for broker {} to reset positions", builder, - node); + node); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - builder, - Optional.ofNullable(node), - timer - ); + builder, + Optional.ofNullable(node)); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, error) -> { if (error != null) { log.debug("Sending ListOffset request {} to broker {} failed", - builder, - node, - error); + builder, + node, + error); result.completeExceptionally(error); } else { ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); try { ListOffsetResult listOffsetResult = - offsetFetcherUtils.handleListOffsetResponse(lor); + offsetFetcherUtils.handleListOffsetResponse(lor); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -395,37 +385,35 @@ private CompletableFuture buildListOffsetRequestToNode( * complete. */ private CompletableFuture sendListOffsetsRequestsAndResetPositions( - final Map timestampsToSearch, - final Timer timer) { + final Map timestampsToSearch) { Map> timestampsToSearchByNode = - groupListOffsetRequests(timestampsToSearch, Optional.empty()); + groupListOffsetRequests(timestampsToSearch, Optional.empty()); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); - final long nextResetTimeMs = timer.currentTimeMs() + timer.remainingMs(); + timestampsToSearchByNode.forEach((node, resetTimestamps) -> { - subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), nextResetTimeMs); + subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), + time.milliseconds() + requestTimeoutMs); CompletableFuture partialResult = buildListOffsetRequestToNode( - node, - resetTimestamps, - false, - unsentRequests, - timer - ); + node, + resetTimestamps, + false, + unsentRequests); partialResult.whenComplete((result, error) -> { if (error == null) { offsetFetcherUtils.onSuccessfulResponseForResettingPositions(resetTimestamps, - result); + result); } else { RuntimeException e; if (error instanceof RuntimeException) { e = (RuntimeException) error; } else { e = new RuntimeException("Unexpected failure in ListOffsets request for " + - "resetting positions", error); + "resetting positions", error); } offsetFetcherUtils.onFailedResponseForResettingPositions(resetTimestamps, e); } @@ -458,13 +446,12 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( */ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePositions( - Map partitionsToValidate, - Timer timer) { + Map partitionsToValidate) { final Map> regrouped = - regroupFetchPositionsByLeader(partitionsToValidate); + regroupFetchPositionsByLeader(partitionsToValidate); - long nextResetTimeMs = timer.currentTimeMs() + timer.remainingMs(); + long nextResetTimeMs = time.milliseconds() + requestTimeoutMs; final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); final List unsentRequests = new ArrayList<>(); @@ -483,8 +470,8 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi if (!hasUsableOffsetForLeaderEpochVersion(nodeApiVersions)) { log.debug("Skipping validation of fetch offsets for partitions {} since the broker does not " + - "support the required protocol version (introduced in Kafka 2.3)", - fetchPositions.keySet()); + "support the required protocol version (introduced in Kafka 2.3)", + fetchPositions.keySet()); for (TopicPartition partition : fetchPositions.keySet()) { subscriptionState.completeValidation(partition); } @@ -494,19 +481,19 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi subscriptionState.setNextAllowedRetry(fetchPositions.keySet(), nextResetTimeMs); CompletableFuture partialResult = - buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests, timer); + buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests); partialResult.whenComplete((offsetsResult, error) -> { if (error == null) { offsetFetcherUtils.onSuccessfulResponseForValidatingPositions(fetchPositions, - offsetsResult); + offsetsResult); } else { RuntimeException e; if (error instanceof RuntimeException) { e = (RuntimeException) error; } else { e = new RuntimeException("Unexpected failure in OffsetsForLeaderEpoch " + - "request for validating positions", error); + "request for validating positions", error); } offsetFetcherUtils.onFailedResponseForValidatingPositions(fetchPositions, e); } @@ -531,35 +518,32 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi * positions to fetch. This also adds the request to the list of unsentRequests. */ private CompletableFuture buildOffsetsForLeaderEpochRequestToNode( - final Node node, - final Map fetchPositions, - final List unsentRequests, - final Timer timer) { + final Node node, + final Map fetchPositions, + List unsentRequests) { AbstractRequest.Builder builder = - OffsetsForLeaderEpochUtils.prepareRequest(fetchPositions); + OffsetsForLeaderEpochUtils.prepareRequest(fetchPositions); log.debug("Creating OffsetsForLeaderEpoch request request {} to broker {}", builder, node); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - builder, - Optional.ofNullable(node), - timer - ); + builder, + Optional.ofNullable(node)); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, error) -> { if (error != null) { log.debug("Sending OffsetsForLeaderEpoch request {} to broker {} failed", - builder, - node, - error); + builder, + node, + error); result.completeExceptionally(error); } else { OffsetsForLeaderEpochResponse offsetsForLeaderEpochResponse = (OffsetsForLeaderEpochResponse) response.responseBody(); log.trace("Received OffsetsForLeaderEpoch response {} from broker {}", offsetsForLeaderEpochResponse, node); try { OffsetsForLeaderEpochUtils.OffsetForEpochResult listOffsetResult = - OffsetsForLeaderEpochUtils.handleResponse(fetchPositions, offsetsForLeaderEpochResponse); + OffsetsForLeaderEpochUtils.handleResponse(fetchPositions, offsetsForLeaderEpochResponse); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -595,7 +579,7 @@ private ListOffsetsRequestState(Map timestampsToSearch, private void addPartitionsToRetry(Set partitionsToRetry) { remainingToSearch.putAll(partitionsToRetry.stream() - .collect(Collectors.toMap(tp -> tp, timestampsToSearch::get))); + .collect(Collectors.toMap(tp -> tp, timestampsToSearch::get))); } } @@ -623,8 +607,8 @@ private void addPartialResult(ListOffsetResult partialResult) { if (expectedResponses.decrementAndGet() == 0) { ListOffsetResult result = - new ListOffsetResult(fetchedTimestampOffsets, - partitionsToRetry); + new ListOffsetResult(fetchedTimestampOffsets, + partitionsToRetry); resultFuture.complete(result); } } catch (RuntimeException e) { @@ -644,8 +628,8 @@ private void addPartialResult(ListOffsetResult partialResult) { * metadata update). */ private Map> groupListOffsetRequests( - final Map timestampsToSearch, - final Optional listOffsetsRequestState) { + final Map timestampsToSearch, + final Optional listOffsetsRequestState) { final Map partitionDataMap = new HashMap<>(); for (Map.Entry entry : timestampsToSearch.entrySet()) { TopicPartition tp = entry.getKey(); @@ -659,9 +643,9 @@ private Map serverAssignor = Optional.ofNullable(config.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); coordinator = new CoordinatorRequestManager(time, logContext, - defaultApiTimeoutMs, retryBackoffMs, retryBackoffMaxMs, backgroundEventHandler, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 9f217477a1e08..8e4c680ce3e0f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -29,8 +29,6 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; 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.Collections; @@ -83,8 +81,8 @@ public TopicMetadataRequestManager(final LogContext context, final ConsumerConfi public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { // Prune any requests which have timed out List expiredRequests = inflightRequests.stream() - .filter(req -> req.timer.isExpired()) - .collect(Collectors.toList()); + .filter(req -> req.isExpired(currentTimeMs)) + .collect(Collectors.toList()); expiredRequests.forEach(TopicMetadataRequestState::expire); List requests = inflightRequests.stream() @@ -101,12 +99,12 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * * @return the future of the metadata request. */ - public CompletableFuture>> requestAllTopicsMetadata(final Timer timer) { + public CompletableFuture>> requestAllTopicsMetadata(final long expirationTimeMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( - logContext, - timer, - retryBackoffMs, - retryBackoffMaxMs); + logContext, + expirationTimeMs, + retryBackoffMs, + retryBackoffMaxMs); inflightRequests.add(newRequest); return newRequest.future; } @@ -117,14 +115,13 @@ public CompletableFuture>> requestAllTopicsMetad * @param topic to be requested. * @return the future of the metadata request. */ - public CompletableFuture>> requestTopicMetadata(final String topic, - final Timer timer) { + public CompletableFuture>> requestTopicMetadata(final String topic, final long expirationTimeMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( - logContext, - topic, - timer, - retryBackoffMs, - retryBackoffMaxMs); + logContext, + topic, + expirationTimeMs, + retryBackoffMs, + retryBackoffMaxMs); inflightRequests.add(newRequest); return newRequest.future; } @@ -137,24 +134,24 @@ List inflightRequests() { class TopicMetadataRequestState extends RequestState { private final String topic; private final boolean allTopics; - private final Timer timer; + private final long expirationTimeMs; CompletableFuture>> future; public TopicMetadataRequestState(final LogContext logContext, - final Timer timer, + final long expirationTimeMs, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs); + retryBackoffMaxMs); future = new CompletableFuture<>(); this.topic = null; this.allTopics = true; - this.timer = timer; + this.expirationTimeMs = expirationTimeMs; } public TopicMetadataRequestState(final LogContext logContext, final String topic, - final Timer timer, + final long expirationTimeMs, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, @@ -162,7 +159,7 @@ public TopicMetadataRequestState(final LogContext logContext, future = new CompletableFuture<>(); this.topic = topic; this.allTopics = false; - this.timer = timer; + this.expirationTimeMs = expirationTimeMs; } /** @@ -170,7 +167,7 @@ public TopicMetadataRequestState(final LogContext logContext, * {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest} if needed. */ private Optional send(final long currentTimeMs) { - if (timer.isExpired()) { + if (currentTimeMs >= expirationTimeMs) { return Optional.empty(); } @@ -186,18 +183,20 @@ private Optional send(final long currentTim return Optional.of(createUnsentRequest(request)); } + private boolean isExpired(final long currentTimeMs) { + return currentTimeMs >= expirationTimeMs; + } + private void expire() { completeFutureAndRemoveRequest( - new TimeoutException("Timeout expired while fetching topic metadata")); + new TimeoutException("Timeout expired while fetching topic metadata")); } private NetworkClientDelegate.UnsentRequest createUnsentRequest( - final MetadataRequest.Builder request) { + final MetadataRequest.Builder request) { NetworkClientDelegate.UnsentRequest unsent = new NetworkClientDelegate.UnsentRequest( request, - Optional.empty(), - timer - ); + Optional.empty()); return unsent.whenComplete((response, exception) -> { if (response == null) { @@ -211,7 +210,7 @@ private NetworkClientDelegate.UnsentRequest createUnsentRequest( private void handleError(final Throwable exception, final long completionTimeMs) { if (exception instanceof RetriableException) { - if (timer.isExpired()) { + if (completionTimeMs >= expirationTimeMs) { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); } else { @@ -229,7 +228,7 @@ private void handleResponse(final ClientResponse response) { future.complete(res); inflightRequests.remove(this); } catch (RetriableException e) { - if (timer.isExpired()) { + if (responseTimeMs >= expirationTimeMs) { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); } else { @@ -287,4 +286,4 @@ public String topic() { return topic; } } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index 7a939ce3cfd16..d77f74de8c50e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -17,23 +17,60 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.internals.RelaxedCompletableFuture; import org.apache.kafka.common.TopicPartition; + +import java.util.Collections; import java.util.Map; /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ -public class AsyncCommitApplicationEvent extends CommitApplicationEvent { +public class AsyncCommitApplicationEvent extends ApplicationEvent implements CompletableEvent { + + private final RelaxedCompletableFuture future; + + /** + * Offsets to commit per partition. + */ + private final Map offsets; public AsyncCommitApplicationEvent(final Map offsets) { - super(offsets, Type.COMMIT_ASYNC); + super(Type.COMMIT_SYNC); + this.offsets = Collections.unmodifiableMap(offsets); + this.future = new RelaxedCompletableFuture<>(); + + for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { + if (offsetAndMetadata.offset() < 0) { + throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); + } + } + } + + @Override + public RelaxedCompletableFuture future() { + return future; + } + + public Map offsets() { + return offsets; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + AsyncCommitApplicationEvent that = (AsyncCommitApplicationEvent) o; + + return offsets.equals(that.offsets); } @Override - public String toString() { - return "AsyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - '}'; + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + offsets.hashCode(); + return result; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 61eafa88a2f9a..2cda3a5d6696d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -31,7 +31,7 @@ */ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { - private final RelaxedCompletableFuture future; + private final CompletableFuture future; private final Timer timer; private final long deadlineMs; @@ -43,7 +43,7 @@ protected CompletableApplicationEvent(Type type, Timer timer) { } @Override - public RelaxedCompletableFuture future() { + public CompletableFuture future() { return future; } @@ -55,21 +55,6 @@ public T get() { return ConsumerUtils.getResult(future, timer); } - public void chain(final CompletableFuture providedFuture) { - Objects.requireNonNull( - providedFuture, - () -> String.format("Could not chain the event's future (%s) to the provided future because the provided future was null", this.future) - ); - - providedFuture.whenComplete((value, exception) -> { - if (exception != null) { - this.future.completeExceptionally(exception); - } else { - this.future.complete(value); - } - }); - } - @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 43dfee6ab18b5..1e85d0199a3d8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -18,35 +18,53 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + +import java.util.Collections; import java.util.Map; +import java.util.Objects; /** * Event to commit offsets waiting for a response and retrying on expected retriable errors until * the timer expires. */ -public class SyncCommitApplicationEvent extends CommitApplicationEvent { +public class SyncCommitApplicationEvent extends CompletableApplicationEvent { /** - * Time to wait for a response, retrying on retriable errors. + * Offsets to commit per partition. */ - private final long retryTimeoutMs; + private final Map offsets; + + public SyncCommitApplicationEvent(final Map offsets, final Timer timer) { + super(Type.COMMIT_SYNC, timer); + this.offsets = Collections.unmodifiableMap(offsets); + + for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { + if (offsetAndMetadata.offset() < 0) { + throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); + } + } + } - public SyncCommitApplicationEvent(final Map offsets, - final long retryTimeoutMs) { - super(offsets, Type.COMMIT_SYNC); - this.retryTimeoutMs = retryTimeoutMs; + public Map offsets() { + return offsets; } - public Long retryTimeoutMs() { - return retryTimeoutMs; + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + SyncCommitApplicationEvent that = (SyncCommitApplicationEvent) o; + + return offsets.equals(that.offsets); } @Override - public String toString() { - return "SyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - ", retryTimeout=" + retryTimeoutMs + "ms" + - '}'; + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + offsets.hashCode(); + return result; } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 7bc7329618bff..513799cabe29b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -181,7 +181,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA CoordinatorRequestManager coordinator = spy(new CoordinatorRequestManager( time, logContext, - requestTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, backgroundEventHandler, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index d12c1e1df7a45..e24e34972bc54 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -198,7 +198,6 @@ private CoordinatorRequestManager setupCoordinatorManager(String groupId) { return new CoordinatorRequestManager( time, new LogContext(), - DEFAULT_API_TIMEOUT_MS, RETRY_BACKOFF_MS, RETRY_BACKOFF_MS, this.backgroundEventHandler, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java index e589baad9170f..4eb4d6d27790a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; @@ -34,7 +33,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -99,7 +97,6 @@ public class MembershipManagerImplTest { private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private Time time; - private int defaultApiTimeoutMs; @BeforeEach public void setup() { @@ -110,7 +107,6 @@ public void setup() { backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; time = testBuilder.time; - defaultApiTimeoutMs = testBuilder.config.getInt(CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG); } @AfterEach @@ -122,18 +118,18 @@ public void tearDown() { private MembershipManagerImpl createMembershipManagerJoiningGroup() { MembershipManagerImpl manager = spy(new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time)); manager.transitionToJoining(); return manager; } private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId) { MembershipManagerImpl manager = spy(new MembershipManagerImpl( - GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time)); manager.transitionToJoining(); return manager; } @@ -141,9 +137,9 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId, String serverAssignor) { MembershipManagerImpl manager = new MembershipManagerImpl( - GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, - Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, logContext, Optional.empty(), backgroundEventHandler, time); + GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, + Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, + metadata, logContext, Optional.empty(), backgroundEventHandler, time); manager.transitionToJoining(); return manager; } @@ -167,17 +163,16 @@ public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() { // First join should register to get metadata updates MembershipManagerImpl manager = new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time); manager.transitionToJoining(); clearInvocations(metadata); // Following joins should not register again. receiveEmptyAssignment(manager); mockLeaveGroup(); - Timer timer = time.timer(defaultApiTimeoutMs); - manager.leaveGroup(timer); + manager.leaveGroup(); assertEquals(MemberState.LEAVING, manager.state()); manager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, manager.state()); @@ -200,12 +195,12 @@ public void testTransitionToReconcilingOnlyIfAssignmentReceived() { assertEquals(MemberState.JOINING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithoutAssignment = - createConsumerGroupHeartbeatResponse(null); + createConsumerGroupHeartbeatResponse(null); membershipManager.onHeartbeatResponseReceived(responseWithoutAssignment.data()); assertNotEquals(MemberState.RECONCILING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithAssignment = - createConsumerGroupHeartbeatResponse(createAssignment(true)); + createConsumerGroupHeartbeatResponse(createAssignment(true)); membershipManager.onHeartbeatResponseReceived(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @@ -230,7 +225,7 @@ public void testMemberIdAndEpochResetOnFencedMembers() { public void testTransitionToFatal() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(null); + createConsumerGroupHeartbeatResponse(null); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); @@ -245,9 +240,9 @@ public void testTransitionToFatal() { @Test public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); @@ -288,8 +283,7 @@ public void testListenersGetNotifiedOnTransitionsToLeavingGroup() { clearInvocations(listener); mockLeaveGroup(); - Timer timer = time.timer(defaultApiTimeoutMs); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); verify(listener).onMemberEpochUpdated(Optional.empty(), Optional.empty()); } @@ -302,17 +296,17 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { int epoch = 5; membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(epoch)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(epoch)); verify(listener).onMemberEpochUpdated(Optional.of(epoch), Optional.of(MEMBER_ID)); clearInvocations(listener); membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(epoch)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(epoch)); verify(listener, never()).onMemberEpochUpdated(any(), any()); } @@ -392,8 +386,7 @@ public void testFencingWhenStateIsLeaving() { // Start leaving group. mockLeaveGroup(); - Timer timer = time.timer(defaultApiTimeoutMs); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); // Get fenced while leaving. Member should not trigger any callback or try to @@ -419,19 +412,18 @@ public void testLeaveGroupEpoch() { // Static member should leave the group with epoch -2. MembershipManagerImpl membershipManager = createMemberInStableState("instance1"); mockLeaveGroup(); - Timer timer = time.timer(defaultApiTimeoutMs); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH, - membershipManager.memberEpoch()); + membershipManager.memberEpoch()); // Dynamic member should leave the group with epoch -1. membershipManager = createMemberInStableState(null); mockLeaveGroup(); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, - membershipManager.memberEpoch()); + membershipManager.memberEpoch()); } /** @@ -483,8 +475,8 @@ public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { // Reconciliation that does not complete stuck on revocation commit. CompletableFuture commitResult = - mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId1, topic1, - Arrays.asList(1, 2), true); + mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId1, topic1, + Arrays.asList(1, 2), true); Map> assignment1 = topicIdPartitionsMap(topicId1, 1, 2); assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); @@ -526,7 +518,7 @@ public void testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU Uuid topicId1 = Uuid.randomUuid(); String topic1 = "topic1"; MembershipManagerImpl membershipManager = - mockMemberSuccessfullyReceivesAndAcksAssignment(topicId1, topic1, Collections.singletonList(0)); + mockMemberSuccessfullyReceivesAndAcksAssignment(topicId1, topic1, Collections.singletonList(0)); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); clearInvocations(membershipManager, subscriptionState); @@ -538,8 +530,8 @@ public void testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU Uuid topicId2 = Uuid.randomUuid(); String topic2 = "topic2"; CompletableFuture commitResult = - mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId2, topic2, - Arrays.asList(1, 2), false); + mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId2, topic2, + Arrays.asList(1, 2), false); verify(metadata).requestUpdate(anyBoolean()); assertEquals(Collections.singleton(topicId2), membershipManager.topicsAwaitingReconciliation()); @@ -756,12 +748,11 @@ public void testLeaveGroupWhenMemberOwnsAssignment() { @Test public void testLeaveGroupWhenMemberAlreadyLeaving() { MembershipManager membershipManager = createMemberInStableState(); - Timer timer = time.timer(defaultApiTimeoutMs); // First leave attempt. Should trigger the callbacks and stay LEAVING until // callbacks complete and the heartbeat is sent out. mockLeaveGroup(); - CompletableFuture leaveResult1 = membershipManager.leaveGroup(timer); + CompletableFuture leaveResult1 = membershipManager.leaveGroup(); assertFalse(leaveResult1.isDone()); assertEquals(MemberState.LEAVING, membershipManager.state()); verify(subscriptionState).assignFromSubscribed(Collections.emptySet()); @@ -771,7 +762,7 @@ public void testLeaveGroupWhenMemberAlreadyLeaving() { // trigger any callbacks, and return a future that will complete when the ongoing first // leave operation completes. mockLeaveGroup(); - CompletableFuture leaveResult2 = membershipManager.leaveGroup(timer); + CompletableFuture leaveResult2 = membershipManager.leaveGroup(); verify(subscriptionState, never()).rebalanceListener(); assertFalse(leaveResult2.isDone()); @@ -788,12 +779,11 @@ public void testLeaveGroupWhenMemberAlreadyLeaving() { @Test public void testLeaveGroupWhenMemberAlreadyLeft() { - Timer timer = time.timer(defaultApiTimeoutMs); MembershipManager membershipManager = createMemberInStableState(); // Leave group triggered and completed mockLeaveGroup(); - CompletableFuture leaveResult1 = membershipManager.leaveGroup(timer); + CompletableFuture leaveResult1 = membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); @@ -805,7 +795,7 @@ public void testLeaveGroupWhenMemberAlreadyLeft() { // Call to leave group again, when member already left. Should be no-op (no callbacks, // no assignment updated) mockLeaveGroup(); - CompletableFuture leaveResult2 = membershipManager.leaveGroup(timer); + CompletableFuture leaveResult2 = membershipManager.leaveGroup(); assertTrue(leaveResult2.isDone()); assertFalse(leaveResult2.isCompletedExceptionally()); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); @@ -848,12 +838,11 @@ public void testFatalFailureWhenStateIsPrepareLeaving() { @Test public void testFatalFailureWhenStateIsLeaving() { - Timer timer = time.timer(defaultApiTimeoutMs); MembershipManagerImpl membershipManager = createMemberInStableState(); // Start leaving group. mockLeaveGroup(); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); // Get fatal failure while waiting to send the heartbeat to leave. Member should @@ -868,12 +857,11 @@ public void testFatalFailureWhenStateIsLeaving() { @Test public void testFatalFailureWhenMemberAlreadyLeft() { - Timer timer = time.timer(defaultApiTimeoutMs); MembershipManagerImpl membershipManager = createMemberInStableState(); // Start leaving group. mockLeaveGroup(); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); // Last heartbeat sent. @@ -896,9 +884,9 @@ public void testUpdateStateFailsOnResponsesWithErrors() { // Updating state with a heartbeat response containing errors cannot be performed and // should fail. ConsumerGroupHeartbeatResponse unknownMemberResponse = - createConsumerGroupHeartbeatResponseWithError(); + createConsumerGroupHeartbeatResponseWithError(); assertThrows(IllegalArgumentException.class, - () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); + () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); } /** @@ -1040,14 +1028,14 @@ public void testMemberKeepsUnresolvedAssignmentWaitingForMetadataUntilResolved() String topic1Name = "topic1"; Uuid topic2 = Uuid.randomUuid(); ConsumerGroupHeartbeatResponseData.Assignment assignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic1) - .setPartitions(Collections.singletonList(0)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic2) - .setPartitions(Arrays.asList(1, 3)) - )); + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic1) + .setPartitions(Collections.singletonList(0)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic2) + .setPartitions(Arrays.asList(1, 3)) + )); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topic1, topic1Name)); // Receive assignment partly in metadata - reconcile+ack what's in metadata, keep the @@ -1094,7 +1082,7 @@ public void testReconcileNewPartitionsAssignedWhenOtherPartitionsOwned() { TopicIdPartition ownedPartition = new TopicIdPartition(topicId, new TopicPartition(topicName, 0)); MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, - Collections.singletonList(ownedPartition)); + Collections.singletonList(ownedPartition)); // New assignment received, adding partitions 1 and 2 to the previously owned partition 0. receiveAssignment(topicId, Arrays.asList(0, 1, 2), membershipManager); @@ -1199,7 +1187,7 @@ public void testReconcilePartitionsRevokedWithFailedAutoCommitCompletesRevocatio // Complete commit request commitResult.completeExceptionally(new KafkaException("Commit request failed with " + - "non-retriable error")); + "non-retriable error")); testRevocationOfAllPartitionsCompleted(membershipManager); } @@ -1235,10 +1223,10 @@ public void testMetadataUpdatesReconcilesUnresolvedAssignments() { // Assignment not in metadata ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(Arrays.asList(0, 1)))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(Arrays.asList(0, 1)))); MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -1264,10 +1252,10 @@ public void testMetadataUpdatesRequestsAnotherUpdateIfNeeded() { // Assignment not in metadata ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(Arrays.asList(0, 1)))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(Arrays.asList(0, 1)))); MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -1373,11 +1361,11 @@ public void testListenerCallbacksBasic() { // Step 3: assign partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - topicPartitions(topicName, 0, 1), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 0, 1), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1401,21 +1389,21 @@ public void testListenerCallbacksBasic() { // Step 6: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(topicName, 0, 1), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0, 1), + true ); assertTrue(membershipManager.reconciliationInProgress()); // Step 7: assign partitions should still be called, even though it's empty performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - Collections.emptySortedSet(), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet(), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1438,9 +1426,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartition(membershipManager, topicId, topicName); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.of(new IllegalArgumentException("Intentional onPartitionsRevoked() error")), - Optional.empty(), - Optional.empty() + Optional.of(new IllegalArgumentException("Intentional onPartitionsRevoked() error")), + Optional.empty(), + Optional.empty() ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1464,11 +1452,11 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 3: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(topicName, 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1491,9 +1479,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { Uuid topicId = Uuid.randomUuid(); mockOwnedPartition(membershipManager, topicId, topicName); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.empty(), - Optional.of(new IllegalArgumentException("Intentional onPartitionsAssigned() error")), - Optional.empty() + Optional.empty(), + Optional.of(new IllegalArgumentException("Intentional onPartitionsAssigned() error")), + Optional.empty() ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1517,22 +1505,22 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { // Step 3: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions("topic1", 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions("topic1", 0), + true ); assertTrue(membershipManager.reconciliationInProgress()); // Step 4: assign partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - Collections.emptySortedSet(), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet(), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1700,9 +1688,9 @@ private void testOnPartitionsLost(Optional lostError) { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.empty(), - Optional.empty(), - lostError + Optional.empty(), + Optional.empty(), + lostError ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1719,11 +1707,11 @@ private void testOnPartitionsLost(Optional lostError) { // Step 3: invoke the callback performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, - topicPartitions("topic1", 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, + topicPartitions("topic1", 0), + true ); // Step 4: Receive ack and make sure we're done and our listener was called appropriately @@ -1737,10 +1725,10 @@ private void testOnPartitionsLost(Optional lostError) { private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { return new ConsumerRebalanceListenerInvoker( - new LogContext(), - subscriptionState, - new MockTime(1), - new RebalanceCallbackMetricsManager(new Metrics()) + new LogContext(), + subscriptionState, + new MockTime(1), + new RebalanceCallbackMetricsManager(new Metrics()) ); } @@ -1788,10 +1776,10 @@ private Map> topicIdPartitionsMap(Uuid topicId, int... } private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(MembershipManagerImpl membershipManager, - ConsumerRebalanceListenerInvoker invoker, - ConsumerRebalanceListenerMethodName expectedMethodName, - SortedSet expectedPartitions, - boolean complete) { + ConsumerRebalanceListenerInvoker invoker, + ConsumerRebalanceListenerMethodName expectedMethodName, + SortedSet expectedPartitions, + boolean complete) { // We expect only our enqueued event in the background queue. assertEquals(1, backgroundEventQueue.size()); assertNotNull(backgroundEventQueue.peek()); @@ -1802,10 +1790,10 @@ private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(Membersh assertEquals(expectedPartitions, neededEvent.partitions()); ConsumerRebalanceListenerCallbackCompletedEvent invokedEvent = invokeRebalanceCallbacks( - invoker, - neededEvent.methodName(), - neededEvent.partitions(), - neededEvent.future() + invoker, + neededEvent.methodName(), + neededEvent.partitions(), + neededEvent.future() ); if (complete) { @@ -1861,7 +1849,7 @@ public void testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() { } private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( - Uuid topicId, String topicName, List partitions) { + Uuid topicId, String topicName, List partitions) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, Collections.emptyList()); @@ -1879,7 +1867,7 @@ private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( } private CompletableFuture mockEmptyAssignmentAndRevocationStuckOnCommit( - MembershipManagerImpl membershipManager) { + MembershipManagerImpl membershipManager) { CompletableFuture commitResult = mockRevocationNoCallbacks(true); receiveEmptyAssignment(membershipManager); @@ -1894,8 +1882,8 @@ private CompletableFuture mockEmptyAssignmentAndRevocationStuckOnCommit( } private CompletableFuture mockNewAssignmentAndRevocationStuckOnCommit( - MembershipManagerImpl membershipManager, Uuid topicId, String topicName, - List partitions, boolean mockMetadata) { + MembershipManagerImpl membershipManager, Uuid topicId, String topicName, + List partitions, boolean mockMetadata) { CompletableFuture commitResult = mockRevocationNoCallbacks(true); if (mockMetadata) { when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); @@ -1989,7 +1977,7 @@ private void testRevocationCompleted(MembershipManagerImpl membershipManager, verify(subscriptionState).markPendingRevocation(anySet()); List expectedTopicPartitionAssignment = - buildTopicPartitions(expectedCurrentAssignment); + buildTopicPartitions(expectedCurrentAssignment); verify(subscriptionState).assignFromSubscribed(new HashSet<>(expectedTopicPartitionAssignment)); } @@ -2075,29 +2063,29 @@ private void receiveAssignment(Map> topicIdPartitionLis private void receiveAssignment(Uuid topicId, List partitions, MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(partitions))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions, MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(partitions))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); + createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } private void receiveEmptyAssignment(MembershipManager membershipManager) { // New empty assignment received, revoking owned partition. ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.emptyList()); + .setTopicPartitions(Collections.emptyList()); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } @@ -2123,14 +2111,13 @@ private void testFencedMemberReleasesAssignmentAndTransitionsToJoining(Membershi * transition to {@link MemberState#UNSUBSCRIBED} */ private void testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(MembershipManager membershipManager) { - Timer timer = time.timer(defaultApiTimeoutMs); mockLeaveGroup(); - CompletableFuture leaveResult = membershipManager.leaveGroup(timer); + CompletableFuture leaveResult = membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); assertFalse(leaveResult.isDone(), "Leave group result should not complete until the " + - "heartbeat request to leave is sent out."); + "heartbeat request to leave is sent out."); membershipManager.onHeartbeatRequestSent(); @@ -2151,7 +2138,6 @@ private void mockLeaveGroup() { private ConsumerRebalanceListenerCallbackCompletedEvent mockPrepareLeavingStuckOnUserCallback( MembershipManagerImpl membershipManager, ConsumerRebalanceListenerInvoker invoker) { - Timer timer = time.timer(defaultApiTimeoutMs); String topicName = "topic1"; TopicPartition ownedPartition = new TopicPartition(topicName, 0); @@ -2162,7 +2148,7 @@ private ConsumerRebalanceListenerCallbackCompletedEvent mockPrepareLeavingStuckO when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); doNothing().when(subscriptionState).markPendingRevocation(anySet()); when(commitRequestManager.autoCommitEnabled()).thenReturn(false); - membershipManager.leaveGroup(timer); + membershipManager.leaveGroup(); return performCallback( membershipManager, invoker, @@ -2184,12 +2170,12 @@ private void testStateUpdateOnFatalFailure(MembershipManagerImpl membershipManag } private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse( - ConsumerGroupHeartbeatResponseData.Assignment assignment) { + ConsumerGroupHeartbeatResponseData.Assignment assignment) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH) - .setAssignment(assignment)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignment(assignment)); } /** @@ -2198,19 +2184,19 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse( * receives a heartbeat response to the join request, and the response includes an assignment. */ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithBumpedEpoch( - ConsumerGroupHeartbeatResponseData.Assignment assignment) { + ConsumerGroupHeartbeatResponseData.Assignment assignment) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH + 1) - .setAssignment(assignment)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH + 1) + .setAssignment(assignment)); } private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError() { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(5)); + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(5)); } private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean mockMetadata) { @@ -2223,14 +2209,14 @@ private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean m when(metadata.topicNames()).thenReturn(topicNames); } return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic1) - .setPartitions(Arrays.asList(0, 1, 2)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic2) - .setPartitions(Arrays.asList(3, 4, 5)) - )); + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic1) + .setPartitions(Arrays.asList(0, 1, 2)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic2) + .setPartitions(Arrays.asList(3, 4, 5)) + )); } private MembershipManagerImpl memberJoinWithAssignment() { @@ -2243,4 +2229,4 @@ private MembershipManagerImpl memberJoinWithAssignment() { assertFalse(membershipManager.currentAssignment().isEmpty()); return membershipManager; } -} +} \ No newline at end of file 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 336d71a501cc5..0d1eb7b992d7e 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 @@ -44,6 +44,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -63,7 +64,7 @@ public void setup() { @Test public void testSuccessfulResponse() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); prepareFindCoordinatorResponse(Errors.NONE); ncd.add(unsentRequest); @@ -78,7 +79,7 @@ public void testSuccessfulResponse() throws Exception { public void testTimeoutBeforeSend() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { client.setUnreachable(mockNode(), REQUEST_TIMEOUT_MS); - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); time.sleep(REQUEST_TIMEOUT_MS); @@ -91,7 +92,7 @@ public void testTimeoutBeforeSend() throws Exception { @Test public void testTimeoutAfterSend() throws Exception { try (NetworkClientDelegate ncd = newNetworkClientDelegate()) { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); ncd.add(unsentRequest); ncd.poll(0, time.milliseconds()); time.sleep(REQUEST_TIMEOUT_MS); @@ -103,7 +104,7 @@ public void testTimeoutAfterSend() throws Exception { @Test public void testEnsureCorrectCompletionTimeOnFailure() { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); long timeMs = time.milliseconds(); unsentRequest.handler().onFailure(timeMs, new TimeoutException()); @@ -113,7 +114,7 @@ public void testEnsureCorrectCompletionTimeOnFailure() { @Test public void testEnsureCorrectCompletionTimeOnComplete() { - NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(REQUEST_TIMEOUT_MS); + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); long timeMs = time.milliseconds(); final ClientResponse response = mock(ClientResponse.class); when(response.receivedTimeMs()).thenReturn(timeMs); @@ -122,6 +123,23 @@ public void testEnsureCorrectCompletionTimeOnComplete() { assertEquals(timeMs, unsentRequest.handler().completionTimeMs()); } + @Test + public void testEnsureTimerSetOnAdd() { + NetworkClientDelegate ncd = newNetworkClientDelegate(); + NetworkClientDelegate.UnsentRequest findCoordRequest = newUnsentFindCoordinatorRequest(); + assertNull(findCoordRequest.timer()); + + // NetworkClientDelegate#add + ncd.add(findCoordRequest); + assertEquals(1, ncd.unsentRequests().size()); + assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); + + // NetworkClientDelegate#addAll + ncd.addAll(Collections.singletonList(findCoordRequest)); + assertEquals(1, ncd.unsentRequests().size()); + assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); + } + public NetworkClientDelegate newNetworkClientDelegate() { LogContext logContext = new LogContext(); Properties properties = new Properties(); @@ -132,15 +150,14 @@ public NetworkClientDelegate newNetworkClientDelegate() { return new NetworkClientDelegate(this.time, new ConsumerConfig(properties), logContext, this.client); } - public NetworkClientDelegate.UnsentRequest newUnsentFindCoordinatorRequest(long timeoutMs) { + public NetworkClientDelegate.UnsentRequest newUnsentFindCoordinatorRequest() { Objects.requireNonNull(GROUP_ID); NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() - .setKey(GROUP_ID) - .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) - ), - Optional.empty(), - time.timer(timeoutMs) + new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() + .setKey(GROUP_ID) + .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) + ), + Optional.empty() ); return req; } @@ -154,4 +171,4 @@ public void prepareFindCoordinatorResponse(Errors error) { private Node mockNode() { return new Node(0, "localhost", 99); } -} +} \ No newline at end of file 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 6573ca50b31bc..b0e12b8526fc5 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 @@ -45,7 +45,6 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -102,7 +101,6 @@ public class OffsetsRequestManagerTest { private static final IsolationLevel DEFAULT_ISOLATION_LEVEL = IsolationLevel.READ_COMMITTED; private static final int RETRY_BACKOFF_MS = 500; private static final int REQUEST_TIMEOUT_MS = 500; - private static final int DEFAULT_API_TIMEOUT_MS = 60000; @BeforeEach public void setup() { @@ -114,51 +112,48 @@ public void setup() { time = new MockTime(0); apiVersions = mock(ApiVersions.class); requestManager = new OffsetsRequestManager( - subscriptionState, - metadata, - DEFAULT_ISOLATION_LEVEL, - time, - RETRY_BACKOFF_MS, - REQUEST_TIMEOUT_MS, - apiVersions, - mock(NetworkClientDelegate.class), - backgroundEventHandler, - logContext + subscriptionState, + metadata, + DEFAULT_ISOLATION_LEVEL, + time, + RETRY_BACKOFF_MS, + REQUEST_TIMEOUT_MS, + apiVersions, + mock(NetworkClientDelegate.class), + backgroundEventHandler, + logContext ); } @Test public void testListOffsetsRequest_Success() throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); Map expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(result, expectedOffsets, timer); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets); } @Test public void testListOffsetsWaitingForMetadataUpdate_Timeout() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets(timestampsToSearch, false, timer); + requestManager.fetchOffsets(timestampsToSearch, false); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); assertEquals(0, res.unsentRequests.size()); // Metadata update not happening within the time boundaries of the request future, so // future should time out. @@ -167,7 +162,7 @@ public void testListOffsetsWaitingForMetadataUpdate_Timeout() { @Test public void testListOffsetsRequestMultiplePartitions() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = new HashMap<>(); timestampsToSearch.put(TEST_PARTITION_1, ListOffsetsRequest.EARLIEST_TIMESTAMP); timestampsToSearch.put(TEST_PARTITION_2, ListOffsetsRequest.EARLIEST_TIMESTAMP); @@ -177,30 +172,26 @@ public void testListOffsetsRequestMultiplePartitions() throws ExecutionException partitionLeaders.put(TEST_PARTITION_1, LEADER_1); partitionLeaders.put(TEST_PARTITION_2, LEADER_1); mockSuccessfulRequest(partitionLeaders); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); Map expectedOffsets = timestampsToSearch.entrySet().stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); - verifySuccessfulPollAndResponseReceived(result, expectedOffsets, timer); + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets); } @Test public void testListOffsetsRequestEmpty() throws ExecutionException, InterruptedException { - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( - Collections.emptyMap(), - false, - timer); + Collections.emptyMap(), + false); assertEquals(0, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); assertTrue(pollResult.unsentRequests.isEmpty()); assertEquals(0, requestManager.requestsToRetry()); @@ -213,51 +204,47 @@ public void testListOffsetsRequestEmpty() throws ExecutionException, Interrupted @Test public void testListOffsetsRequestUnknownOffset() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); List topicResponses = Collections.singletonList( - mockUnknownOffsetResponse(TEST_PARTITION_1)); + mockUnknownOffsetResponse(TEST_PARTITION_1)); - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, topicResponses); clientResponse.onComplete(); Map expectedOffsets = - Collections.singletonMap(TEST_PARTITION_1, null); + Collections.singletonMap(TEST_PARTITION_1, null); verifyRequestSuccessfullyCompleted(result, expectedOffsets); } @Test public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets(timestampsToSearch, - false, - timer); + requestManager.fetchOffsets(timestampsToSearch, + false); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); assertEquals(0, res.unsentRequests.size()); assertFalse(fetchOffsetsFuture.isDone()); @@ -268,36 +255,34 @@ public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws Execu assertEquals(1, requestManager.requestsToSend()); Map expectedOffsets = Collections.singletonMap( - TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets, timer); + TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); } @ParameterizedTest @MethodSource("retriableErrors") public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent to single broker - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res); assertFalse(fetchOffsetsFuture.isDone()); // Response received with error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, - Collections.singletonMap(TEST_PARTITION_1, error)); + unsentRequest, + Collections.singletonMap(TEST_PARTITION_1, error)); clientResponse.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); assertEquals(1, requestManager.requestsToRetry()); @@ -309,45 +294,43 @@ public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throw assertEquals(1, requestManager.requestsToSend()); Map expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); - verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets, timer); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); } @Test public void testRequestNotSupportedErrorReturnsNullOffset() throws ExecutionException, - InterruptedException { + InterruptedException { testResponseWithErrorCodeAndUnknownOffsets(Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT); } @Test public void testRequestWithUnknownOffsetInResponseReturnsNullOffset() throws ExecutionException, - InterruptedException { + InterruptedException { testResponseWithErrorCodeAndUnknownOffsets(Errors.NONE); } private void testResponseWithErrorCodeAndUnknownOffsets(Errors error) throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent to single broker - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res); assertFalse(fetchOffsetsFuture.isDone()); // Response received with error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, - Collections.singletonMap(TEST_PARTITION_1, error)); + unsentRequest, + Collections.singletonMap(TEST_PARTITION_1, error)); clientResponse.onComplete(); // Null offsets should be returned for each partition @@ -362,23 +345,21 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E timestampsToSearch.put(TEST_PARTITION_2, ListOffsetsRequest.EARLIEST_TIMESTAMP); Map expectedOffsets = timestampsToSearch.entrySet().stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); // List offsets request to 2 brokers successfully built Map partitionLeaders = new HashMap<>(); partitionLeaders.put(TEST_PARTITION_1, LEADER_1); partitionLeaders.put(TEST_PARTITION_2, LEADER_2); mockSuccessfulRequest(partitionLeaders); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + timestampsToSearch, + false); assertEquals(2, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Requests successfully sent to both brokers - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res, 2); assertFalse(fetchOffsetsFuture.isDone()); @@ -386,13 +367,13 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E // broker but retriable UNKNOWN_LEADER_EPOCH received from second broker. NetworkClientDelegate.UnsentRequest unsentRequest1 = res.unsentRequests.get(0); ClientResponse clientResponse1 = buildClientResponse( - unsentRequest1, - Collections.singletonMap(TEST_PARTITION_1, expectedOffsets.get(TEST_PARTITION_1))); + unsentRequest1, + Collections.singletonMap(TEST_PARTITION_1, expectedOffsets.get(TEST_PARTITION_1))); clientResponse1.onComplete(); NetworkClientDelegate.UnsentRequest unsentRequest2 = res.unsentRequests.get(1); ClientResponse clientResponse2 = buildClientResponseWithErrors( - unsentRequest2, - Collections.singletonMap(TEST_PARTITION_2, Errors.UNKNOWN_LEADER_EPOCH)); + unsentRequest2, + Collections.singletonMap(TEST_PARTITION_2, Errors.UNKNOWN_LEADER_EPOCH)); clientResponse2.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); @@ -405,11 +386,11 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E assertEquals(1, requestManager.requestsToSend()); // Following poll should send the request and get a successful response - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, - Collections.singletonMap(TEST_PARTITION_2, expectedOffsets.get(TEST_PARTITION_2))); + Collections.singletonMap(TEST_PARTITION_2, expectedOffsets.get(TEST_PARTITION_2))); clientResponse.onComplete(); // Verify global result with the offset initially retrieved, and the offset that @@ -420,27 +401,25 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E @Test public void testRequestFailedResponse_NonRetriableAuthError() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res); // Response received with non-retriable auth error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.TOPIC_AUTHORIZATION_FAILED)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.TOPIC_AUTHORIZATION_FAILED)); clientResponse.onComplete(); verifyRequestCompletedWithErrorResponse(fetchOffsetsFuture, TopicAuthorizationException.class); @@ -451,27 +430,25 @@ public void testRequestFailedResponse_NonRetriableAuthError() { @Test public void testRequestFailedResponse_NonRetriableErrorTimeout() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res); // Response received NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.BROKER_NOT_AVAILABLE)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.BROKER_NOT_AVAILABLE)); clientResponse.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); @@ -485,27 +462,25 @@ public void testRequestFailedResponse_NonRetriableErrorTimeout() { @Test public void testRequestFails_AuthenticationException() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false, - timer); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); // Request successfully sent - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(res); // Response received with auth error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = - buildClientResponseWithAuthenticationException(unsentRequest); + buildClientResponseWithAuthenticationException(unsentRequest); clientResponse.onComplete(); // Request completed with error. Nothing pending to be sent or retried @@ -516,19 +491,17 @@ public void testRequestFails_AuthenticationException() { @Test public void testResetPositionsSendNoRequestIfNoPartitionsNeedingReset() { - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.emptySet()); - requestManager.resetPositionsIfNeeded(timer); + when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.emptySet()); + requestManager.resetPositionsIfNeeded(); assertEquals(0, requestManager.requestsToSend()); } @Test public void testResetPositionsMissingLeader() { mockFailedRequest_MissingLeader(); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST); - requestManager.resetPositionsIfNeeded(timer); + requestManager.resetPositionsIfNeeded(); verify(metadata).requestUpdate(true); assertEquals(0, requestManager.requestsToSend()); } @@ -542,26 +515,25 @@ public void testResetPositionsSuccess_NoLeaderEpochInResponse() { @Test public void testResetPositionsSuccess_LeaderEpochInResponse() { Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(5)); + Optional.of(5)); testResetPositionsSuccessWithLeaderEpoch(leaderAndEpoch); verify(metadata).updateLastSeenEpochIfNewer(TEST_PARTITION_1, leaderAndEpoch.epoch.get()); } @Test public void testResetOffsetsAuthorizationFailure() { - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); - requestManager.resetPositionsIfNeeded(timer); + requestManager.resetPositionsIfNeeded(); // Reset positions response with TopicAuthorizationException - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); Errors topicAuthorizationFailedError = Errors.TOPIC_AUTHORIZATION_FAILED; ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError)); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); @@ -572,7 +544,7 @@ public void testResetOffsetsAuthorizationFailure() { // Following resetPositions should enqueue the previous exception in the background event queue // without performing any request - assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded(timer)); + assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded()); assertEquals(0, requestManager.requestsToSend()); // Check that the event was enqueued during resetPositionsIfNeeded @@ -594,25 +566,24 @@ public void testValidatePositionsSuccess() { int currentOffset = 5; int expectedEndOffset = 100; Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(3)); + Optional.of(3)); TopicPartition tp = TEST_PARTITION_1; SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(currentOffset, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - requestManager.validatePositionsIfNeeded(timer); + requestManager.validatePositionsIfNeeded(); assertEquals(1, requestManager.requestsToSend(), "Invalid request count"); verify(subscriptionState).setNextAllowedRetry(any(), anyLong()); // Validate positions response with end offsets when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(LEADER_1, leaderAndEpoch.epoch)); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = pollResult.unsentRequests.get(0); ClientResponse clientResponse = buildOffsetsForLeaderEpochResponse(unsentRequest, - Collections.singletonList(tp), expectedEndOffset); + Collections.singletonList(tp), expectedEndOffset); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); assertFalse(unsentRequest.future().isCompletedExceptionally()); @@ -621,16 +592,15 @@ public void testValidatePositionsSuccess() { @Test public void testValidatePositionsMissingLeader() { - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(Node.noNode()), - Optional.of(5)); + Optional.of(5)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(5L, - Optional.of(10), leaderAndEpoch); - when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + Optional.of(10), leaderAndEpoch); + when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); NodeApiVersions nodeApiVersions = NodeApiVersions.create(); when(apiVersions.get(LEADER_1.idString())).thenReturn(nodeApiVersions); - requestManager.validatePositionsIfNeeded(timer); + requestManager.validatePositionsIfNeeded(); verify(metadata).requestUpdate(true); assertEquals(0, requestManager.requestsToSend()); } @@ -638,19 +608,18 @@ public void testValidatePositionsMissingLeader() { @Test public void testValidatePositionsFailureWithUnrecoverableAuthException() { Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(5)); + Optional.of(5)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(5L, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - requestManager.validatePositionsIfNeeded(timer); + requestManager.validatePositionsIfNeeded(); // Validate positions response with TopicAuthorizationException - NetworkClientDelegate.PollResult res = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = - buildOffsetsForLeaderEpochResponseWithErrors(unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); + buildOffsetsForLeaderEpochResponseWithErrors(unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); @@ -658,7 +627,7 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { // Following validatePositions should raise the previous exception without performing any // request - assertThrows(TopicAuthorizationException.class, () -> requestManager.validatePositionsIfNeeded(timer)); + assertThrows(TopicAuthorizationException.class, () -> requestManager.validatePositionsIfNeeded()); assertEquals(0, requestManager.requestsToSend()); } @@ -666,32 +635,30 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { public void testValidatePositionsAbortIfNoApiVersionsToCheckAgainstThenRecovers() { int currentOffset = 5; Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(3)); + Optional.of(3)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(currentOffset, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); // No api version info initially available when(apiVersions.get(LEADER_1.idString())).thenReturn(null); - requestManager.validatePositionsIfNeeded(timer); + requestManager.validatePositionsIfNeeded(); assertEquals(0, requestManager.requestsToSend(), "Invalid request count"); verify(subscriptionState, never()).completeValidation(TEST_PARTITION_1); verify(subscriptionState, never()).setNextAllowedRetry(any(), anyLong()); // Api version updated, next validate positions should successfully build the request when(apiVersions.get(LEADER_1.idString())).thenReturn(NodeApiVersions.create()); - when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); - requestManager.validatePositionsIfNeeded(timer); + requestManager.validatePositionsIfNeeded(); assertEquals(1, requestManager.requestsToSend(), "Invalid request count"); } private void mockSuccessfulBuildRequestForValidatingPositions(SubscriptionState.FetchPosition position, Node leader) { - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingValidation(timer.currentTimeMs())).thenReturn(Collections.singleton(TEST_PARTITION_1)); + when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); NodeApiVersions nodeApiVersions = NodeApiVersions.create(); when(apiVersions.get(leader.idString())).thenReturn(nodeApiVersions); @@ -703,18 +670,17 @@ private void testResetPositionsSuccessWithLeaderEpoch(Metadata.LeaderAndEpoch le OffsetResetStrategy strategy = OffsetResetStrategy.EARLIEST; long offset = 5L; Map expectedOffsets = Collections.singletonMap(tp, - new OffsetAndTimestamp(offset, 1L, leaderAndEpoch.epoch)); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - when(subscriptionState.partitionsNeedingReset(timer.currentTimeMs())).thenReturn(Collections.singleton(tp)); + new OffsetAndTimestamp(offset, 1L, leaderAndEpoch.epoch)); + when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(tp)); when(subscriptionState.resetStrategy(any())).thenReturn(strategy); mockSuccessfulRequest(Collections.singletonMap(tp, leader)); - requestManager.resetPositionsIfNeeded(timer); + requestManager.resetPositionsIfNeeded(); assertEquals(1, requestManager.requestsToSend()); // Reset positions response with offsets when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(leader, leaderAndEpoch.epoch)); - NetworkClientDelegate.PollResult pollResult = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = pollResult.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, expectedOffsets); clientResponse.onComplete(); @@ -723,36 +689,36 @@ private void testResetPositionsSuccessWithLeaderEpoch(Metadata.LeaderAndEpoch le } private ListOffsetsResponseData.ListOffsetsTopicResponse mockUnknownOffsetResponse( - TopicPartition tp) { + TopicPartition tp) { return new ListOffsetsResponseData.ListOffsetsTopicResponse() - .setName(tp.topic()) - .setPartitions(Collections.singletonList(new ListOffsetsResponseData.ListOffsetsPartitionResponse() - .setPartitionIndex(tp.partition()) - .setErrorCode(Errors.NONE.code()) - .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))); + .setName(tp.topic()) + .setPartitions(Collections.singletonList(new ListOffsetsResponseData.ListOffsetsPartitionResponse() + .setPartitionIndex(tp.partition()) + .setErrorCode(Errors.NONE.code()) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))); } private static Stream retriableErrors() { return Stream.of( - Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER), - Arguments.of(Errors.REPLICA_NOT_AVAILABLE), - Arguments.of(Errors.KAFKA_STORAGE_ERROR), - Arguments.of(Errors.OFFSET_NOT_AVAILABLE), - Arguments.of(Errors.LEADER_NOT_AVAILABLE), - Arguments.of(Errors.FENCED_LEADER_EPOCH), - Arguments.of(Errors.BROKER_NOT_AVAILABLE), - Arguments.of(Errors.INVALID_REQUEST), - Arguments.of(Errors.UNKNOWN_LEADER_EPOCH), - Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION)); + Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER), + Arguments.of(Errors.REPLICA_NOT_AVAILABLE), + Arguments.of(Errors.KAFKA_STORAGE_ERROR), + Arguments.of(Errors.OFFSET_NOT_AVAILABLE), + Arguments.of(Errors.LEADER_NOT_AVAILABLE), + Arguments.of(Errors.FENCED_LEADER_EPOCH), + Arguments.of(Errors.BROKER_NOT_AVAILABLE), + Arguments.of(Errors.INVALID_REQUEST), + Arguments.of(Errors.UNKNOWN_LEADER_EPOCH), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION)); } private void verifySuccessfulPollAndResponseReceived( - CompletableFuture> actualResult, - Map expectedResult, Timer timer) throws ExecutionException, - InterruptedException { + CompletableFuture> actualResult, + Map expectedResult) throws ExecutionException, + InterruptedException { // Following poll should send the request and get a response - NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, expectedResult); @@ -763,7 +729,7 @@ private void verifySuccessfulPollAndResponseReceived( private void mockSuccessfulRequest(Map partitionLeaders) { partitionLeaders.forEach((tp, broker) -> { when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(broker, - Metadata.LeaderAndEpoch.noLeaderOrEpoch().epoch)); + Metadata.LeaderAndEpoch.noLeaderOrEpoch().epoch)); when(subscriptionState.isAssigned(tp)).thenReturn(true); }); when(metadata.fetch()).thenReturn(testClusterMetadata(partitionLeaders)); @@ -771,7 +737,7 @@ private void mockSuccessfulRequest(Map partitionLeaders) { private void mockFailedRequest_MissingLeader() { when(metadata.currentLeader(any(TopicPartition.class))).thenReturn( - new Metadata.LeaderAndEpoch(Optional.empty(), Optional.of(1))); + new Metadata.LeaderAndEpoch(Optional.empty(), Optional.of(1))); when(subscriptionState.isAssigned(any(TopicPartition.class))).thenReturn(true); } @@ -787,8 +753,8 @@ private void verifySuccessfulPollAwaitingResponse(NetworkClientDelegate.PollResu } private void verifyRequestSuccessfullyCompleted( - CompletableFuture> actualResult, - Map expectedResult) throws ExecutionException, InterruptedException { + CompletableFuture> actualResult, + Map expectedResult) throws ExecutionException, InterruptedException { assertEquals(0, requestManager.requestsToRetry()); assertEquals(0, requestManager.requestsToSend()); @@ -799,8 +765,8 @@ private void verifyRequestSuccessfullyCompleted( // Validate that the subscription state has been updated for all non-null offsets retrieved Map validExpectedOffsets = expectedResult.entrySet().stream() - .filter(entry -> entry.getValue() != null) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .filter(entry -> entry.getValue() != null) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); verifySubscriptionStateUpdated(validExpectedOffsets); } @@ -809,7 +775,7 @@ private void verifySubscriptionStateUpdated(Map offsetCaptor = ArgumentCaptor.forClass(Long.class); verify(subscriptionState, times(expectedResult.size())).updateLastStableOffset(tpCaptor.capture(), - offsetCaptor.capture()); + offsetCaptor.capture()); List updatedTp = tpCaptor.getAllValues(); List updatedOffsets = offsetCaptor.getAllValues(); @@ -818,8 +784,8 @@ private void verifySubscriptionStateUpdated(Map updatedOffsets.contains(offsetAndTimestamp.offset())) - .forEach(Assertions::assertTrue); + .map(offsetAndTimestamp -> updatedOffsets.contains(offsetAndTimestamp.offset())) + .forEach(Assertions::assertTrue); } private void verifyRequestCompletedWithErrorResponse(CompletableFuture> actualResult, @@ -836,27 +802,27 @@ private Metadata.LeaderAndEpoch testLeaderEpoch(Node leader, Optional e private Cluster testClusterMetadata(Map partitionLeaders) { List partitions = - partitionLeaders.keySet().stream() - .map(tp -> new PartitionInfo(tp.topic(), tp.partition(), - partitionLeaders.get(tp), null, null)) - .collect(Collectors.toList()); + partitionLeaders.keySet().stream() + .map(tp -> new PartitionInfo(tp.topic(), tp.partition(), + partitionLeaders.get(tp), null, null)) + .collect(Collectors.toList()); return new Cluster("clusterId", partitionLeaders.values(), partitions, - Collections.emptySet(), - Collections.emptySet()); + Collections.emptySet(), + Collections.emptySet()); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionsOffsets) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionsOffsets) { List topicResponses = new - ArrayList<>(); + ArrayList<>(); partitionsOffsets.forEach((tp, offsetAndTimestamp) -> { ListOffsetsResponseData.ListOffsetsTopicResponse topicResponse = ListOffsetsResponse.singletonListOffsetsTopicResponse( - tp, Errors.NONE, - offsetAndTimestamp.timestamp(), - offsetAndTimestamp.offset(), - offsetAndTimestamp.leaderEpoch().orElse(ListOffsetsResponse.UNKNOWN_EPOCH)); + tp, Errors.NONE, + offsetAndTimestamp.timestamp(), + offsetAndTimestamp.offset(), + offsetAndTimestamp.leaderEpoch().orElse(ListOffsetsResponse.UNKNOWN_EPOCH)); topicResponses.add(topicResponse); }); @@ -864,9 +830,9 @@ private ClientResponse buildClientResponse( } private ClientResponse buildOffsetsForLeaderEpochResponse( - final NetworkClientDelegate.UnsentRequest request, - final List partitions, - final int endOffset) { + final NetworkClientDelegate.UnsentRequest request, + final List partitions, + final int endOffset) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); @@ -879,29 +845,29 @@ private ClientResponse buildOffsetsForLeaderEpochResponse( data.topics().add(topic); } topic.partitions().add(new OffsetForLeaderEpochResponseData.EpochEndOffset() - .setPartition(tp.partition()) - .setErrorCode(Errors.NONE.code()) - .setLeaderEpoch(3) - .setEndOffset(endOffset)); + .setPartition(tp.partition()) + .setErrorCode(Errors.NONE.code()) + .setLeaderEpoch(3) + .setEndOffset(endOffset)); }); OffsetsForLeaderEpochResponse response = new OffsetsForLeaderEpochResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), - request.handler(), - "-1", - request.timer().currentTimeMs(), - request.timer().currentTimeMs(), - false, - null, - null, - response + new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response ); } private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionErrors) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionErrors) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); @@ -914,78 +880,78 @@ private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( data.topics().add(topic); } topic.partitions().add(new OffsetForLeaderEpochResponseData.EpochEndOffset() - .setPartition(tp.partition()) - .setErrorCode(partitionErrors.get(tp).code())); + .setPartition(tp.partition()) + .setErrorCode(partitionErrors.get(tp).code())); }); OffsetsForLeaderEpochResponse response = new OffsetsForLeaderEpochResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), - request.handler(), - "-1", - request.timer().currentTimeMs(), - request.timer().currentTimeMs(), - false, - null, - null, - response + new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response ); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final List topicResponses) { + final NetworkClientDelegate.UnsentRequest request, + final List topicResponses) { return buildClientResponse(request, topicResponses, false, null); } private ClientResponse buildClientResponseWithErrors( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionErrors) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionErrors) { List topicResponses = new ArrayList<>(); partitionErrors.forEach((tp, error) -> topicResponses.add(ListOffsetsResponse.singletonListOffsetsTopicResponse( - tp, - error, - ListOffsetsResponse.UNKNOWN_TIMESTAMP, - ListOffsetsResponse.UNKNOWN_OFFSET, - ListOffsetsResponse.UNKNOWN_EPOCH))); + tp, + error, + ListOffsetsResponse.UNKNOWN_TIMESTAMP, + ListOffsetsResponse.UNKNOWN_OFFSET, + ListOffsetsResponse.UNKNOWN_EPOCH))); return buildClientResponse(request, topicResponses, false, null); } private ClientResponse buildClientResponseWithAuthenticationException( - final NetworkClientDelegate.UnsentRequest request) { + final NetworkClientDelegate.UnsentRequest request) { return buildClientResponse(request, Collections.emptyList(), true, - new AuthenticationException("Authentication failed")); + new AuthenticationException("Authentication failed")); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final List topicResponses, - final boolean disconnected, - final AuthenticationException authenticationException) { + final NetworkClientDelegate.UnsentRequest request, + final List topicResponses, + final boolean disconnected, + final AuthenticationException authenticationException) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof ListOffsetsRequest); ListOffsetsRequest offsetFetchRequest = (ListOffsetsRequest) abstractRequest; ListOffsetsResponse response = buildListOffsetsResponse(topicResponses); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), - request.handler(), - "-1", - request.timer().currentTimeMs(), - request.timer().currentTimeMs(), - disconnected, - null, - authenticationException, - response + new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + disconnected, + null, + authenticationException, + response ); } private ListOffsetsResponse buildListOffsetsResponse( - List offsetsTopicResponses) { + List offsetsTopicResponses) { ListOffsetsResponseData responseData = new ListOffsetsResponseData() - .setThrottleTimeMs(0) - .setTopics(offsetsTopicResponses); + .setThrottleTimeMs(0) + .setTopics(offsetsTopicResponses); return new ListOffsetsResponse(responseData); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index e2013bfa1e511..9992b8b7d5c71 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -36,7 +35,6 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -64,7 +62,6 @@ public class TopicMetadataRequestManagerTest { private MockTime time; private TopicMetadataRequestManager topicMetadataRequestManager; - private int defaultApiTimeoutMs; @BeforeEach public void setup() { @@ -74,29 +71,25 @@ public void setup() { props.put(ALLOW_AUTO_CREATE_TOPICS_CONFIG, false); props.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - ConsumerConfig config = new ConsumerConfig(props); this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager( new LogContext(), - config)); - this.defaultApiTimeoutMs = config.getInt(CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG); + new ConsumerConfig(props))); } @Test public void testPoll_SuccessfulRequestTopicMetadata() { String topic = "hello"; - Timer timer = time.timer(defaultApiTimeoutMs); - this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); - timer.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); + this.time.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); } @Test public void testPoll_SuccessfulRequestAllTopicsMetadata() { - Timer timer = time.timer(defaultApiTimeoutMs); - this.topicMetadataRequestManager.requestAllTopicsMetadata(timer); - timer.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.topicMetadataRequestManager.requestAllTopicsMetadata(Long.MAX_VALUE); + this.time.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); } @@ -104,15 +97,13 @@ public void testPoll_SuccessfulRequestAllTopicsMetadata() { @MethodSource("exceptionProvider") public void testTopicExceptionAndInflightRequests(final Errors error, final boolean shouldRetry) { String topic = "hello"; - Timer timer = time.timer(defaultApiTimeoutMs); - this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); - timer.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); + this.time.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - error, - timer)); + error)); List inflights = this.topicMetadataRequestManager.inflightRequests(); if (shouldRetry) { @@ -126,14 +117,12 @@ public void testTopicExceptionAndInflightRequests(final Errors error, final bool @ParameterizedTest @MethodSource("exceptionProvider") public void testAllTopicsExceptionAndInflightRequests(final Errors error, final boolean shouldRetry) { - Timer timer = time.timer(defaultApiTimeoutMs); - this.topicMetadataRequestManager.requestAllTopicsMetadata(timer); - timer.sleep(100); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.topicMetadataRequestManager.requestAllTopicsMetadata(Long.MAX_VALUE); + this.time.sleep(100); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); res.unsentRequests.get(0).future().complete(buildAllTopicsMetadataClientResponse( - res.unsentRequests.get(0), - error, - timer)); + res.unsentRequests.get(0), + error)); List inflights = this.topicMetadataRequestManager.inflightRequests(); if (shouldRetry) { @@ -148,35 +137,33 @@ public void testExpiringRequest() { String topic = "hello"; // Request topic metadata with 1000ms expiration - Timer timer = time.timer(1000); + long now = this.time.milliseconds(); CompletableFuture>> future = - this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); + this.topicMetadataRequestManager.requestTopicMetadata(topic, now + 1000L); assertEquals(1, this.topicMetadataRequestManager.inflightRequests().size()); // Poll the request manager to get the list of requests to send // - fail the request with a RetriableException - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - Errors.REQUEST_TIMED_OUT, - timer)); + Errors.REQUEST_TIMED_OUT)); // Sleep for long enough to exceed the backoff delay but still within the expiration // - fail the request again with a RetriableException - timer.sleep(500); - res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.time.sleep(500); + res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res.unsentRequests.get(0), topic, - Errors.REQUEST_TIMED_OUT, - timer)); + Errors.REQUEST_TIMED_OUT)); // Sleep for long enough to expire the request which should fail - timer.sleep(1000); - res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + this.time.sleep(1000); + res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(0, res.unsentRequests.size()); assertEquals(0, this.topicMetadataRequestManager.inflightRequests().size()); assertTrue(future.isCompletedExceptionally()); @@ -186,9 +173,9 @@ public void testExpiringRequest() { @MethodSource("hardFailureExceptionProvider") public void testHardFailures(Exception exception) { String topic = "hello"; - Timer timer = time.timer(defaultApiTimeoutMs); - this.topicMetadataRequestManager.requestTopicMetadata(topic, timer); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + + this.topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).future().completeExceptionally(exception); @@ -203,66 +190,63 @@ public void testHardFailures(Exception exception) { @Test public void testNetworkTimeout() { String topic = "hello"; - Timer timer = time.timer(defaultApiTimeoutMs); - topicMetadataRequestManager.requestTopicMetadata(topic, timer); - NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + + topicMetadataRequestManager.requestTopicMetadata(topic, Long.MAX_VALUE); + NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res.unsentRequests.size()); - NetworkClientDelegate.PollResult res2 = this.topicMetadataRequestManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res2 = this.topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(0, res2.unsentRequests.size()); // Mimic a network timeout - res.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException()); + res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); - long backoffMs = topicMetadataRequestManager.inflightRequests().get(0).remainingBackoffMs(timer.currentTimeMs()); + long backoffMs = topicMetadataRequestManager.inflightRequests().get(0).remainingBackoffMs(time.milliseconds()); // Sleep for exponential backoff - 1ms - timer.sleep(backoffMs - 1); - res2 = topicMetadataRequestManager.poll(timer.currentTimeMs()); + time.sleep(backoffMs - 1); + res2 = topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(0, res2.unsentRequests.size()); - timer.sleep(1); - res2 = topicMetadataRequestManager.poll(timer.currentTimeMs()); + time.sleep(1); + res2 = topicMetadataRequestManager.poll(this.time.milliseconds()); assertEquals(1, res2.unsentRequests.size()); res2.unsentRequests.get(0).future().complete(buildTopicMetadataClientResponse( res2.unsentRequests.get(0), topic, - Errors.NONE, - timer)); + Errors.NONE)); assertTrue(topicMetadataRequestManager.inflightRequests().isEmpty()); } private ClientResponse buildTopicMetadataClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final String topic, - final Errors error, - final Timer timer) { + final NetworkClientDelegate.UnsentRequest request, + final String topic, + final Errors error) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof MetadataRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; Cluster cluster = mockCluster(3, 0); List topics = new ArrayList<>(); topics.add(new MetadataResponse.TopicMetadata(error, topic, false, - Collections.emptyList())); + Collections.emptyList())); final MetadataResponse metadataResponse = RequestTestUtils.metadataResponse(cluster.nodes(), - cluster.clusterResource().clusterId(), - cluster.controller().id(), - topics); + cluster.clusterResource().clusterId(), + cluster.controller().id(), + topics); return new ClientResponse( - new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), - request.handler(), - "-1", - timer.currentTimeMs(), - timer.currentTimeMs(), - false, - null, - null, - metadataResponse); + new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + metadataResponse); } private ClientResponse buildAllTopicsMetadataClientResponse( final NetworkClientDelegate.UnsentRequest request, - final Errors error, - final Timer timer) { + final Errors error) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof MetadataRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; @@ -280,8 +264,8 @@ private ClientResponse buildAllTopicsMetadataClientResponse( new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), request.handler(), "-1", - timer.currentTimeMs(), - timer.currentTimeMs(), + time.milliseconds(), + time.milliseconds(), false, null, null, @@ -309,9 +293,9 @@ private static Collection exceptionProvider() { private static Collection hardFailureExceptionProvider() { return Arrays.asList( - Arguments.of(new TimeoutException("timeout")), - Arguments.of(new KafkaException("non-retriable exception")), - Arguments.of(new NetworkException("retriable-exception"))); + Arguments.of(new TimeoutException("timeout")), + Arguments.of(new KafkaException("non-retriable exception")), + Arguments.of(new NetworkException("retriable-exception"))); } -} +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 4b44b28a7a79d..44a274627d9be 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -107,9 +107,9 @@ public void testPrepClosingLeaveGroupEvent() { Timer timer = time.timer(Long.MAX_VALUE); LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); - when(membershipManager.leaveGroup(any())).thenReturn(CompletableFuture.completedFuture(null)); + when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); - verify(membershipManager).leaveGroup(any()); + verify(membershipManager).leaveGroup(); assertTrue(event.future().isDone()); } From 5d6664479279e46e791fdb045bb36e7c753708b2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:03:28 -0800 Subject: [PATCH 030/130] Update CoordinatorRequestManager.java --- .../internals/CoordinatorRequestManager.java | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index 4af6ff8340e08..f565c560e1c48 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -62,12 +62,12 @@ public class CoordinatorRequestManager implements RequestManager { private Node coordinator; public CoordinatorRequestManager( - final Time time, - final LogContext logContext, - final long retryBackoffMs, - final long retryBackoffMaxMs, - final BackgroundEventHandler errorHandler, - final String groupId + final Time time, + final LogContext logContext, + final long retryBackoffMs, + final long retryBackoffMaxMs, + final BackgroundEventHandler errorHandler, + final String groupId ) { Objects.requireNonNull(groupId); this.time = time; @@ -75,10 +75,10 @@ public CoordinatorRequestManager( this.backgroundEventHandler = errorHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( - logContext, - CoordinatorRequestManager.class.getSimpleName(), - retryBackoffMs, - retryBackoffMaxMs + logContext, + CoordinatorRequestManager.class.getSimpleName(), + retryBackoffMs, + retryBackoffMaxMs ); } @@ -108,11 +108,11 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) { coordinatorRequestState.onSendAttempt(currentTimeMs); FindCoordinatorRequestData data = new FindCoordinatorRequestData() - .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) - .setKey(this.groupId); + .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) + .setKey(this.groupId); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder(data), - Optional.empty() + new FindCoordinatorRequest.Builder(data), + Optional.empty() ); return unsentRequest.whenComplete((clientResponse, throwable) -> { @@ -134,7 +134,7 @@ NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long curren public void markCoordinatorUnknown(final String cause, final long currentTimeMs) { if (this.coordinator != null) { log.info("Group coordinator {} is unavailable or invalid due to cause: {}. " - + "Rediscovery will be attempted.", this.coordinator, cause); + + "Rediscovery will be attempted.", this.coordinator, cause); this.coordinator = null; timeMarkedUnknownMs = currentTimeMs; totalDisconnectedMin = 0; @@ -149,16 +149,16 @@ public void markCoordinatorUnknown(final String cause, final long currentTimeMs) } private void onSuccessfulResponse( - final long currentTimeMs, - final FindCoordinatorResponseData.Coordinator coordinator + final long currentTimeMs, + final FindCoordinatorResponseData.Coordinator coordinator ) { // use MAX_VALUE - node.id as the coordinator id to allow separate connections // for the coordinator in the underlying network client layer int coordinatorConnectionId = Integer.MAX_VALUE - coordinator.nodeId(); this.coordinator = new Node( - coordinatorConnectionId, - coordinator.host(), - coordinator.port()); + coordinatorConnectionId, + coordinator.host(), + coordinator.port()); log.info("Discovered group coordinator {}", coordinator); coordinatorRequestState.onSuccessfulAttempt(currentTimeMs); } @@ -192,8 +192,8 @@ private void onFailedResponse(final long currentTimeMs, final Throwable exceptio * @param response the response for finding the coordinator. null if an exception is thrown. */ private void onResponse( - final long currentTimeMs, - final FindCoordinatorResponse response + final long currentTimeMs, + final FindCoordinatorResponse response ) { // handles Runtime exception Optional coordinator = response.coordinatorByKey(this.groupId); From fc9d6e8484a9c70ffba7efe31af9b26db567b6e0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:07:40 -0800 Subject: [PATCH 031/130] Updates --- .../clients/consumer/internals/ConsumerNetworkThread.java | 5 +---- .../kafka/clients/consumer/internals/ConsumerUtils.java | 1 - .../clients/consumer/internals/FetchRequestManager.java | 8 +------- .../kafka/clients/consumer/internals/RequestManagers.java | 7 +++---- .../clients/consumer/internals/ConsumerTestBuilder.java | 3 +-- 5 files changed, 6 insertions(+), 18 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index d469c342956ae..aa352cd68a22e 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 @@ -34,7 +34,6 @@ import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; @@ -129,7 +128,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - List> futures = applicationEventProcessor.process(); + applicationEventProcessor.process(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -145,8 +144,6 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); - -// futures.forEach(RelaxedCompletableFuture::attempted); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 8411808176a1e..5a7745b40dd0d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -209,7 +209,6 @@ public static void refreshCommittedOffsets(final Map T getResult(Future future, Timer timer) { try { - timer.update(); return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); } catch (ExecutionException e) { throw maybeWrapAsKafkaException(e.getCause()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java index b00f79b12e61a..bef9f2c9cad5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; @@ -42,7 +41,6 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager { private final NetworkClientDelegate networkClientDelegate; - private final int defaultApiTimeoutMs; FetchRequestManager(final LogContext logContext, final Time time, @@ -52,11 +50,9 @@ public class FetchRequestManager extends AbstractFetch implements RequestManager final FetchBuffer fetchBuffer, final FetchMetricsManager metricsManager, final NetworkClientDelegate networkClientDelegate, - final ApiVersions apiVersions, - final int defaultApiTimeoutMs) { + final ApiVersions apiVersions) { super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time, apiVersions); this.networkClientDelegate = networkClientDelegate; - this.defaultApiTimeoutMs = defaultApiTimeoutMs; } @Override @@ -106,8 +102,6 @@ public PollResult pollOnClose() { private PollResult pollInternal(Map fetchRequests, ResponseHandler successHandler, ResponseHandler errorHandler) { - Timer timer = time.timer(defaultApiTimeoutMs); - List requests = fetchRequests.entrySet().stream().map(entry -> { final Node fetchTarget = entry.getKey(); final FetchSessionHandler.FetchRequestData data = entry.getValue(); 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 2c5c902ce95e2..7a0c488c25c2b 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 @@ -132,13 +132,13 @@ protected RequestManagers create() { final FetchConfig fetchConfig = new FetchConfig(config); long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); - final int defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); + final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions, metadata, fetchConfig.isolationLevel, time, retryBackoffMs, - defaultApiTimeoutMs, + requestTimeoutMs, apiVersions, networkClientDelegate, backgroundEventHandler, @@ -151,8 +151,7 @@ protected RequestManagers create() { fetchBuffer, fetchMetricsManager, networkClientDelegate, - apiVersions, - defaultApiTimeoutMs); + apiVersions); final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(logContext, config); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 513799cabe29b..3d677734c6b25 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -256,8 +256,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA fetchBuffer, metricsManager, networkClientDelegate, - apiVersions, - requestTimeoutMs)); + apiVersions)); this.requestManagers = new RequestManagers(logContext, offsetsRequestManager, topicMetadataRequestManager, From c301d5f370882bf5569d42625efeb9a8aa918d4f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:09:30 -0800 Subject: [PATCH 032/130] Reverting whitespace and other unwanted changes --- .../clients/consumer/internals/HeartbeatRequestManager.java | 5 ++--- .../kafka/clients/consumer/internals/MembershipManager.java | 1 - .../clients/consumer/internals/MembershipManagerImpl.java | 2 +- .../kafka/clients/consumer/internals/RequestManagers.java | 4 +++- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index 9fd0d186336ca..7408f584cdb4f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -274,9 +274,8 @@ private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final long curr private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest(final boolean ignoreResponse) { NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( - new ConsumerGroupHeartbeatRequest.Builder(heartbeatState.buildRequestData()), - coordinatorRequestManager.coordinator() - ); + new ConsumerGroupHeartbeatRequest.Builder(this.heartbeatState.buildRequestData()), + coordinatorRequestManager.coordinator()); if (ignoreResponse) return logResponse(request); else diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index 1f06b52515b16..f95552cc3a353 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; -import org.apache.kafka.common.utils.Timer; import java.util.Map; import java.util.Optional; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index ab34a8ffb91b2..dd035506d4bea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -573,7 +573,7 @@ public CompletableFuture leaveGroup() { if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) { // Member already leaving. No-op and return existing leave group future that will // complete when the ongoing leave operation completes. - return leaveGroupInProgress.orElseThrow(() -> new IllegalStateException("Member already leaving, but no Future available")); + return leaveGroupInProgress.get(); } transitionTo(MemberState.PREPARE_LEAVING); 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 7a0c488c25c2b..2d90a3ad7082e 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 @@ -152,7 +152,9 @@ protected RequestManagers create() { fetchMetricsManager, networkClientDelegate, apiVersions); - final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(logContext, config); + final TopicMetadataRequestManager topic = new TopicMetadataRequestManager( + logContext, + config); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; CoordinatorRequestManager coordinator = null; From e4d0658686c083b60d11eb83df2294b14ded026a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:12:24 -0800 Subject: [PATCH 033/130] Reverting whitespace-only changes --- .../internals/CommitRequestManager.java | 94 +++++++++---------- .../internals/CoordinatorRequestManager.java | 24 ++--- 2 files changed, 59 insertions(+), 59 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 76489a0dc2118..be989ef5d6cc7 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 @@ -94,15 +94,15 @@ public class CommitRequestManager implements RequestManager, MemberStateListener private final MemberInfo memberInfo; public CommitRequestManager( - final Time time, - final LogContext logContext, - final SubscriptionState subscriptions, - final ConsumerConfig config, - final CoordinatorRequestManager coordinatorRequestManager, - final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, - final String groupId, - final Optional groupInstanceId, - final Metrics metrics) { + final Time time, + final LogContext logContext, + final SubscriptionState subscriptions, + final ConsumerConfig config, + final CoordinatorRequestManager coordinatorRequestManager, + final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + final String groupId, + final Optional groupInstanceId, + final Metrics metrics) { this(time, logContext, subscriptions, @@ -327,7 +327,7 @@ private void autoCommitSyncNowWithRetries(OffsetCommitRequestState requestAttemp log.debug("Completed auto-commit of offsets {}", allConsumedOffsets); } else if (throwable instanceof RetriableCommitFailedException) { log.debug("Auto-commit of offsets {} failed due to retriable error: {}", - allConsumedOffsets, throwable.getMessage()); + allConsumedOffsets, throwable.getMessage()); } else { log.warn("Auto-commit of offsets {} failed", allConsumedOffsets, throwable); } @@ -645,9 +645,9 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { } OffsetCommitRequestData data = new OffsetCommitRequestData() - .setGroupId(this.groupId) - .setGroupInstanceId(groupInstanceId.orElse(null)) - .setTopics(new ArrayList<>(requestTopicDataMap.values())); + .setGroupId(this.groupId) + .setGroupInstanceId(groupInstanceId.orElse(null)) + .setTopics(new ArrayList<>(requestTopicDataMap.values())); if (memberInfo.memberId.isPresent()) { data = data.setMemberId(memberInfo.memberId.get()); } @@ -909,20 +909,20 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { OffsetFetchRequest.Builder builder; if (memberInfo.memberId.isPresent() && memberInfo.memberEpoch.isPresent()) { builder = new OffsetFetchRequest.Builder( - groupId, - memberInfo.memberId.get(), - memberInfo.memberEpoch.get(), - true, - new ArrayList<>(this.requestedPartitions), - throwOnFetchStableOffsetUnsupported); + groupId, + memberInfo.memberId.get(), + memberInfo.memberEpoch.get(), + true, + new ArrayList<>(this.requestedPartitions), + throwOnFetchStableOffsetUnsupported); } else { // Building request without passing member ID/epoch to leave the logic to choose // default values when not present on the request builder. builder = new OffsetFetchRequest.Builder( - groupId, - true, - new ArrayList<>(this.requestedPartitions), - throwOnFetchStableOffsetUnsupported); + groupId, + true, + new ArrayList<>(this.requestedPartitions), + throwOnFetchStableOffsetUnsupported); } return buildRequestWithResponseHandling(builder); } @@ -1005,7 +1005,7 @@ private void onSuccess(final long currentTimeMs, final OffsetFetchResponse response) { Set unauthorizedTopics = null; Map responseData = - response.partitionDataMap(groupId); + response.partitionDataMap(groupId); Map offsets = new HashMap<>(responseData.size()); Set unstableTxnOffsetTopicPartitions = new HashSet<>(); for (Map.Entry entry : responseData.entrySet()) { @@ -1030,7 +1030,7 @@ private void onSuccess(final long currentTimeMs, // Fail with a non-retriable KafkaException for all unexpected partition // errors (even if they are retriable) future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " + - "response for partition " + tp + ": " + error.message())); + "response for partition " + tp + ": " + error.message())); return; } } else if (partitionData.offset >= 0) { @@ -1048,10 +1048,10 @@ private void onSuccess(final long currentTimeMs, } else if (!unstableTxnOffsetTopicPartitions.isEmpty()) { // TODO: Optimization question: Do we need to retry all partitions upon a single partition error? log.info("The following partitions still have unstable offsets " + - "which are not cleared on the broker side: {}" + - ", this could be either " + - "transactional offsets waiting for completion, or " + - "normal offsets waiting for replication after appending to local log", unstableTxnOffsetTopicPartitions); + "which are not cleared on the broker side: {}" + + ", this could be either " + + "transactional offsets waiting for completion, or " + + "normal offsets waiting for replication after appending to local log", unstableTxnOffsetTopicPartitions); future.completeExceptionally(new UnstableOffsetCommitException("There are " + "unstable offsets for the requested topic partitions")); } else { @@ -1061,7 +1061,7 @@ private void onSuccess(final long currentTimeMs, } private CompletableFuture> chainFuture( - final CompletableFuture> otherFuture) { + final CompletableFuture> otherFuture) { return this.future.whenComplete((r, t) -> { if (t != null) { otherFuture.completeExceptionally(t); @@ -1074,12 +1074,12 @@ private CompletableFuture> chainFuture( @Override public String toString() { return "OffsetFetchRequestState{" + - "requestedPartitions=" + requestedPartitions + - ", memberId=" + memberInfo.memberId.orElse("undefined") + - ", memberEpoch=" + (memberInfo.memberEpoch.isPresent() ? memberInfo.memberEpoch.get() : "undefined") + - ", future=" + future + - ", " + toStringBase() + - '}'; + "requestedPartitions=" + requestedPartitions + + ", memberId=" + memberInfo.memberId.orElse("undefined") + + ", memberEpoch=" + (memberInfo.memberEpoch.isPresent() ? memberInfo.memberEpoch.get() : "undefined") + + ", future=" + future + + ", " + toStringBase() + + '}'; } } @@ -1122,9 +1122,9 @@ OffsetCommitRequestState addOffsetCommitRequest(OffsetCommitRequestState request */ private CompletableFuture> addOffsetFetchRequest(final OffsetFetchRequestState request) { Optional dupe = - unsentOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny(); + unsentOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny(); Optional inflight = - inflightOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny(); + inflightOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny(); if (dupe.isPresent() || inflight.isPresent()) { log.info("Duplicated OffsetFetchRequest: " + request.requestedPartitions); @@ -1153,16 +1153,16 @@ List drain(final long currentTimeMs) { // Add all unsent offset commit requests to the unsentRequests list unsentRequests.addAll( - unsentOffsetCommits.stream() - .filter(request -> request.canSendRequest(currentTimeMs)) - .peek(request -> request.onSendAttempt(currentTimeMs)) - .map(OffsetCommitRequestState::toUnsentRequest) - .collect(Collectors.toList())); + unsentOffsetCommits.stream() + .filter(request -> request.canSendRequest(currentTimeMs)) + .peek(request -> request.onSendAttempt(currentTimeMs)) + .map(OffsetCommitRequestState::toUnsentRequest) + .collect(Collectors.toList())); // Partition the unsent offset fetch requests into sendable and non-sendable lists Map> partitionedBySendability = - unsentOffsetFetches.stream() - .collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs))); + unsentOffsetFetches.stream() + .collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs))); failAndRemoveExpiredFetchRequests(currentTimeMs); @@ -1221,8 +1221,8 @@ private static class AutoCommitState { private boolean hasInflightCommit; public AutoCommitState( - final Time time, - final long autoCommitInterval) { + final Time time, + final long autoCommitInterval) { this.autoCommitInterval = autoCommitInterval; this.timer = time.timer(autoCommitInterval); this.hasInflightCommit = false; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index f565c560e1c48..38a7a639f2322 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -62,12 +62,12 @@ public class CoordinatorRequestManager implements RequestManager { private Node coordinator; public CoordinatorRequestManager( - final Time time, - final LogContext logContext, - final long retryBackoffMs, - final long retryBackoffMaxMs, - final BackgroundEventHandler errorHandler, - final String groupId + final Time time, + final LogContext logContext, + final long retryBackoffMs, + final long retryBackoffMaxMs, + final BackgroundEventHandler errorHandler, + final String groupId ) { Objects.requireNonNull(groupId); this.time = time; @@ -111,8 +111,8 @@ NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long curren .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) .setKey(this.groupId); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder(data), - Optional.empty() + new FindCoordinatorRequest.Builder(data), + Optional.empty() ); return unsentRequest.whenComplete((clientResponse, throwable) -> { @@ -149,8 +149,8 @@ public void markCoordinatorUnknown(final String cause, final long currentTimeMs) } private void onSuccessfulResponse( - final long currentTimeMs, - final FindCoordinatorResponseData.Coordinator coordinator + final long currentTimeMs, + final FindCoordinatorResponseData.Coordinator coordinator ) { // use MAX_VALUE - node.id as the coordinator id to allow separate connections // for the coordinator in the underlying network client layer @@ -192,8 +192,8 @@ private void onFailedResponse(final long currentTimeMs, final Throwable exceptio * @param response the response for finding the coordinator. null if an exception is thrown. */ private void onResponse( - final long currentTimeMs, - final FindCoordinatorResponse response + final long currentTimeMs, + final FindCoordinatorResponse response ) { // handles Runtime exception Optional coordinator = response.coordinatorByKey(this.groupId); From 669edb94f2ef8698a8319a49fccd8bfc00dd04af Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:14:34 -0800 Subject: [PATCH 034/130] Whitespace --- .../internals/CommitRequestManager.java | 3 +- .../internals/CoordinatorRequestManager.java | 3 +- .../internals/NetworkClientDelegate.java | 39 ++--- .../internals/OffsetsRequestManager.java | 145 +++++++++--------- 4 files changed, 97 insertions(+), 93 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 be989ef5d6cc7..af06be1183424 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 @@ -1263,4 +1263,5 @@ static class MemberInfo { this.memberEpoch = Optional.empty(); } } -} \ No newline at end of file +} + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index 38a7a639f2322..e181589e8a757 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -219,4 +219,5 @@ private void onResponse( public Optional coordinator() { return Optional.ofNullable(this.coordinator); } -} \ No newline at end of file +} + 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 0637b951e0075..228ee42080721 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 @@ -64,10 +64,10 @@ public class NetworkClientDelegate implements AutoCloseable { private final long retryBackoffMs; public NetworkClientDelegate( - final Time time, - final ConsumerConfig config, - final LogContext logContext, - final KafkaClient client) { + final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client) { this.time = time; this.client = client; this.log = logContext.logger(getClass()); @@ -310,11 +310,11 @@ Optional node() { @Override public String toString() { return "UnsentRequest{" + - "requestBuilder=" + requestBuilder + - ", handler=" + handler + - ", node=" + node + - ", timer=" + timer + - '}'; + "requestBuilder=" + requestBuilder + + ", handler=" + handler + + ", node=" + node + + ", timer=" + timer + + '}'; } } @@ -376,17 +376,18 @@ public static Supplier supplier(final Time time, @Override protected NetworkClientDelegate create() { KafkaClient client = ClientUtils.createNetworkClient(config, - metrics, - CONSUMER_METRIC_GROUP_PREFIX, - logContext, - apiVersions, - time, - CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, - metadata, - fetchMetricsManager.throttleTimeSensor(), - clientTelemetrySender); + metrics, + CONSUMER_METRIC_GROUP_PREFIX, + logContext, + apiVersions, + time, + CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, + metadata, + fetchMetricsManager.throttleTimeSensor(), + clientTelemetrySender); return new NetworkClientDelegate(time, config, logContext, client); } }; } -} \ No newline at end of file +} + 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 bcc66005a0af5..b32516edff38b 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 @@ -119,7 +119,7 @@ public OffsetsRequestManager(final SubscriptionState subscriptionState, this.networkClientDelegate = networkClientDelegate; this.backgroundEventHandler = backgroundEventHandler; this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState, - time, retryBackoffMs, apiVersions); + 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. @@ -152,33 +152,33 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * processed, following a call to {@link #poll(long)} */ public CompletableFuture> fetchOffsets( - final Map timestampsToSearch, - final boolean requireTimestamps) { + final Map timestampsToSearch, + final boolean requireTimestamps) { if (timestampsToSearch.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyMap()); } metadata.addTransientTopics(OffsetFetcherUtils.topicsForPartitions(timestampsToSearch.keySet())); ListOffsetsRequestState listOffsetsRequestState = new ListOffsetsRequestState( - timestampsToSearch, - requireTimestamps, - offsetFetcherUtils, - isolationLevel); + timestampsToSearch, + requireTimestamps, + offsetFetcherUtils, + isolationLevel); listOffsetsRequestState.globalResult.whenComplete((result, error) -> { metadata.clearTransientTopics(); if (error != null) { log.debug("Fetch offsets completed with error for partitions and timestamps {}.", - timestampsToSearch, error); + timestampsToSearch, error); } else { log.debug("Fetch offsets completed successfully for partitions and timestamps {}." + - " Result {}", timestampsToSearch, result); + " Result {}", timestampsToSearch, result); } }); fetchOffsetsByTimes(timestampsToSearch, requireTimestamps, listOffsetsRequestState); return listOffsetsRequestState.globalResult.thenApply(result -> - OffsetFetcherUtils.buildOffsetsForTimesResult(timestampsToSearch, result.fetchedOffsets)); + OffsetFetcherUtils.buildOffsetsForTimesResult(timestampsToSearch, result.fetchedOffsets)); } /** @@ -223,7 +223,7 @@ public CompletableFuture resetPositionsIfNeeded() { */ public CompletableFuture validatePositionsIfNeeded() { Map partitionsToValidate = - offsetFetcherUtils.getPartitionsToValidate(); + offsetFetcherUtils.getPartitionsToValidate(); if (partitionsToValidate.isEmpty()) { return CompletableFuture.completedFuture(null); } @@ -245,7 +245,7 @@ private void fetchOffsetsByTimes(final Map timestampsToSea } try { List unsentRequests = buildListOffsetsRequests( - timestampsToSearch, requireTimestamps, listOffsetsRequestState); + timestampsToSearch, requireTimestamps, listOffsetsRequestState); requestsToSend.addAll(unsentRequests); } catch (StaleMetadataException e) { requestsToRetry.add(listOffsetsRequestState); @@ -261,7 +261,7 @@ public void onUpdate(ClusterResource clusterResource) { requestsToRetry.clear(); requestsToProcess.forEach(requestState -> { Map timestampsToSearch = - new HashMap<>(requestState.remainingToSearch); + new HashMap<>(requestState.remainingToSearch); requestState.remainingToSearch.clear(); fetchOffsetsByTimes(timestampsToSearch, requestState.requireTimestamps, requestState); }); @@ -278,12 +278,12 @@ public void onUpdate(ClusterResource clusterResource) { * that can be polled to obtain the corresponding timestamps and offsets. */ private List buildListOffsetsRequests( - final Map timestampsToSearch, - final boolean requireTimestamps, - final ListOffsetsRequestState listOffsetsRequestState) { + final Map timestampsToSearch, + final boolean requireTimestamps, + final ListOffsetsRequestState listOffsetsRequestState) { log.debug("Building ListOffsets request for partitions {}", timestampsToSearch); Map> timestampsToSearchByNode = - groupListOffsetRequests(timestampsToSearch, Optional.of(listOffsetsRequestState)); + groupListOffsetRequests(timestampsToSearch, Optional.of(listOffsetsRequestState)); if (timestampsToSearchByNode.isEmpty()) { throw new StaleMetadataException(); } @@ -296,12 +296,12 @@ private List buildListOffsetsRequests( listOffsetsRequestState.fetchedOffsets.putAll(multiNodeResult.fetchedOffsets); listOffsetsRequestState.addPartitionsToRetry(multiNodeResult.partitionsToRetry); offsetFetcherUtils.updateSubscriptionState(multiNodeResult.fetchedOffsets, - isolationLevel); + isolationLevel); if (listOffsetsRequestState.remainingToSearch.size() == 0) { ListOffsetResult listOffsetResult = - new ListOffsetResult(listOffsetsRequestState.fetchedOffsets, - listOffsetsRequestState.remainingToSearch.keySet()); + new ListOffsetResult(listOffsetsRequestState.fetchedOffsets, + listOffsetsRequestState.remainingToSearch.keySet()); listOffsetsRequestState.globalResult.complete(listOffsetResult); } else { requestsToRetry.add(listOffsetsRequestState); @@ -316,10 +316,10 @@ private List buildListOffsetsRequests( Node node = entry.getKey(); CompletableFuture partialResult = buildListOffsetRequestToNode( - node, - entry.getValue(), - requireTimestamps, - unsentRequests); + node, + entry.getValue(), + requireTimestamps, + unsentRequests); partialResult.whenComplete((result, error) -> { if (error != null) { @@ -337,35 +337,35 @@ private List buildListOffsetsRequests( * target timestamps. This also adds the request to the list of unsentRequests. */ private CompletableFuture buildListOffsetRequestToNode( - Node node, - Map targetTimes, - boolean requireTimestamps, - List unsentRequests) { + Node node, + Map targetTimes, + boolean requireTimestamps, + List unsentRequests) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamps, isolationLevel, false) - .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); + .forConsumer(requireTimestamps, isolationLevel, false) + .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); log.debug("Creating ListOffset request {} for broker {} to reset positions", builder, - node); + node); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - builder, - Optional.ofNullable(node)); + builder, + Optional.ofNullable(node)); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, error) -> { if (error != null) { log.debug("Sending ListOffset request {} to broker {} failed", - builder, - node, - error); + builder, + node, + error); result.completeExceptionally(error); } else { ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); try { ListOffsetResult listOffsetResult = - offsetFetcherUtils.handleListOffsetResponse(lor); + offsetFetcherUtils.handleListOffsetResponse(lor); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -385,9 +385,9 @@ private CompletableFuture buildListOffsetRequestToNode( * complete. */ private CompletableFuture sendListOffsetsRequestsAndResetPositions( - final Map timestampsToSearch) { + final Map timestampsToSearch) { Map> timestampsToSearchByNode = - groupListOffsetRequests(timestampsToSearch, Optional.empty()); + groupListOffsetRequests(timestampsToSearch, Optional.empty()); final AtomicInteger expectedResponses = new AtomicInteger(0); final CompletableFuture globalResult = new CompletableFuture<>(); @@ -395,25 +395,25 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( timestampsToSearchByNode.forEach((node, resetTimestamps) -> { subscriptionState.setNextAllowedRetry(resetTimestamps.keySet(), - time.milliseconds() + requestTimeoutMs); + time.milliseconds() + requestTimeoutMs); CompletableFuture partialResult = buildListOffsetRequestToNode( - node, - resetTimestamps, - false, - unsentRequests); + node, + resetTimestamps, + false, + unsentRequests); partialResult.whenComplete((result, error) -> { if (error == null) { offsetFetcherUtils.onSuccessfulResponseForResettingPositions(resetTimestamps, - result); + result); } else { RuntimeException e; if (error instanceof RuntimeException) { e = (RuntimeException) error; } else { e = new RuntimeException("Unexpected failure in ListOffsets request for " + - "resetting positions", error); + "resetting positions", error); } offsetFetcherUtils.onFailedResponseForResettingPositions(resetTimestamps, e); } @@ -446,10 +446,10 @@ private CompletableFuture sendListOffsetsRequestsAndResetPositions( */ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePositions( - Map partitionsToValidate) { + Map partitionsToValidate) { final Map> regrouped = - regroupFetchPositionsByLeader(partitionsToValidate); + regroupFetchPositionsByLeader(partitionsToValidate); long nextResetTimeMs = time.milliseconds() + requestTimeoutMs; final AtomicInteger expectedResponses = new AtomicInteger(0); @@ -470,8 +470,8 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi if (!hasUsableOffsetForLeaderEpochVersion(nodeApiVersions)) { log.debug("Skipping validation of fetch offsets for partitions {} since the broker does not " + - "support the required protocol version (introduced in Kafka 2.3)", - fetchPositions.keySet()); + "support the required protocol version (introduced in Kafka 2.3)", + fetchPositions.keySet()); for (TopicPartition partition : fetchPositions.keySet()) { subscriptionState.completeValidation(partition); } @@ -481,19 +481,19 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi subscriptionState.setNextAllowedRetry(fetchPositions.keySet(), nextResetTimeMs); CompletableFuture partialResult = - buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests); + buildOffsetsForLeaderEpochRequestToNode(node, fetchPositions, unsentRequests); partialResult.whenComplete((offsetsResult, error) -> { if (error == null) { offsetFetcherUtils.onSuccessfulResponseForValidatingPositions(fetchPositions, - offsetsResult); + offsetsResult); } else { RuntimeException e; if (error instanceof RuntimeException) { e = (RuntimeException) error; } else { e = new RuntimeException("Unexpected failure in OffsetsForLeaderEpoch " + - "request for validating positions", error); + "request for validating positions", error); } offsetFetcherUtils.onFailedResponseForValidatingPositions(fetchPositions, e); } @@ -518,32 +518,32 @@ private CompletableFuture sendOffsetsForLeaderEpochRequestsAndValidatePosi * positions to fetch. This also adds the request to the list of unsentRequests. */ private CompletableFuture buildOffsetsForLeaderEpochRequestToNode( - final Node node, - final Map fetchPositions, - List unsentRequests) { + final Node node, + final Map fetchPositions, + List unsentRequests) { AbstractRequest.Builder builder = - OffsetsForLeaderEpochUtils.prepareRequest(fetchPositions); + OffsetsForLeaderEpochUtils.prepareRequest(fetchPositions); log.debug("Creating OffsetsForLeaderEpoch request request {} to broker {}", builder, node); NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( - builder, - Optional.ofNullable(node)); + builder, + Optional.ofNullable(node)); unsentRequests.add(unsentRequest); CompletableFuture result = new CompletableFuture<>(); unsentRequest.whenComplete((response, error) -> { if (error != null) { log.debug("Sending OffsetsForLeaderEpoch request {} to broker {} failed", - builder, - node, - error); + builder, + node, + error); result.completeExceptionally(error); } else { OffsetsForLeaderEpochResponse offsetsForLeaderEpochResponse = (OffsetsForLeaderEpochResponse) response.responseBody(); log.trace("Received OffsetsForLeaderEpoch response {} from broker {}", offsetsForLeaderEpochResponse, node); try { OffsetsForLeaderEpochUtils.OffsetForEpochResult listOffsetResult = - OffsetsForLeaderEpochUtils.handleResponse(fetchPositions, offsetsForLeaderEpochResponse); + OffsetsForLeaderEpochUtils.handleResponse(fetchPositions, offsetsForLeaderEpochResponse); result.complete(listOffsetResult); } catch (RuntimeException e) { result.completeExceptionally(e); @@ -579,7 +579,7 @@ private ListOffsetsRequestState(Map timestampsToSearch, private void addPartitionsToRetry(Set partitionsToRetry) { remainingToSearch.putAll(partitionsToRetry.stream() - .collect(Collectors.toMap(tp -> tp, timestampsToSearch::get))); + .collect(Collectors.toMap(tp -> tp, timestampsToSearch::get))); } } @@ -607,8 +607,8 @@ private void addPartialResult(ListOffsetResult partialResult) { if (expectedResponses.decrementAndGet() == 0) { ListOffsetResult result = - new ListOffsetResult(fetchedTimestampOffsets, - partitionsToRetry); + new ListOffsetResult(fetchedTimestampOffsets, + partitionsToRetry); resultFuture.complete(result); } } catch (RuntimeException e) { @@ -628,8 +628,8 @@ private void addPartialResult(ListOffsetResult partialResult) { * metadata update). */ private Map> groupListOffsetRequests( - final Map timestampsToSearch, - final Optional listOffsetsRequestState) { + final Map timestampsToSearch, + final Optional listOffsetsRequestState) { final Map partitionDataMap = new HashMap<>(); for (Map.Entry entry : timestampsToSearch.entrySet()) { TopicPartition tp = entry.getKey(); @@ -643,9 +643,9 @@ private Map Date: Wed, 21 Feb 2024 15:15:53 -0800 Subject: [PATCH 035/130] UGHGGHGHGHGGH --- .../kafka/clients/consumer/internals/CommitRequestManager.java | 1 - .../clients/consumer/internals/CoordinatorRequestManager.java | 1 - .../kafka/clients/consumer/internals/NetworkClientDelegate.java | 1 - .../kafka/clients/consumer/internals/OffsetsRequestManager.java | 1 - 4 files changed, 4 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 af06be1183424..9206783d561be 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 @@ -1264,4 +1264,3 @@ static class MemberInfo { } } } - diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index e181589e8a757..d6a72812a52f9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -220,4 +220,3 @@ public Optional coordinator() { return Optional.ofNullable(this.coordinator); } } - 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 228ee42080721..141f5f955c8b5 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 @@ -390,4 +390,3 @@ protected NetworkClientDelegate create() { }; } } - 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 b32516edff38b..34f4b30c44dd3 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 @@ -661,4 +661,3 @@ int requestsToSend() { return requestsToSend.size(); } } - From 733f3f0f039061d95596580e5d1fa828b37ca8bb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:16:57 -0800 Subject: [PATCH 036/130] Update TopicMetadataRequestManager.java --- .../TopicMetadataRequestManager.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 8e4c680ce3e0f..75a5ed08d1554 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -81,8 +81,8 @@ public TopicMetadataRequestManager(final LogContext context, final ConsumerConfi public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { // Prune any requests which have timed out List expiredRequests = inflightRequests.stream() - .filter(req -> req.isExpired(currentTimeMs)) - .collect(Collectors.toList()); + .filter(req -> req.isExpired(currentTimeMs)) + .collect(Collectors.toList()); expiredRequests.forEach(TopicMetadataRequestState::expire); List requests = inflightRequests.stream() @@ -101,10 +101,10 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { */ public CompletableFuture>> requestAllTopicsMetadata(final long expirationTimeMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( - logContext, - expirationTimeMs, - retryBackoffMs, - retryBackoffMaxMs); + logContext, + expirationTimeMs, + retryBackoffMs, + retryBackoffMaxMs); inflightRequests.add(newRequest); return newRequest.future; } @@ -117,11 +117,11 @@ public CompletableFuture>> requestAllTopicsMetad */ public CompletableFuture>> requestTopicMetadata(final String topic, final long expirationTimeMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( - logContext, - topic, - expirationTimeMs, - retryBackoffMs, - retryBackoffMaxMs); + logContext, + topic, + expirationTimeMs, + retryBackoffMs, + retryBackoffMaxMs); inflightRequests.add(newRequest); return newRequest.future; } @@ -142,7 +142,7 @@ public TopicMetadataRequestState(final LogContext logContext, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs); + retryBackoffMaxMs); future = new CompletableFuture<>(); this.topic = null; this.allTopics = true; @@ -189,11 +189,11 @@ private boolean isExpired(final long currentTimeMs) { private void expire() { completeFutureAndRemoveRequest( - new TimeoutException("Timeout expired while fetching topic metadata")); + new TimeoutException("Timeout expired while fetching topic metadata")); } private NetworkClientDelegate.UnsentRequest createUnsentRequest( - final MetadataRequest.Builder request) { + final MetadataRequest.Builder request) { NetworkClientDelegate.UnsentRequest unsent = new NetworkClientDelegate.UnsentRequest( request, Optional.empty()); @@ -286,4 +286,4 @@ public String topic() { return topic; } } -} \ No newline at end of file +} From 8de90d6aa3687a8f6cef22f491a3bfd0b83e2441 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:20:59 -0800 Subject: [PATCH 037/130] Update ApplicationEventProcessor.java --- .../events/ApplicationEventProcessor.java | 67 +------------------ 1 file changed, 2 insertions(+), 65 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 5bed996bfe67f..6495c7973cbf4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -33,7 +33,6 @@ import org.apache.kafka.common.utils.Timer; import org.slf4j.Logger; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -69,19 +68,8 @@ public ApplicationEventProcessor(final LogContext logContext, * an event generates an error. In such cases, the processor will log an exception, but we do not want those * errors to be propagated to the caller. */ - public List> process() { - List> futures = new ArrayList<>(); - - process((event, error) -> { - error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); - - if (event instanceof CompletableApplicationEvent) { - CompletableFuture future = ((CompletableApplicationEvent) event).future(); - futures.add(future); - } - }); - - return futures; + public boolean process() { + return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override @@ -162,28 +150,12 @@ private void process(final PollApplicationEvent event) { } private void process(final AsyncCommitApplicationEvent event) { - if (!requestManagers.commitRequestManager.isPresent()) { - // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id - // upfront, so we should never get to this block. - Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set"); - event.future().completeExceptionally(exception); - return; - } - CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture commitResult = manager.commitAsync(event.offsets()); chain(commitResult, event.future()); } private void process(final SyncCommitApplicationEvent event) { - if (!requestManagers.commitRequestManager.isPresent()) { - // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id - // upfront, so we should never get to this block. - Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set"); - event.future().completeExceptionally(exception); - return; - } - CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture commitResult = manager.commitSync(event.offsets(), event.deadlineMs()); chain(commitResult, event.future()); @@ -196,11 +168,6 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } - Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to fetch committed offset due to exceeding timeout")) - return; - CommitRequestManager manager = requestManagers.commitRequestManager.get(); long expirationTimeMs = getExpirationTimeForTimeout(event.deadlineMs()); CompletableFuture> future = manager.fetchOffsets(event.partitions(), expirationTimeMs); @@ -226,11 +193,6 @@ private void process(final AssignmentChangeApplicationEvent event) { } private void process(final ListOffsetsApplicationEvent event) { - final Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to list offsets due to exceeding timeout")) - return; - final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets( event.timestampsToSearch(), event.requireTimestamps() @@ -267,41 +229,21 @@ private void process(final UnsubscribeApplicationEvent event) { return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to unsubscribe due to exceeding timeout")) - return; - CompletableFuture result = membershipManager.leaveGroup(); chain(result, event.future()); } private void process(final ResetPositionsApplicationEvent event) { - Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to reset positions due to exceeding timeout")) - return; - CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); chain(result, event.future()); } private void process(final ValidatePositionsApplicationEvent event) { - Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to validate positions due to exceeding timeout")) - return; - CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); chain(result, event.future()); } private void process(final TopicMetadataApplicationEvent event) { - Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to retrieve topic metadata due to exceeding timeout")) - return; - final CompletableFuture>> future; if (event.isAllTopics()) { @@ -341,11 +283,6 @@ private void process(final LeaveOnCloseApplicationEvent event) { Objects.requireNonNull(requestManagers.heartbeatRequestManager.get().membershipManager(), "Expecting " + "membership manager to be non-null"); log.debug("Leaving group before closing"); - final Timer timer = timer(event); - - if (maybeTimeout(event, timer, "Unable to leave group due to exceeding timeout")) - return; - CompletableFuture future = membershipManager.leaveGroup(); // The future will be completed on heartbeat sent chain(future, event.future()); From 296163f0bedc8e94661661405fa849389fa5f603 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 15:22:56 -0800 Subject: [PATCH 038/130] Update ApplicationEventProcessor.java --- .../events/ApplicationEventProcessor.java | 24 ++++--------------- 1 file changed, 4 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 6495c7973cbf4..4c764a8bd2e3c 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 @@ -167,10 +167,8 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { "offset because the CommittedRequestManager is not available. Check if group.id was set correctly")); return; } - CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = getExpirationTimeForTimeout(event.deadlineMs()); - CompletableFuture> future = manager.fetchOffsets(event.partitions(), expirationTimeMs); + CompletableFuture> future = manager.fetchOffsets(event.partitions(), event.deadlineMs()); chain(future, event.future()); } @@ -193,10 +191,9 @@ private void process(final AssignmentChangeApplicationEvent event) { } private void process(final ListOffsetsApplicationEvent event) { - final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets( - event.timestampsToSearch(), - event.requireTimestamps() - ); + final CompletableFuture> future = + requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), + event.requireTimestamps()); chain(future, event.future()); } @@ -306,19 +303,6 @@ private boolean maybeTimeout(CompletableApplicationEvent event, Timer timer, } } - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; - } - private void chain(final CompletableFuture primary, final CompletableFuture secondary) { Objects.requireNonNull( primary, From e761ba55b73f8f48c58b534135b43b0552d9af76 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:02:19 -0800 Subject: [PATCH 039/130] Updates --- .../internals/AsyncKafkaConsumer.java | 23 +++--- .../events/AsyncCommitApplicationEvent.java | 51 +------------ .../events/CommitApplicationEvent.java | 74 +++++++++++++++++++ .../events/CompletableApplicationEvent.java | 7 +- .../events/SyncCommitApplicationEvent.java | 40 +--------- .../internals/ConsumerNetworkThreadTest.java | 4 +- .../internals/FetchRequestManagerTest.java | 8 +- 7 files changed, 101 insertions(+), 106 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.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 b7bbace2eac7f..ba59487e5f307 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 @@ -43,6 +43,7 @@ import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; @@ -768,8 +769,9 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - AsyncCommitApplicationEvent event = new AsyncCommitApplicationEvent(offsets); - CompletableFuture future = commit(event.offsets(), event, event.future()); + Timer timer = time.timer(Long.MAX_VALUE); + AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets, timer); + CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { if (t == null) { @@ -790,13 +792,12 @@ public void commitAsync(Map offsets, OffsetCo } } - private CompletableFuture commit(final Map offsets, - final ApplicationEvent event, - final CompletableFuture future) { + private CompletableFuture commit(final CommitApplicationEvent commitEvent) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); maybeThrowInvalidGroupIdException(); + Map offsets = commitEvent.offsets(); log.debug("Committing offsets: {}", offsets); offsets.forEach(this::updateLastSeenEpochIfNewer); @@ -804,8 +805,8 @@ private CompletableFuture commit(final Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitApplicationEvent event = new SyncCommitApplicationEvent(offsets, requestTimer); - CompletableFuture future = commit(event.offsets(), event, event.future()); - wakeupTrigger.setActiveTask(future); - ConsumerUtils.getResult(future, requestTimer); + SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, requestTimer); + CompletableFuture commitFuture = commit(syncCommitEvent); + wakeupTrigger.setActiveTask(commitFuture); + ConsumerUtils.getResult(commitFuture, requestTimer); interceptors.onCommit(offsets); } finally { wakeupTrigger.clearTask(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index d77f74de8c50e..acf9d10f9f64c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -17,60 +17,17 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.internals.RelaxedCompletableFuture; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; -import java.util.Collections; import java.util.Map; /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ -public class AsyncCommitApplicationEvent extends ApplicationEvent implements CompletableEvent { +public class AsyncCommitApplicationEvent extends CommitApplicationEvent { - private final RelaxedCompletableFuture future; - - /** - * Offsets to commit per partition. - */ - private final Map offsets; - - public AsyncCommitApplicationEvent(final Map offsets) { - super(Type.COMMIT_SYNC); - this.offsets = Collections.unmodifiableMap(offsets); - this.future = new RelaxedCompletableFuture<>(); - - for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { - if (offsetAndMetadata.offset() < 0) { - throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); - } - } - } - - @Override - public RelaxedCompletableFuture future() { - return future; - } - - public Map offsets() { - return offsets; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - AsyncCommitApplicationEvent that = (AsyncCommitApplicationEvent) o; - - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + public AsyncCommitApplicationEvent(final Map offsets, Timer timer) { + super(Type.COMMIT_ASYNC, timer, offsets); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java new file mode 100644 index 0000000000000..b780db6672389 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +public abstract class CommitApplicationEvent extends CompletableApplicationEvent { + + /** + * Offsets to commit per partition. + */ + private final Map offsets; + + protected CommitApplicationEvent(final Type type, + final Timer timer, + final Map offsets) { + super(type, timer); + this.offsets = validate(offsets); + } + + private static Map validate(final Map offsets) { + Objects.requireNonNull(offsets, "Commit offsets should be non-null"); + + for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { + if (offsetAndMetadata.offset() < 0) { + throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); + } + } + + return Collections.unmodifiableMap(offsets); + } + + public Map offsets() { + return offsets; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + CommitApplicationEvent that = (CommitApplicationEvent) o; + + return offsets.equals(that.offsets); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + offsets.hashCode(); + return result; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 2cda3a5d6696d..06a30f3cc6077 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.clients.consumer.internals.RelaxedCompletableFuture; import org.apache.kafka.common.utils.Timer; import java.util.Objects; @@ -37,7 +36,7 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im protected CompletableApplicationEvent(Type type, Timer timer) { super(type); - this.future = new RelaxedCompletableFuture<>(); + this.future = new CompletableFuture<>(); this.timer = timer; this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } @@ -68,7 +67,9 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(future, deadlineMs); + int result = super.hashCode(); + result = 31 * result + Objects.hash(future, deadlineMs); + return result; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 1e85d0199a3d8..319f0f8e67def 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -20,51 +20,15 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Timer; -import java.util.Collections; import java.util.Map; -import java.util.Objects; /** * Event to commit offsets waiting for a response and retrying on expected retriable errors until * the timer expires. */ -public class SyncCommitApplicationEvent extends CompletableApplicationEvent { - - /** - * Offsets to commit per partition. - */ - private final Map offsets; +public class SyncCommitApplicationEvent extends CommitApplicationEvent { public SyncCommitApplicationEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_SYNC, timer); - this.offsets = Collections.unmodifiableMap(offsets); - - for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { - if (offsetAndMetadata.offset() < 0) { - throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); - } - } - } - - public Map offsets() { - return offsets; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - SyncCommitApplicationEvent that = (SyncCommitApplicationEvent) o; - - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + super(Type.COMMIT_SYNC, timer, offsets); } } 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 05045ae9a9620..9769c9bb2a3e9 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 @@ -154,7 +154,7 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>()); + ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>(), time.timer(100L)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(AsyncCommitApplicationEvent.class)); @@ -291,7 +291,7 @@ void testEnsureEventsAreCompleted() { client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); CompletableApplicationEvent event1 = spy(new SyncCommitApplicationEvent(Collections.emptyMap(), timer)); - ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap()); + ApplicationEvent event2 = new SyncCommitApplicationEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); 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 ef2f162ea8618..2024b693d473a 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 @@ -3585,8 +3585,7 @@ private void buildFetcher(MetricConfig metricConfig, metricsManager, networkClientDelegate, fetchCollector, - apiVersions, - requestTimeoutMs)); + apiVersions)); ConsumerNetworkClient consumerNetworkClient = new ConsumerNetworkClient( logContext, client, @@ -3645,9 +3644,8 @@ public TestableFetchRequestManager(LogContext logContext, FetchMetricsManager metricsManager, NetworkClientDelegate networkClientDelegate, FetchCollector fetchCollector, - ApiVersions apiVersions, - int requestTimeoutMs) { - super(logContext, time, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, networkClientDelegate, apiVersions, requestTimeoutMs); + ApiVersions apiVersions) { + super(logContext, time, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, networkClientDelegate, apiVersions); this.fetchCollector = fetchCollector; } From e5bdf822c254e16bdcd8bfa4526c9cf10d7537f6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:08:02 -0800 Subject: [PATCH 040/130] Reverting unnecessary changesd --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 2 +- .../internals/events/AsyncCommitApplicationEvent.java | 8 ++++++++ .../internals/events/CommitApplicationEvent.java | 9 +-------- .../internals/events/SyncCommitApplicationEvent.java | 7 +++++++ 4 files changed, 17 insertions(+), 9 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 ba59487e5f307..c5b1cd44efda2 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 @@ -1643,7 +1643,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { private void throwIfNoAssignorsConfigured() { if (assignors.isEmpty()) throw new IllegalStateException("Must configure at least one partition assigner class name to " + - ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); } private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index acf9d10f9f64c..2472975ed4107 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -30,4 +30,12 @@ public class AsyncCommitApplicationEvent extends CommitApplicationEvent { public AsyncCommitApplicationEvent(final Map offsets, Timer timer) { super(Type.COMMIT_ASYNC, timer, offsets); } + + @Override + public String toString() { + return "AsyncCommitApplicationEvent{" + + toStringBase() + + ", offsets=" + offsets() + + '}'; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index b780db6672389..5da22be82f8ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -22,7 +22,6 @@ import java.util.Collections; import java.util.Map; -import java.util.Objects; public abstract class CommitApplicationEvent extends CompletableApplicationEvent { @@ -35,19 +34,13 @@ protected CommitApplicationEvent(final Type type, final Timer timer, final Map offsets) { super(type, timer); - this.offsets = validate(offsets); - } - - private static Map validate(final Map offsets) { - Objects.requireNonNull(offsets, "Commit offsets should be non-null"); + this.offsets = Collections.unmodifiableMap(offsets); for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); } } - - return Collections.unmodifiableMap(offsets); } public Map offsets() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 319f0f8e67def..ab8c015a19cfd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -31,4 +31,11 @@ public class SyncCommitApplicationEvent extends CommitApplicationEvent { public SyncCommitApplicationEvent(final Map offsets, final Timer timer) { super(Type.COMMIT_SYNC, timer, offsets); } + + public String toString() { + return "SyncCommitApplicationEvent{" + + toStringBase() + + ", offsets=" + offsets() + + '}'; + } } From 60f9a0a80a76dea615b163b4c84cecae2bc74b92 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:11:51 -0800 Subject: [PATCH 041/130] Whitespace and stuff --- .../events/SyncCommitApplicationEvent.java | 1 + .../internals/CommitRequestManagerTest.java | 146 +++++++++--------- .../HeartbeatRequestManagerTest.java | 86 +++++------ 3 files changed, 117 insertions(+), 116 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index ab8c015a19cfd..71e5a8124c5e9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -32,6 +32,7 @@ public SyncCommitApplicationEvent(final Map o super(Type.COMMIT_SYNC, timer, offsets); } + @Override public String toString() { return "SyncCommitApplicationEvent{" + toStringBase() + 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 d8b9835432117..c27494d69a038 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 @@ -158,10 +158,10 @@ public void testPollEnsureAutocommitSent() { commitRequestManger.updateAutoCommitTimer(time.milliseconds()); List pollResults = assertPoll(1, commitRequestManger); pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse( - "t1", - 1, - (short) 1, - Errors.NONE))); + "t1", + 1, + (short) 1, + Errors.NONE))); assertEquals(0.03, (double) getMetric("commit-rate").metricValue(), 0.01); assertEquals(1.0, getMetric("commit-total").metricValue()); @@ -191,9 +191,9 @@ public void testPollEnsureCorrectInflightRequestBufferSize() { NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds()); assertEquals(4, result.unsentRequests.size()); assertTrue(result.unsentRequests - .stream().anyMatch(r -> r.requestBuilder() instanceof OffsetCommitRequest.Builder)); + .stream().anyMatch(r -> r.requestBuilder() instanceof OffsetCommitRequest.Builder)); assertTrue(result.unsentRequests - .stream().anyMatch(r -> r.requestBuilder() instanceof OffsetFetchRequest.Builder)); + .stream().anyMatch(r -> r.requestBuilder() instanceof OffsetFetchRequest.Builder)); assertFalse(commitManager.pendingRequests.hasUnsentRequests()); assertEquals(2, commitManager.pendingRequests.inflightOffsetFetches.size()); @@ -258,10 +258,10 @@ public void testAsyncAutocommitNotRetriedAfterException() { futures = assertPoll(1, commitRequestManger); assertEmptyPendingRequests(commitRequestManger); futures.get(0).onComplete(mockOffsetCommitResponse( - "topic", - 1, - (short) 1, - Errors.NONE)); + "topic", + 1, + (short) 1, + Errors.NONE)); } // This is the case of the sync commit triggered from an API call to commitSync or when the @@ -506,10 +506,10 @@ public void testOffsetFetchRequestEnsureDuplicatedRequestSucceed() { Set partitions = new HashSet<>(); partitions.add(new TopicPartition("t1", 0)); List>> futures = sendAndVerifyDuplicatedOffsetFetchRequests( - commitRequestManger, - partitions, - 2, - Errors.NONE); + commitRequestManger, + partitions, + 2, + Errors.NONE); futures.forEach(f -> { assertTrue(f.isDone()); assertFalse(f.isCompletedExceptionally()); @@ -971,8 +971,8 @@ private void commitOffsetWithAssertedLatency(CommitRequestManager commitRequestM final String topic = "topic"; final int partition = 1; Map offsets = Collections.singletonMap( - new TopicPartition(topic, partition), - new OffsetAndMetadata(0)); + new TopicPartition(topic, partition), + new OffsetAndMetadata(0)); long commitCreationTimeMs = time.milliseconds(); commitRequestManager.commitAsync(offsets); @@ -983,12 +983,12 @@ private void commitOffsetWithAssertedLatency(CommitRequestManager commitRequestM time.sleep(latencyMs); long commitReceivedTimeMs = time.milliseconds(); res.unsentRequests.get(0).future().complete(mockOffsetCommitResponse( - topic, - partition, - (short) 1, - commitCreationTimeMs, - commitReceivedTimeMs, - Errors.NONE)); + topic, + partition, + (short) 1, + commitCreationTimeMs, + commitReceivedTimeMs, + Errors.NONE)); } private void completeOffsetFetchRequestWithError(CommitRequestManager commitRequestManager, @@ -1084,7 +1084,7 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b partitions.add(tp2); long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture> future = - commitRequestManger.fetchOffsets(partitions, expirationTimeMs); + commitRequestManger.fetchOffsets(partitions, expirationTimeMs); NetworkClientDelegate.PollResult res = commitRequestManger.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); @@ -1095,10 +1095,10 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b topicPartitionData.put(tp2, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", Errors.NONE)); res.unsentRequests.get(0).handler().onComplete(buildOffsetFetchClientResponse( - res.unsentRequests.get(0), - topicPartitionData, - Errors.NONE, - false)); + res.unsentRequests.get(0), + topicPartitionData, + Errors.NONE, + false)); if (isRetriable) testRetriable(commitRequestManger, Collections.singletonList(future)); else @@ -1137,10 +1137,10 @@ private static Stream partitionDataErrorSupplier() { } private List>> sendAndVerifyDuplicatedOffsetFetchRequests( - final CommitRequestManager commitRequestManger, - final Set partitions, - int numRequest, - final Errors error) { + final CommitRequestManager commitRequestManger, + final Set partitions, + int numRequest, + final Errors error) { List>> futures = new ArrayList<>(); long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; for (int i = 0; i < numRequest; i++) { @@ -1202,30 +1202,30 @@ private CommitRequestManager create(final boolean autoCommitEnabled, final long props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); return spy(new CommitRequestManager( - this.time, - this.logContext, - this.subscriptionState, - new ConsumerConfig(props), - this.coordinatorRequestManager, - this.offsetCommitCallbackInvoker, - DEFAULT_GROUP_ID, - Optional.of(DEFAULT_GROUP_INSTANCE_ID), - retryBackoffMs, - retryBackoffMaxMs, - OptionalDouble.of(0), - metrics)); + this.time, + this.logContext, + this.subscriptionState, + new ConsumerConfig(props), + this.coordinatorRequestManager, + this.offsetCommitCallbackInvoker, + DEFAULT_GROUP_ID, + Optional.of(DEFAULT_GROUP_INSTANCE_ID), + retryBackoffMs, + retryBackoffMaxMs, + OptionalDouble.of(0), + metrics)); } private ClientResponse buildOffsetFetchClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final Set topicPartitions, - final Errors error) { + final NetworkClientDelegate.UnsentRequest request, + final Set topicPartitions, + final Errors error) { HashMap topicPartitionData = new HashMap<>(); topicPartitions.forEach(tp -> topicPartitionData.put(tp, new OffsetFetchResponse.PartitionData( - 100L, - Optional.of(1), - "metadata", - Errors.NONE))); + 100L, + Optional.of(1), + "metadata", + Errors.NONE))); return buildOffsetFetchClientResponse(request, topicPartitionData, error, false); } @@ -1274,14 +1274,14 @@ public ClientResponse mockOffsetCommitResponse(String topic, when(response.data()).thenReturn(responseData); return new ClientResponse( new RequestHeader(ApiKeys.OFFSET_COMMIT, apiKeyVersion, "", 1), - null, - "-1", - createdTimeMs, - receivedTimeMs, - false, - null, - null, - new OffsetCommitResponse(responseData) + null, + "-1", + createdTimeMs, + receivedTimeMs, + false, + null, + null, + new OffsetCommitResponse(responseData) ); } @@ -1312,25 +1312,25 @@ public ClientResponse mockOffsetCommitResponseDisconnected(String topic, int par } private ClientResponse buildOffsetFetchClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final Map topicPartitionData, - final Errors error, - final boolean disconnected) { + final NetworkClientDelegate.UnsentRequest request, + final Map topicPartitionData, + final Errors error, + final boolean disconnected) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof OffsetFetchRequest); OffsetFetchRequest offsetFetchRequest = (OffsetFetchRequest) abstractRequest; OffsetFetchResponse response = - new OffsetFetchResponse(error, topicPartitionData); + new OffsetFetchResponse(error, topicPartitionData); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), - request.handler(), - "-1", - time.milliseconds(), - time.milliseconds(), - disconnected, - null, - null, - response + new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + disconnected, + null, + null, + response ); } @@ -1339,4 +1339,4 @@ private KafkaMetric getMetric(String name) { name, CONSUMER_COORDINATOR_METRICS)); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 11bffbd9e0fd5..4016b74b27bd6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -130,11 +130,11 @@ private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { cleanup(); ConsumerTestBuilder.GroupInformation gi = new ConsumerTestBuilder.GroupInformation( - DEFAULT_GROUP_ID, - groupInstanceId, - 0, - 0.0, - Optional.of(DEFAULT_REMOTE_ASSIGNOR) + DEFAULT_GROUP_ID, + groupInstanceId, + 0, + 0.0, + Optional.of(DEFAULT_REMOTE_ASSIGNOR) ); setUp(Optional.of(gi)); @@ -179,7 +179,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); ConsumerGroupHeartbeatRequest heartbeatRequest = - (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); + (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); // Should include epoch 0 to join and no member ID. assertTrue(heartbeatRequest.data().memberId().isEmpty()); @@ -303,8 +303,8 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); + .setMemberId(memberId) + .setMemberEpoch(memberEpoch)); membershipManager.onHeartbeatResponseReceived(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload @@ -541,10 +541,10 @@ public void testHeartbeatState() { new ConsumerGroupHeartbeatResponseData.Assignment(); assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(1) - .setAssignment(assignmentTopic1)); + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) + .setMemberId(memberId) + .setMemberEpoch(1) + .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); membershipManager.onHeartbeatResponseReceived(rs1.data()); @@ -558,24 +558,24 @@ public void testPollTimerExpiration() { membershipManager = mock(MembershipManager.class); heartbeatState = mock(HeartbeatRequestManager.HeartbeatState.class); heartbeatRequestState = spy(new HeartbeatRequestManager.HeartbeatRequestState( - new LogContext(), - time, - heartbeatIntervalMs, - retryBackoffMs, - retryBackoffMaxMs, - 0)); + new LogContext(), + time, + heartbeatIntervalMs, + retryBackoffMs, + retryBackoffMaxMs, + 0)); backgroundEventHandler = mock(BackgroundEventHandler.class); heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler); + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); - // On poll timer expiration, the member should transition to stale and a last heartbeat + // On poll timer expiration, the member should transition to stale and a last heartbeat // should be sent to leave the group time.sleep(maxPollIntervalMs); assertHeartbeat(heartbeatRequestManager, heartbeatIntervalMs); @@ -680,9 +680,9 @@ private void mockStableMember() { membershipManager.onSubscriptionUpdated(); // Heartbeat response without assignment to set the state to STABLE. ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) + .setMemberId(memberId) + .setMemberEpoch(memberEpoch)); membershipManager.onHeartbeatResponseReceived(rs1.data()); assertEquals(MemberState.STABLE, membershipManager.state()); } @@ -777,22 +777,22 @@ private KafkaMetric getMetric(final String name) { } private HeartbeatRequestManager createHeartbeatRequestManager( - final CoordinatorRequestManager coordinatorRequestManager, - final MembershipManager membershipManager, - final HeartbeatRequestManager.HeartbeatState heartbeatState, - final HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState, - final BackgroundEventHandler backgroundEventHandler) { + final CoordinatorRequestManager coordinatorRequestManager, + final MembershipManager membershipManager, + final HeartbeatRequestManager.HeartbeatState heartbeatState, + final HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState, + final BackgroundEventHandler backgroundEventHandler) { LogContext logContext = new LogContext(); pollTimer = time.timer(maxPollIntervalMs); return new HeartbeatRequestManager( - logContext, - pollTimer, - config(), - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler, - metrics); - } -} \ No newline at end of file + logContext, + pollTimer, + config(), + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler, + metrics); + } +} From 2699626f0591ee0d9c44ebbd9da4160cddd77cf5 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:15:32 -0800 Subject: [PATCH 042/130] Whitespace --- .../internals/ConsumerNetworkThreadTest.java | 22 +- .../internals/MembershipManagerImplTest.java | 290 +++++++------- .../internals/NetworkClientDelegateTest.java | 10 +- .../internals/OffsetsRequestManagerTest.java | 358 +++++++++--------- 4 files changed, 340 insertions(+), 340 deletions(-) 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 9769c9bb2a3e9..0be113f76aa3f 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 @@ -128,12 +128,12 @@ public void testStartupAndTearDown() throws InterruptedException { // There's a nonzero amount of time between starting the thread and having it // begin to execute our code. Wait for a bit before checking... TestUtils.waitForCondition(isStarted, - "The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms"); + "The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms"); consumerNetworkThread.close(Duration.ofMillis(DEFAULT_MAX_WAIT_MS)); TestUtils.waitForCondition(isClosed, - "The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms"); + "The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms"); } @Test @@ -238,22 +238,22 @@ void testFetchTopicMetadata() { @Test void testPollResultTimer() { NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) - .setKey("foobar")), - Optional.empty()); + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) + .setKey("foobar")), + Optional.empty()); req.setTimer(time, DEFAULT_REQUEST_TIMEOUT_MS); // purposely setting a non-MAX time to ensure it is returning Long.MAX_VALUE upon success NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult( - 10, - Collections.singletonList(req)); + 10, + Collections.singletonList(req)); assertEquals(10, networkClient.addAll(success)); NetworkClientDelegate.PollResult failure = new NetworkClientDelegate.PollResult( - 10, - new ArrayList<>()); + 10, + new ArrayList<>()); assertEquals(10, networkClient.addAll(failure)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java index 4eb4d6d27790a..f5c65d58bfc54 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java @@ -118,18 +118,18 @@ public void tearDown() { private MembershipManagerImpl createMembershipManagerJoiningGroup() { MembershipManagerImpl manager = spy(new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time)); manager.transitionToJoining(); return manager; } private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId) { MembershipManagerImpl manager = spy(new MembershipManagerImpl( - GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time)); manager.transitionToJoining(); return manager; } @@ -137,9 +137,9 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId, String serverAssignor) { MembershipManagerImpl manager = new MembershipManagerImpl( - GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, - Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, logContext, Optional.empty(), backgroundEventHandler, time); + GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, + Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, + metadata, logContext, Optional.empty(), backgroundEventHandler, time); manager.transitionToJoining(); return manager; } @@ -163,9 +163,9 @@ public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() { // First join should register to get metadata updates MembershipManagerImpl manager = new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time); manager.transitionToJoining(); clearInvocations(metadata); @@ -195,12 +195,12 @@ public void testTransitionToReconcilingOnlyIfAssignmentReceived() { assertEquals(MemberState.JOINING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithoutAssignment = - createConsumerGroupHeartbeatResponse(null); + createConsumerGroupHeartbeatResponse(null); membershipManager.onHeartbeatResponseReceived(responseWithoutAssignment.data()); assertNotEquals(MemberState.RECONCILING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithAssignment = - createConsumerGroupHeartbeatResponse(createAssignment(true)); + createConsumerGroupHeartbeatResponse(createAssignment(true)); membershipManager.onHeartbeatResponseReceived(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @@ -225,7 +225,7 @@ public void testMemberIdAndEpochResetOnFencedMembers() { public void testTransitionToFatal() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(null); + createConsumerGroupHeartbeatResponse(null); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); @@ -240,9 +240,9 @@ public void testTransitionToFatal() { @Test public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( - GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), - subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler, time); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); @@ -296,17 +296,17 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { int epoch = 5; membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(epoch)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(epoch)); verify(listener).onMemberEpochUpdated(Optional.of(epoch), Optional.of(MEMBER_ID)); clearInvocations(listener); membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(epoch)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(epoch)); verify(listener, never()).onMemberEpochUpdated(any(), any()); } @@ -415,7 +415,7 @@ public void testLeaveGroupEpoch() { membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH, - membershipManager.memberEpoch()); + membershipManager.memberEpoch()); // Dynamic member should leave the group with epoch -1. membershipManager = createMemberInStableState(null); @@ -423,7 +423,7 @@ public void testLeaveGroupEpoch() { membershipManager.leaveGroup(); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, - membershipManager.memberEpoch()); + membershipManager.memberEpoch()); } /** @@ -475,8 +475,8 @@ public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { // Reconciliation that does not complete stuck on revocation commit. CompletableFuture commitResult = - mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId1, topic1, - Arrays.asList(1, 2), true); + mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId1, topic1, + Arrays.asList(1, 2), true); Map> assignment1 = topicIdPartitionsMap(topicId1, 1, 2); assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); @@ -518,7 +518,7 @@ public void testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU Uuid topicId1 = Uuid.randomUuid(); String topic1 = "topic1"; MembershipManagerImpl membershipManager = - mockMemberSuccessfullyReceivesAndAcksAssignment(topicId1, topic1, Collections.singletonList(0)); + mockMemberSuccessfullyReceivesAndAcksAssignment(topicId1, topic1, Collections.singletonList(0)); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); clearInvocations(membershipManager, subscriptionState); @@ -530,8 +530,8 @@ public void testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU Uuid topicId2 = Uuid.randomUuid(); String topic2 = "topic2"; CompletableFuture commitResult = - mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId2, topic2, - Arrays.asList(1, 2), false); + mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId2, topic2, + Arrays.asList(1, 2), false); verify(metadata).requestUpdate(anyBoolean()); assertEquals(Collections.singleton(topicId2), membershipManager.topicsAwaitingReconciliation()); @@ -884,9 +884,9 @@ public void testUpdateStateFailsOnResponsesWithErrors() { // Updating state with a heartbeat response containing errors cannot be performed and // should fail. ConsumerGroupHeartbeatResponse unknownMemberResponse = - createConsumerGroupHeartbeatResponseWithError(); + createConsumerGroupHeartbeatResponseWithError(); assertThrows(IllegalArgumentException.class, - () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); + () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); } /** @@ -1028,14 +1028,14 @@ public void testMemberKeepsUnresolvedAssignmentWaitingForMetadataUntilResolved() String topic1Name = "topic1"; Uuid topic2 = Uuid.randomUuid(); ConsumerGroupHeartbeatResponseData.Assignment assignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic1) - .setPartitions(Collections.singletonList(0)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic2) - .setPartitions(Arrays.asList(1, 3)) - )); + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic1) + .setPartitions(Collections.singletonList(0)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic2) + .setPartitions(Arrays.asList(1, 3)) + )); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topic1, topic1Name)); // Receive assignment partly in metadata - reconcile+ack what's in metadata, keep the @@ -1082,7 +1082,7 @@ public void testReconcileNewPartitionsAssignedWhenOtherPartitionsOwned() { TopicIdPartition ownedPartition = new TopicIdPartition(topicId, new TopicPartition(topicName, 0)); MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, - Collections.singletonList(ownedPartition)); + Collections.singletonList(ownedPartition)); // New assignment received, adding partitions 1 and 2 to the previously owned partition 0. receiveAssignment(topicId, Arrays.asList(0, 1, 2), membershipManager); @@ -1187,7 +1187,7 @@ public void testReconcilePartitionsRevokedWithFailedAutoCommitCompletesRevocatio // Complete commit request commitResult.completeExceptionally(new KafkaException("Commit request failed with " + - "non-retriable error")); + "non-retriable error")); testRevocationOfAllPartitionsCompleted(membershipManager); } @@ -1223,10 +1223,10 @@ public void testMetadataUpdatesReconcilesUnresolvedAssignments() { // Assignment not in metadata ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(Arrays.asList(0, 1)))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(Arrays.asList(0, 1)))); MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -1252,10 +1252,10 @@ public void testMetadataUpdatesRequestsAnotherUpdateIfNeeded() { // Assignment not in metadata ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(Arrays.asList(0, 1)))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(Arrays.asList(0, 1)))); MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -1361,11 +1361,11 @@ public void testListenerCallbacksBasic() { // Step 3: assign partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - topicPartitions(topicName, 0, 1), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 0, 1), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1389,21 +1389,21 @@ public void testListenerCallbacksBasic() { // Step 6: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(topicName, 0, 1), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0, 1), + true ); assertTrue(membershipManager.reconciliationInProgress()); // Step 7: assign partitions should still be called, even though it's empty performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - Collections.emptySortedSet(), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet(), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1426,9 +1426,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartition(membershipManager, topicId, topicName); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.of(new IllegalArgumentException("Intentional onPartitionsRevoked() error")), - Optional.empty(), - Optional.empty() + Optional.of(new IllegalArgumentException("Intentional onPartitionsRevoked() error")), + Optional.empty(), + Optional.empty() ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1452,11 +1452,11 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 3: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(topicName, 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1479,9 +1479,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { Uuid topicId = Uuid.randomUuid(); mockOwnedPartition(membershipManager, topicId, topicName); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.empty(), - Optional.of(new IllegalArgumentException("Intentional onPartitionsAssigned() error")), - Optional.empty() + Optional.empty(), + Optional.of(new IllegalArgumentException("Intentional onPartitionsAssigned() error")), + Optional.empty() ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1505,22 +1505,22 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { // Step 3: revoke partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions("topic1", 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions("topic1", 0), + true ); assertTrue(membershipManager.reconciliationInProgress()); // Step 4: assign partitions performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - Collections.emptySortedSet(), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet(), + true ); assertFalse(membershipManager.reconciliationInProgress()); @@ -1688,9 +1688,9 @@ private void testOnPartitionsLost(Optional lostError) { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( - Optional.empty(), - Optional.empty(), - lostError + Optional.empty(), + Optional.empty(), + lostError ); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1707,11 +1707,11 @@ private void testOnPartitionsLost(Optional lostError) { // Step 3: invoke the callback performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, - topicPartitions("topic1", 0), - true + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, + topicPartitions("topic1", 0), + true ); // Step 4: Receive ack and make sure we're done and our listener was called appropriately @@ -1725,10 +1725,10 @@ private void testOnPartitionsLost(Optional lostError) { private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { return new ConsumerRebalanceListenerInvoker( - new LogContext(), - subscriptionState, - new MockTime(1), - new RebalanceCallbackMetricsManager(new Metrics()) + new LogContext(), + subscriptionState, + new MockTime(1), + new RebalanceCallbackMetricsManager(new Metrics()) ); } @@ -1776,10 +1776,10 @@ private Map> topicIdPartitionsMap(Uuid topicId, int... } private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(MembershipManagerImpl membershipManager, - ConsumerRebalanceListenerInvoker invoker, - ConsumerRebalanceListenerMethodName expectedMethodName, - SortedSet expectedPartitions, - boolean complete) { + ConsumerRebalanceListenerInvoker invoker, + ConsumerRebalanceListenerMethodName expectedMethodName, + SortedSet expectedPartitions, + boolean complete) { // We expect only our enqueued event in the background queue. assertEquals(1, backgroundEventQueue.size()); assertNotNull(backgroundEventQueue.peek()); @@ -1790,10 +1790,10 @@ private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(Membersh assertEquals(expectedPartitions, neededEvent.partitions()); ConsumerRebalanceListenerCallbackCompletedEvent invokedEvent = invokeRebalanceCallbacks( - invoker, - neededEvent.methodName(), - neededEvent.partitions(), - neededEvent.future() + invoker, + neededEvent.methodName(), + neededEvent.partitions(), + neededEvent.future() ); if (complete) { @@ -1849,7 +1849,7 @@ public void testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() { } private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( - Uuid topicId, String topicName, List partitions) { + Uuid topicId, String topicName, List partitions) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, Collections.emptyList()); @@ -1867,7 +1867,7 @@ private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( } private CompletableFuture mockEmptyAssignmentAndRevocationStuckOnCommit( - MembershipManagerImpl membershipManager) { + MembershipManagerImpl membershipManager) { CompletableFuture commitResult = mockRevocationNoCallbacks(true); receiveEmptyAssignment(membershipManager); @@ -1882,8 +1882,8 @@ private CompletableFuture mockEmptyAssignmentAndRevocationStuckOnCommit( } private CompletableFuture mockNewAssignmentAndRevocationStuckOnCommit( - MembershipManagerImpl membershipManager, Uuid topicId, String topicName, - List partitions, boolean mockMetadata) { + MembershipManagerImpl membershipManager, Uuid topicId, String topicName, + List partitions, boolean mockMetadata) { CompletableFuture commitResult = mockRevocationNoCallbacks(true); if (mockMetadata) { when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); @@ -1977,7 +1977,7 @@ private void testRevocationCompleted(MembershipManagerImpl membershipManager, verify(subscriptionState).markPendingRevocation(anySet()); List expectedTopicPartitionAssignment = - buildTopicPartitions(expectedCurrentAssignment); + buildTopicPartitions(expectedCurrentAssignment); verify(subscriptionState).assignFromSubscribed(new HashSet<>(expectedTopicPartitionAssignment)); } @@ -2063,29 +2063,29 @@ private void receiveAssignment(Map> topicIdPartitionLis private void receiveAssignment(Uuid topicId, List partitions, MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(partitions))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions, MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(partitions))); + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topicId) + .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); + createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } private void receiveEmptyAssignment(MembershipManager membershipManager) { // New empty assignment received, revoking owned partition. ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.emptyList()); + .setTopicPartitions(Collections.emptyList()); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); } @@ -2117,7 +2117,7 @@ private void testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(Memb assertEquals(MemberState.LEAVING, membershipManager.state()); assertFalse(leaveResult.isDone(), "Leave group result should not complete until the " + - "heartbeat request to leave is sent out."); + "heartbeat request to leave is sent out."); membershipManager.onHeartbeatRequestSent(); @@ -2170,12 +2170,12 @@ private void testStateUpdateOnFatalFailure(MembershipManagerImpl membershipManag } private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse( - ConsumerGroupHeartbeatResponseData.Assignment assignment) { + ConsumerGroupHeartbeatResponseData.Assignment assignment) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH) - .setAssignment(assignment)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignment(assignment)); } /** @@ -2184,19 +2184,19 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse( * receives a heartbeat response to the join request, and the response includes an assignment. */ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithBumpedEpoch( - ConsumerGroupHeartbeatResponseData.Assignment assignment) { + ConsumerGroupHeartbeatResponseData.Assignment assignment) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH + 1) - .setAssignment(assignment)); + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH + 1) + .setAssignment(assignment)); } private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError() { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(5)); + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(5)); } private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean mockMetadata) { @@ -2209,14 +2209,14 @@ private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean m when(metadata.topicNames()).thenReturn(topicNames); } return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic1) - .setPartitions(Arrays.asList(0, 1, 2)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(topic2) - .setPartitions(Arrays.asList(3, 4, 5)) - )); + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic1) + .setPartitions(Arrays.asList(0, 1, 2)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(topic2) + .setPartitions(Arrays.asList(3, 4, 5)) + )); } private MembershipManagerImpl memberJoinWithAssignment() { @@ -2229,4 +2229,4 @@ private MembershipManagerImpl memberJoinWithAssignment() { assertFalse(membershipManager.currentAssignment().isEmpty()); return membershipManager; } -} \ No newline at end of file +} 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 0d1eb7b992d7e..4fdcf917d6c35 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 @@ -153,10 +153,10 @@ public NetworkClientDelegate newNetworkClientDelegate() { public NetworkClientDelegate.UnsentRequest newUnsentFindCoordinatorRequest() { Objects.requireNonNull(GROUP_ID); NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() - .setKey(GROUP_ID) - .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) - ), + new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() + .setKey(GROUP_ID) + .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id()) + ), Optional.empty() ); return req; @@ -171,4 +171,4 @@ public void prepareFindCoordinatorResponse(Errors error) { private Node mockNode() { return new Node(0, "localhost", 99); } -} \ No newline at end of file +} 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 b0e12b8526fc5..0998fba272f9e 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 @@ -112,28 +112,28 @@ public void setup() { time = new MockTime(0); apiVersions = mock(ApiVersions.class); requestManager = new OffsetsRequestManager( - subscriptionState, - metadata, - DEFAULT_ISOLATION_LEVEL, - time, - RETRY_BACKOFF_MS, - REQUEST_TIMEOUT_MS, - apiVersions, - mock(NetworkClientDelegate.class), - backgroundEventHandler, - logContext + subscriptionState, + metadata, + DEFAULT_ISOLATION_LEVEL, + time, + RETRY_BACKOFF_MS, + REQUEST_TIMEOUT_MS, + apiVersions, + mock(NetworkClientDelegate.class), + backgroundEventHandler, + logContext ); } @Test public void testListOffsetsRequest_Success() throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -144,12 +144,12 @@ public void testListOffsetsRequest_Success() throws ExecutionException, Interrup @Test public void testListOffsetsWaitingForMetadataUpdate_Timeout() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets(timestampsToSearch, false); + requestManager.fetchOffsets(timestampsToSearch, false); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); @@ -162,7 +162,7 @@ public void testListOffsetsWaitingForMetadataUpdate_Timeout() { @Test public void testListOffsetsRequestMultiplePartitions() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = new HashMap<>(); timestampsToSearch.put(TEST_PARTITION_1, ListOffsetsRequest.EARLIEST_TIMESTAMP); timestampsToSearch.put(TEST_PARTITION_2, ListOffsetsRequest.EARLIEST_TIMESTAMP); @@ -173,21 +173,21 @@ public void testListOffsetsRequestMultiplePartitions() throws ExecutionException partitionLeaders.put(TEST_PARTITION_2, LEADER_1); mockSuccessfulRequest(partitionLeaders); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); Map expectedOffsets = timestampsToSearch.entrySet().stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); verifySuccessfulPollAndResponseReceived(result, expectedOffsets); } @Test public void testListOffsetsRequestEmpty() throws ExecutionException, InterruptedException { CompletableFuture> result = requestManager.fetchOffsets( - Collections.emptyMap(), - false); + Collections.emptyMap(), + false); assertEquals(0, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -204,19 +204,19 @@ public void testListOffsetsRequestEmpty() throws ExecutionException, Interrupted @Test public void testListOffsetsRequestUnknownOffset() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> result = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); List topicResponses = Collections.singletonList( - mockUnknownOffsetResponse(TEST_PARTITION_1)); + mockUnknownOffsetResponse(TEST_PARTITION_1)); NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(retriedPoll); @@ -225,21 +225,21 @@ public void testListOffsetsRequestUnknownOffset() throws ExecutionException, clientResponse.onComplete(); Map expectedOffsets = - Collections.singletonMap(TEST_PARTITION_1, null); + Collections.singletonMap(TEST_PARTITION_1, null); verifyRequestSuccessfullyCompleted(result, expectedOffsets); } @Test public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws ExecutionException, - InterruptedException { + InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // Building list offsets request fails with unknown leader mockFailedRequest_MissingLeader(); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets(timestampsToSearch, - false); + requestManager.fetchOffsets(timestampsToSearch, + false); assertEquals(0, requestManager.requestsToSend()); assertEquals(1, requestManager.requestsToRetry()); verify(metadata).requestUpdate(true); @@ -255,7 +255,7 @@ public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws Execu assertEquals(1, requestManager.requestsToSend()); Map expectedOffsets = Collections.singletonMap( - TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); } @@ -263,13 +263,13 @@ public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws Execu @MethodSource("retriableErrors") public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -281,8 +281,8 @@ public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throw // Response received with error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, - Collections.singletonMap(TEST_PARTITION_1, error)); + unsentRequest, + Collections.singletonMap(TEST_PARTITION_1, error)); clientResponse.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); assertEquals(1, requestManager.requestsToRetry()); @@ -299,25 +299,25 @@ public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throw @Test public void testRequestNotSupportedErrorReturnsNullOffset() throws ExecutionException, - InterruptedException { + InterruptedException { testResponseWithErrorCodeAndUnknownOffsets(Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT); } @Test public void testRequestWithUnknownOffsetInResponseReturnsNullOffset() throws ExecutionException, - InterruptedException { + InterruptedException { testResponseWithErrorCodeAndUnknownOffsets(Errors.NONE); } private void testResponseWithErrorCodeAndUnknownOffsets(Errors error) throws ExecutionException, InterruptedException { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -329,8 +329,8 @@ private void testResponseWithErrorCodeAndUnknownOffsets(Errors error) throws Exe // Response received with error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, - Collections.singletonMap(TEST_PARTITION_1, error)); + unsentRequest, + Collections.singletonMap(TEST_PARTITION_1, error)); clientResponse.onComplete(); // Null offsets should be returned for each partition @@ -345,7 +345,7 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E timestampsToSearch.put(TEST_PARTITION_2, ListOffsetsRequest.EARLIEST_TIMESTAMP); Map expectedOffsets = timestampsToSearch.entrySet().stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); // List offsets request to 2 brokers successfully built Map partitionLeaders = new HashMap<>(); @@ -353,8 +353,8 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E partitionLeaders.put(TEST_PARTITION_2, LEADER_2); mockSuccessfulRequest(partitionLeaders); CompletableFuture> fetchOffsetsFuture = requestManager.fetchOffsets( - timestampsToSearch, - false); + timestampsToSearch, + false); assertEquals(2, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -367,13 +367,13 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E // broker but retriable UNKNOWN_LEADER_EPOCH received from second broker. NetworkClientDelegate.UnsentRequest unsentRequest1 = res.unsentRequests.get(0); ClientResponse clientResponse1 = buildClientResponse( - unsentRequest1, - Collections.singletonMap(TEST_PARTITION_1, expectedOffsets.get(TEST_PARTITION_1))); + unsentRequest1, + Collections.singletonMap(TEST_PARTITION_1, expectedOffsets.get(TEST_PARTITION_1))); clientResponse1.onComplete(); NetworkClientDelegate.UnsentRequest unsentRequest2 = res.unsentRequests.get(1); ClientResponse clientResponse2 = buildClientResponseWithErrors( - unsentRequest2, - Collections.singletonMap(TEST_PARTITION_2, Errors.UNKNOWN_LEADER_EPOCH)); + unsentRequest2, + Collections.singletonMap(TEST_PARTITION_2, Errors.UNKNOWN_LEADER_EPOCH)); clientResponse2.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); @@ -390,7 +390,7 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E verifySuccessfulPollAwaitingResponse(retriedPoll); NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponse(unsentRequest, - Collections.singletonMap(TEST_PARTITION_2, expectedOffsets.get(TEST_PARTITION_2))); + Collections.singletonMap(TEST_PARTITION_2, expectedOffsets.get(TEST_PARTITION_2))); clientResponse.onComplete(); // Verify global result with the offset initially retrieved, and the offset that @@ -401,14 +401,14 @@ public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() throws E @Test public void testRequestFailedResponse_NonRetriableAuthError() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -419,7 +419,7 @@ public void testRequestFailedResponse_NonRetriableAuthError() { // Response received with non-retriable auth error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.TOPIC_AUTHORIZATION_FAILED)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.TOPIC_AUTHORIZATION_FAILED)); clientResponse.onComplete(); verifyRequestCompletedWithErrorResponse(fetchOffsetsFuture, TopicAuthorizationException.class); @@ -430,14 +430,14 @@ public void testRequestFailedResponse_NonRetriableAuthError() { @Test public void testRequestFailedResponse_NonRetriableErrorTimeout() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -448,7 +448,7 @@ public void testRequestFailedResponse_NonRetriableErrorTimeout() { // Response received NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.BROKER_NOT_AVAILABLE)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.BROKER_NOT_AVAILABLE)); clientResponse.onComplete(); assertFalse(fetchOffsetsFuture.isDone()); @@ -462,14 +462,14 @@ public void testRequestFailedResponse_NonRetriableErrorTimeout() { @Test public void testRequestFails_AuthenticationException() { Map timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, - ListOffsetsRequest.EARLIEST_TIMESTAMP); + ListOffsetsRequest.EARLIEST_TIMESTAMP); // List offsets request successfully built mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); CompletableFuture> fetchOffsetsFuture = - requestManager.fetchOffsets( - timestampsToSearch, - false); + requestManager.fetchOffsets( + timestampsToSearch, + false); assertEquals(1, requestManager.requestsToSend()); assertEquals(0, requestManager.requestsToRetry()); @@ -480,7 +480,7 @@ public void testRequestFails_AuthenticationException() { // Response received with auth error NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = - buildClientResponseWithAuthenticationException(unsentRequest); + buildClientResponseWithAuthenticationException(unsentRequest); clientResponse.onComplete(); // Request completed with error. Nothing pending to be sent or retried @@ -515,7 +515,7 @@ public void testResetPositionsSuccess_NoLeaderEpochInResponse() { @Test public void testResetPositionsSuccess_LeaderEpochInResponse() { Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(5)); + Optional.of(5)); testResetPositionsSuccessWithLeaderEpoch(leaderAndEpoch); verify(metadata).updateLastSeenEpochIfNewer(TEST_PARTITION_1, leaderAndEpoch.epoch.get()); } @@ -533,7 +533,7 @@ public void testResetOffsetsAuthorizationFailure() { NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); Errors topicAuthorizationFailedError = Errors.TOPIC_AUTHORIZATION_FAILED; ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError)); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); @@ -566,10 +566,10 @@ public void testValidatePositionsSuccess() { int currentOffset = 5; int expectedEndOffset = 100; Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(3)); + Optional.of(3)); TopicPartition tp = TEST_PARTITION_1; SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(currentOffset, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); @@ -583,7 +583,7 @@ public void testValidatePositionsSuccess() { NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = pollResult.unsentRequests.get(0); ClientResponse clientResponse = buildOffsetsForLeaderEpochResponse(unsentRequest, - Collections.singletonList(tp), expectedEndOffset); + Collections.singletonList(tp), expectedEndOffset); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); assertFalse(unsentRequest.future().isCompletedExceptionally()); @@ -593,9 +593,9 @@ public void testValidatePositionsSuccess() { @Test public void testValidatePositionsMissingLeader() { Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(Node.noNode()), - Optional.of(5)); + Optional.of(5)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(5L, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); NodeApiVersions nodeApiVersions = NodeApiVersions.create(); @@ -608,9 +608,9 @@ public void testValidatePositionsMissingLeader() { @Test public void testValidatePositionsFailureWithUnrecoverableAuthException() { Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(5)); + Optional.of(5)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(5L, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); mockSuccessfulBuildRequestForValidatingPositions(position, LEADER_1); requestManager.validatePositionsIfNeeded(); @@ -619,7 +619,7 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); ClientResponse clientResponse = - buildOffsetsForLeaderEpochResponseWithErrors(unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); + buildOffsetsForLeaderEpochResponseWithErrors(unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); @@ -635,9 +635,9 @@ public void testValidatePositionsFailureWithUnrecoverableAuthException() { public void testValidatePositionsAbortIfNoApiVersionsToCheckAgainstThenRecovers() { int currentOffset = 5; Metadata.LeaderAndEpoch leaderAndEpoch = new Metadata.LeaderAndEpoch(Optional.of(LEADER_1), - Optional.of(3)); + Optional.of(3)); SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(currentOffset, - Optional.of(10), leaderAndEpoch); + Optional.of(10), leaderAndEpoch); when(subscriptionState.partitionsNeedingValidation(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.position(any())).thenReturn(position, position); @@ -670,7 +670,7 @@ private void testResetPositionsSuccessWithLeaderEpoch(Metadata.LeaderAndEpoch le OffsetResetStrategy strategy = OffsetResetStrategy.EARLIEST; long offset = 5L; Map expectedOffsets = Collections.singletonMap(tp, - new OffsetAndTimestamp(offset, 1L, leaderAndEpoch.epoch)); + new OffsetAndTimestamp(offset, 1L, leaderAndEpoch.epoch)); when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(tp)); when(subscriptionState.resetStrategy(any())).thenReturn(strategy); mockSuccessfulRequest(Collections.singletonMap(tp, leader)); @@ -689,34 +689,34 @@ private void testResetPositionsSuccessWithLeaderEpoch(Metadata.LeaderAndEpoch le } private ListOffsetsResponseData.ListOffsetsTopicResponse mockUnknownOffsetResponse( - TopicPartition tp) { + TopicPartition tp) { return new ListOffsetsResponseData.ListOffsetsTopicResponse() - .setName(tp.topic()) - .setPartitions(Collections.singletonList(new ListOffsetsResponseData.ListOffsetsPartitionResponse() - .setPartitionIndex(tp.partition()) - .setErrorCode(Errors.NONE.code()) - .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))); + .setName(tp.topic()) + .setPartitions(Collections.singletonList(new ListOffsetsResponseData.ListOffsetsPartitionResponse() + .setPartitionIndex(tp.partition()) + .setErrorCode(Errors.NONE.code()) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))); } private static Stream retriableErrors() { return Stream.of( - Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER), - Arguments.of(Errors.REPLICA_NOT_AVAILABLE), - Arguments.of(Errors.KAFKA_STORAGE_ERROR), - Arguments.of(Errors.OFFSET_NOT_AVAILABLE), - Arguments.of(Errors.LEADER_NOT_AVAILABLE), - Arguments.of(Errors.FENCED_LEADER_EPOCH), - Arguments.of(Errors.BROKER_NOT_AVAILABLE), - Arguments.of(Errors.INVALID_REQUEST), - Arguments.of(Errors.UNKNOWN_LEADER_EPOCH), - Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION)); + Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER), + Arguments.of(Errors.REPLICA_NOT_AVAILABLE), + Arguments.of(Errors.KAFKA_STORAGE_ERROR), + Arguments.of(Errors.OFFSET_NOT_AVAILABLE), + Arguments.of(Errors.LEADER_NOT_AVAILABLE), + Arguments.of(Errors.FENCED_LEADER_EPOCH), + Arguments.of(Errors.BROKER_NOT_AVAILABLE), + Arguments.of(Errors.INVALID_REQUEST), + Arguments.of(Errors.UNKNOWN_LEADER_EPOCH), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION)); } private void verifySuccessfulPollAndResponseReceived( - CompletableFuture> actualResult, - Map expectedResult) throws ExecutionException, - InterruptedException { + CompletableFuture> actualResult, + Map expectedResult) throws ExecutionException, + InterruptedException { // Following poll should send the request and get a response NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); verifySuccessfulPollAwaitingResponse(retriedPoll); @@ -729,7 +729,7 @@ private void verifySuccessfulPollAndResponseReceived( private void mockSuccessfulRequest(Map partitionLeaders) { partitionLeaders.forEach((tp, broker) -> { when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(broker, - Metadata.LeaderAndEpoch.noLeaderOrEpoch().epoch)); + Metadata.LeaderAndEpoch.noLeaderOrEpoch().epoch)); when(subscriptionState.isAssigned(tp)).thenReturn(true); }); when(metadata.fetch()).thenReturn(testClusterMetadata(partitionLeaders)); @@ -737,7 +737,7 @@ private void mockSuccessfulRequest(Map partitionLeaders) { private void mockFailedRequest_MissingLeader() { when(metadata.currentLeader(any(TopicPartition.class))).thenReturn( - new Metadata.LeaderAndEpoch(Optional.empty(), Optional.of(1))); + new Metadata.LeaderAndEpoch(Optional.empty(), Optional.of(1))); when(subscriptionState.isAssigned(any(TopicPartition.class))).thenReturn(true); } @@ -753,8 +753,8 @@ private void verifySuccessfulPollAwaitingResponse(NetworkClientDelegate.PollResu } private void verifyRequestSuccessfullyCompleted( - CompletableFuture> actualResult, - Map expectedResult) throws ExecutionException, InterruptedException { + CompletableFuture> actualResult, + Map expectedResult) throws ExecutionException, InterruptedException { assertEquals(0, requestManager.requestsToRetry()); assertEquals(0, requestManager.requestsToSend()); @@ -765,8 +765,8 @@ private void verifyRequestSuccessfullyCompleted( // Validate that the subscription state has been updated for all non-null offsets retrieved Map validExpectedOffsets = expectedResult.entrySet().stream() - .filter(entry -> entry.getValue() != null) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .filter(entry -> entry.getValue() != null) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); verifySubscriptionStateUpdated(validExpectedOffsets); } @@ -775,7 +775,7 @@ private void verifySubscriptionStateUpdated(Map offsetCaptor = ArgumentCaptor.forClass(Long.class); verify(subscriptionState, times(expectedResult.size())).updateLastStableOffset(tpCaptor.capture(), - offsetCaptor.capture()); + offsetCaptor.capture()); List updatedTp = tpCaptor.getAllValues(); List updatedOffsets = offsetCaptor.getAllValues(); @@ -784,8 +784,8 @@ private void verifySubscriptionStateUpdated(Map updatedOffsets.contains(offsetAndTimestamp.offset())) - .forEach(Assertions::assertTrue); + .map(offsetAndTimestamp -> updatedOffsets.contains(offsetAndTimestamp.offset())) + .forEach(Assertions::assertTrue); } private void verifyRequestCompletedWithErrorResponse(CompletableFuture> actualResult, @@ -802,27 +802,27 @@ private Metadata.LeaderAndEpoch testLeaderEpoch(Node leader, Optional e private Cluster testClusterMetadata(Map partitionLeaders) { List partitions = - partitionLeaders.keySet().stream() - .map(tp -> new PartitionInfo(tp.topic(), tp.partition(), - partitionLeaders.get(tp), null, null)) - .collect(Collectors.toList()); + partitionLeaders.keySet().stream() + .map(tp -> new PartitionInfo(tp.topic(), tp.partition(), + partitionLeaders.get(tp), null, null)) + .collect(Collectors.toList()); return new Cluster("clusterId", partitionLeaders.values(), partitions, - Collections.emptySet(), - Collections.emptySet()); + Collections.emptySet(), + Collections.emptySet()); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionsOffsets) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionsOffsets) { List topicResponses = new - ArrayList<>(); + ArrayList<>(); partitionsOffsets.forEach((tp, offsetAndTimestamp) -> { ListOffsetsResponseData.ListOffsetsTopicResponse topicResponse = ListOffsetsResponse.singletonListOffsetsTopicResponse( - tp, Errors.NONE, - offsetAndTimestamp.timestamp(), - offsetAndTimestamp.offset(), - offsetAndTimestamp.leaderEpoch().orElse(ListOffsetsResponse.UNKNOWN_EPOCH)); + tp, Errors.NONE, + offsetAndTimestamp.timestamp(), + offsetAndTimestamp.offset(), + offsetAndTimestamp.leaderEpoch().orElse(ListOffsetsResponse.UNKNOWN_EPOCH)); topicResponses.add(topicResponse); }); @@ -830,9 +830,9 @@ private ClientResponse buildClientResponse( } private ClientResponse buildOffsetsForLeaderEpochResponse( - final NetworkClientDelegate.UnsentRequest request, - final List partitions, - final int endOffset) { + final NetworkClientDelegate.UnsentRequest request, + final List partitions, + final int endOffset) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); @@ -845,29 +845,29 @@ private ClientResponse buildOffsetsForLeaderEpochResponse( data.topics().add(topic); } topic.partitions().add(new OffsetForLeaderEpochResponseData.EpochEndOffset() - .setPartition(tp.partition()) - .setErrorCode(Errors.NONE.code()) - .setLeaderEpoch(3) - .setEndOffset(endOffset)); + .setPartition(tp.partition()) + .setErrorCode(Errors.NONE.code()) + .setLeaderEpoch(3) + .setEndOffset(endOffset)); }); OffsetsForLeaderEpochResponse response = new OffsetsForLeaderEpochResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), - request.handler(), - "-1", - time.milliseconds(), - time.milliseconds(), - false, - null, - null, - response + new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response ); } private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionErrors) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionErrors) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); @@ -880,79 +880,79 @@ private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( data.topics().add(topic); } topic.partitions().add(new OffsetForLeaderEpochResponseData.EpochEndOffset() - .setPartition(tp.partition()) - .setErrorCode(partitionErrors.get(tp).code())); + .setPartition(tp.partition()) + .setErrorCode(partitionErrors.get(tp).code())); }); OffsetsForLeaderEpochResponse response = new OffsetsForLeaderEpochResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), - request.handler(), - "-1", - time.milliseconds(), - time.milliseconds(), - false, - null, - null, - response + new RequestHeader(ApiKeys.OFFSET_FOR_LEADER_EPOCH, offsetsForLeaderEpochRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response ); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final List topicResponses) { + final NetworkClientDelegate.UnsentRequest request, + final List topicResponses) { return buildClientResponse(request, topicResponses, false, null); } private ClientResponse buildClientResponseWithErrors( - final NetworkClientDelegate.UnsentRequest request, - final Map partitionErrors) { + final NetworkClientDelegate.UnsentRequest request, + final Map partitionErrors) { List topicResponses = new ArrayList<>(); partitionErrors.forEach((tp, error) -> topicResponses.add(ListOffsetsResponse.singletonListOffsetsTopicResponse( - tp, - error, - ListOffsetsResponse.UNKNOWN_TIMESTAMP, - ListOffsetsResponse.UNKNOWN_OFFSET, - ListOffsetsResponse.UNKNOWN_EPOCH))); + tp, + error, + ListOffsetsResponse.UNKNOWN_TIMESTAMP, + ListOffsetsResponse.UNKNOWN_OFFSET, + ListOffsetsResponse.UNKNOWN_EPOCH))); return buildClientResponse(request, topicResponses, false, null); } private ClientResponse buildClientResponseWithAuthenticationException( - final NetworkClientDelegate.UnsentRequest request) { + final NetworkClientDelegate.UnsentRequest request) { return buildClientResponse(request, Collections.emptyList(), true, - new AuthenticationException("Authentication failed")); + new AuthenticationException("Authentication failed")); } private ClientResponse buildClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final List topicResponses, - final boolean disconnected, - final AuthenticationException authenticationException) { + final NetworkClientDelegate.UnsentRequest request, + final List topicResponses, + final boolean disconnected, + final AuthenticationException authenticationException) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof ListOffsetsRequest); ListOffsetsRequest offsetFetchRequest = (ListOffsetsRequest) abstractRequest; ListOffsetsResponse response = buildListOffsetsResponse(topicResponses); return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), - request.handler(), - "-1", - time.milliseconds(), - time.milliseconds(), - disconnected, - null, - authenticationException, - response + new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + disconnected, + null, + authenticationException, + response ); } private ListOffsetsResponse buildListOffsetsResponse( - List offsetsTopicResponses) { + List offsetsTopicResponses) { ListOffsetsResponseData responseData = new ListOffsetsResponseData() - .setThrottleTimeMs(0) - .setTopics(offsetsTopicResponses); + .setThrottleTimeMs(0) + .setTopics(offsetsTopicResponses); return new ListOffsetsResponse(responseData); } -} \ No newline at end of file +} From b6907ab304736c0e362113deff2b793b2768a5d5 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:18:48 -0800 Subject: [PATCH 043/130] Whitspace --- .../clients/consumer/internals/ConsumerNetworkThreadTest.java | 4 ++-- .../clients/consumer/internals/OffsetsRequestManagerTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 0be113f76aa3f..668bef17c9fbc 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 @@ -128,12 +128,12 @@ public void testStartupAndTearDown() throws InterruptedException { // There's a nonzero amount of time between starting the thread and having it // begin to execute our code. Wait for a bit before checking... TestUtils.waitForCondition(isStarted, - "The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms"); + "The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms"); consumerNetworkThread.close(Duration.ofMillis(DEFAULT_MAX_WAIT_MS)); TestUtils.waitForCondition(isClosed, - "The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms"); + "The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms"); } @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 0998fba272f9e..5ca034d636029 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 @@ -955,4 +955,4 @@ private ListOffsetsResponse buildListOffsetsResponse( return new ListOffsetsResponse(responseData); } -} +} \ No newline at end of file From 567e3f86ba2bb266bd27b4da417e6e47d93575e9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:20:15 -0800 Subject: [PATCH 044/130] Whitespace --- .../internals/ConsumerNetworkThreadTest.java | 2 +- .../TopicMetadataRequestManagerTest.java | 42 +++++++++---------- 2 files changed, 22 insertions(+), 22 deletions(-) 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 668bef17c9fbc..10b6edeb67cd5 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 @@ -353,4 +353,4 @@ private HashMap mockTopicPartitionOffset() { topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); return topicPartitionOffsets; } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index 9992b8b7d5c71..031ecfdd0ac5f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -121,8 +121,8 @@ public void testAllTopicsExceptionAndInflightRequests(final Errors error, final this.time.sleep(100); NetworkClientDelegate.PollResult res = this.topicMetadataRequestManager.poll(this.time.milliseconds()); res.unsentRequests.get(0).future().complete(buildAllTopicsMetadataClientResponse( - res.unsentRequests.get(0), - error)); + res.unsentRequests.get(0), + error)); List inflights = this.topicMetadataRequestManager.inflightRequests(); if (shouldRetry) { @@ -218,30 +218,30 @@ public void testNetworkTimeout() { } private ClientResponse buildTopicMetadataClientResponse( - final NetworkClientDelegate.UnsentRequest request, - final String topic, - final Errors error) { + final NetworkClientDelegate.UnsentRequest request, + final String topic, + final Errors error) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertTrue(abstractRequest instanceof MetadataRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; Cluster cluster = mockCluster(3, 0); List topics = new ArrayList<>(); topics.add(new MetadataResponse.TopicMetadata(error, topic, false, - Collections.emptyList())); + Collections.emptyList())); final MetadataResponse metadataResponse = RequestTestUtils.metadataResponse(cluster.nodes(), - cluster.clusterResource().clusterId(), - cluster.controller().id(), - topics); + cluster.clusterResource().clusterId(), + cluster.controller().id(), + topics); return new ClientResponse( - new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), - request.handler(), - "-1", - time.milliseconds(), - time.milliseconds(), - false, - null, - null, - metadataResponse); + new RequestHeader(ApiKeys.METADATA, metadataRequest.version(), "mockClientId", 1), + request.handler(), + "-1", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + metadataResponse); } private ClientResponse buildAllTopicsMetadataClientResponse( @@ -293,9 +293,9 @@ private static Collection exceptionProvider() { private static Collection hardFailureExceptionProvider() { return Arrays.asList( - Arguments.of(new TimeoutException("timeout")), - Arguments.of(new KafkaException("non-retriable exception")), - Arguments.of(new NetworkException("retriable-exception"))); + Arguments.of(new TimeoutException("timeout")), + Arguments.of(new KafkaException("non-retriable exception")), + Arguments.of(new NetworkException("retriable-exception"))); } } \ No newline at end of file From 0bd6c6654623672c96431620d67beac1881b6c89 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:21:22 -0800 Subject: [PATCH 045/130] Delete RelaxedCompletableFuture.java --- .../internals/RelaxedCompletableFuture.java | 75 ------------------- 1 file changed, 75 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java deleted file mode 100644 index 6a4c4070913ac..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RelaxedCompletableFuture.java +++ /dev/null @@ -1,75 +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 java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -public class RelaxedCompletableFuture extends CompletableFuture { - - private final Lock lock; - private final Condition attemptedCondition; - private boolean wasAttempted; - - public RelaxedCompletableFuture() { - this.lock = new ReentrantLock(); - this.attemptedCondition = lock.newCondition(); - } - - public void attempted() { - try { - lock.lock(); - wasAttempted = true; - attemptedCondition.signalAll(); - } finally { - lock.unlock(); - } - } - - @Override - public T get() throws InterruptedException, ExecutionException { - waitForAttempt(); - return super.get(); - } - - @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - waitForAttempt(); - - if (super.isDone()) { - return super.get(); - } else { - return super.get(timeout, unit); - } - } - - private void waitForAttempt() throws InterruptedException { - try { - lock.lock(); - - if (!wasAttempted) - attemptedCondition.await(); - } finally { - lock.unlock(); - } - } -} From 3e73bb79d6bc4b25a156e58131b11521db0d2f0d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:26:16 -0800 Subject: [PATCH 046/130] Updates --- .../events/ApplicationEventProcessor.java | 63 ++++--------------- .../events/CompletableApplicationEvent.java | 10 +++ 2 files changed, 22 insertions(+), 51 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 4c764a8bd2e3c..3c4d91c8c2598 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -27,10 +26,8 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; 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.List; @@ -152,13 +149,14 @@ private void process(final PollApplicationEvent event) { private void process(final AsyncCommitApplicationEvent event) { CommitRequestManager manager = requestManagers.commitRequestManager.get(); CompletableFuture commitResult = manager.commitAsync(event.offsets()); - chain(commitResult, event.future()); + event.chain(commitResult); } private void process(final SyncCommitApplicationEvent event) { CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture commitResult = manager.commitSync(event.offsets(), event.deadlineMs()); - chain(commitResult, event.future()); + long expirationTimeoutMs = event.deadlineMs(); + CompletableFuture commitResult = manager.commitSync(event.offsets(), expirationTimeoutMs); + event.chain(commitResult); } private void process(final FetchCommittedOffsetsApplicationEvent event) { @@ -168,8 +166,8 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture> future = manager.fetchOffsets(event.partitions(), event.deadlineMs()); - chain(future, event.future()); + long expirationTimeoutMs = event.deadlineMs(); + event.chain(manager.fetchOffsets(event.partitions(), expirationTimeoutMs)); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { @@ -194,7 +192,7 @@ private void process(final ListOffsetsApplicationEvent event) { final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), event.requireTimestamps()); - chain(future, event.future()); + event.chain(future); } /** @@ -227,17 +225,17 @@ private void process(final UnsubscribeApplicationEvent event) { } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); CompletableFuture result = membershipManager.leaveGroup(); - chain(result, event.future()); + event.chain(result); } private void process(final ResetPositionsApplicationEvent event) { CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); - chain(result, event.future()); + event.chain(result); } private void process(final ValidatePositionsApplicationEvent event) { CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); - chain(result, event.future()); + event.chain(result); } private void process(final TopicMetadataApplicationEvent event) { @@ -249,7 +247,7 @@ private void process(final TopicMetadataApplicationEvent event) { future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), event.deadlineMs()); } - chain(future, event.future()); + event.chain(future); } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { @@ -282,44 +280,7 @@ private void process(final LeaveOnCloseApplicationEvent event) { log.debug("Leaving group before closing"); CompletableFuture future = membershipManager.leaveGroup(); // The future will be completed on heartbeat sent - chain(future, event.future()); - } - - /** - * Creates a {@link Timer time} for the network I/O thread that is separate from the timer for the - * application thread. - */ - private Timer timer(CompletableApplicationEvent event) { - return time.timer(event.deadlineMs() - time.milliseconds()); - } - - private boolean maybeTimeout(CompletableApplicationEvent event, Timer timer, String timeoutMessage) { - if (timer.isExpired()) { - Exception exception = new TimeoutException(timeoutMessage); - event.future().completeExceptionally(exception); - return true; - } else { - return false; - } - } - - private void chain(final CompletableFuture primary, final CompletableFuture secondary) { - Objects.requireNonNull( - primary, - () -> String.format("Could not chain the secondary future (%s) to the primary future because the primary future was null", secondary) - ); - Objects.requireNonNull( - secondary, - () -> String.format("Could not chain the secondary future to the primary future (%s) because the secondary future was null", primary) - ); - - primary.whenComplete((value, exception) -> { - if (exception != null) { - secondary.completeExceptionally(exception); - } else { - secondary.complete(value); - } - }); + event.chain(future); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 06a30f3cc6077..5ad2855ab6bed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -54,6 +54,16 @@ public T get() { return ConsumerUtils.getResult(future, timer); } + public void chain(final CompletableFuture providedFuture) { + providedFuture.whenComplete((value, exception) -> { + if (exception != null) { + this.future.completeExceptionally(exception); + } else { + this.future.complete(value); + } + }); + } + @Override public boolean equals(Object o) { if (this == o) return true; From e4c0486ffb37036e36bc5053900e4b36156615da Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:33:26 -0800 Subject: [PATCH 047/130] Updates --- .../internals/AsyncKafkaConsumer.java | 21 +++-- .../events/ApplicationEventHandler.java | 8 +- .../events/ApplicationEventProcessor.java | 14 ++- .../events/CompletableApplicationEvent.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 86 +++++++++---------- .../internals/ConsumerTestBuilder.java | 1 - .../events/ApplicationEventProcessorTest.java | 1 - 7 files changed, 61 insertions(+), 74 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 c5b1cd44efda2..e2dd2d19aa0f3 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 @@ -383,7 +383,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { metrics, offsetCommitCallbackInvoker); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, - time, metadata, applicationEventQueue, requestManagersSupplier); @@ -571,7 +570,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, - time, metadata, applicationEventQueue, requestManagersSupplier @@ -944,7 +942,7 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event); + final Map committedOffsets = applicationEventHandler.addAndGet(event, timer); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -996,7 +994,7 @@ public List partitionsFor(String topic, Duration timeout) { wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent, timer); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1025,7 +1023,7 @@ public Map> listTopics(Duration timeout) { new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, timer); } finally { wakeupTrigger.clearTask(); } @@ -1104,7 +1102,7 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent); + listOffsetsEvent, + timer); return offsetAndTimestampMap .entrySet() .stream() @@ -1278,7 +1277,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer), timer); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1574,7 +1573,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer)); + applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer), timer); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1597,7 +1596,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer)); + applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer), timer); return true; } catch (TimeoutException e) { return false; @@ -1631,7 +1630,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { new FetchCommittedOffsetsApplicationEvent( initializingPartitions, timer); - final Map offsets = applicationEventHandler.addAndGet(event); + final Map offsets = applicationEventHandler.addAndGet(event, timer); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { 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 f7ac2e30481be..867aa88742765 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 @@ -22,6 +22,7 @@ import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -98,16 +99,17 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#get()} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread + * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event) { + public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); add(event); - return event.get(); + return event.get(timer); } @Override 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 3c4d91c8c2598..b3544d159b5ff 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 @@ -44,18 +44,15 @@ public class ApplicationEventProcessor extends EventProcessor { private final Logger log; - private final Time time; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; public ApplicationEventProcessor(final LogContext logContext, - final Time time, final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); - this.time = time; this.requestManagers = requestManagers; this.metadata = metadata; } @@ -166,8 +163,8 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = event.deadlineMs(); - event.chain(manager.fetchOffsets(event.partitions(), expirationTimeoutMs)); + long expirationTimeMs = event.deadlineMs(); + event.chain(manager.fetchOffsets(event.partitions(), expirationTimeMs)); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { @@ -241,10 +238,11 @@ private void process(final ValidatePositionsApplicationEvent event) { private void process(final TopicMetadataApplicationEvent event) { final CompletableFuture>> future; + long expirationTimeMs = event.deadlineMs(); if (event.isAllTopics()) { - future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(event.deadlineMs()); + future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); } else { - future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), event.deadlineMs()); + future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); } event.chain(future); @@ -288,7 +286,6 @@ private void process(final LeaveOnCloseApplicationEvent event) { * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, - final Time time, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, final Supplier requestManagersSupplier) { @@ -298,7 +295,6 @@ protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); return new ApplicationEventProcessor( logContext, - time, applicationEventQueue, requestManagers, metadata diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 5ad2855ab6bed..72d664b33832e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -31,13 +31,11 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; - private final Timer timer; private final long deadlineMs; protected CompletableApplicationEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); - this.timer = timer; this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } @@ -50,7 +48,7 @@ public long deadlineMs() { return deadlineMs; } - public T get() { + public T get(Timer timer) { return ConsumerUtils.getResult(future, timer); } 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 745ea35acaa1b..66cbc54cdacae 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 @@ -161,10 +161,6 @@ public void resetAll() { } private AsyncKafkaConsumer newConsumer() { - return newConsumer(time); - } - - private AsyncKafkaConsumer newConsumer(Time time) { final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); return newConsumer(props); @@ -186,7 +182,7 @@ private AsyncKafkaConsumer newConsumer(Properties props) { return newConsumer(config); } - private AsyncKafkaConsumer newConsumer(ConsumerConfig config, Time time) { + private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { return new AsyncKafkaConsumer<>( config, new StringDeserializer(), @@ -199,10 +195,6 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config, Ti ); } - private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { - return newConsumer(config, time); - } - private AsyncKafkaConsumer newConsumer( FetchBuffer fetchBuffer, ConsumerInterceptors interceptors, @@ -332,7 +324,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -354,7 +346,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); } @Test @@ -362,7 +354,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class))).thenAnswer(invocation -> { + any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); @@ -600,9 +592,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any()); + doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); } @@ -645,7 +637,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class)); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -778,14 +770,15 @@ public void testBeginningOffsets() { Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -795,23 +788,26 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class)); + any(ListOffsetsApplicationEvent.class), + any()); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -846,11 +842,12 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -868,22 +865,23 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test public void testWakeupCommitted() { - Time time = new MockTime(); - consumer = newConsumer(time); + consumer = newConsumer(); final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); + Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return event.get(); + return ConsumerUtils.getResult(event.future(), timer); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1303,10 +1301,7 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( - methodName, - partitions - ); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. @@ -1511,20 +1506,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } } @@ -1539,11 +1534,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test @@ -1592,8 +1587,8 @@ public void testLongPollWaitIsLimited() { */ @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { + consumer = newConsumer(); Time time = new MockTime(); - consumer = newConsumer(time); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); CountDownLatch latch = new CountDownLatch(3); @@ -1626,8 +1621,8 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { */ @Test public void testProcessBackgroundEventsWithoutDelay() { + consumer = newConsumer(); Time time = new MockTime(); - consumer = newConsumer(time); Timer timer = time.timer(1000); // Create a future that is already completed. @@ -1646,8 +1641,8 @@ public void testProcessBackgroundEventsWithoutDelay() { */ @Test public void testProcessBackgroundEventsTimesOut() throws Exception { + consumer = newConsumer(); Time time = new MockTime(); - consumer = newConsumer(time); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); @@ -1725,13 +1720,13 @@ private void completeCommitSyncApplicationEventSuccessfully() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { @@ -1747,4 +1742,3 @@ private void forceCommitCallbackInvocation() { consumer.commitAsync(); } } - diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 3d677734c6b25..4ed001518cb7e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -268,7 +268,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA ); this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, - time, applicationEventQueue, requestManagers, metadata diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 44a274627d9be..425d4e06f83f3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -87,7 +87,6 @@ public void setup() { ); processor = new ApplicationEventProcessor( logContext, - time, applicationEventQueue, requestManagers, metadata From a9ce15ae6823a1bd74ea929d2752b4cc5a19bad7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:41:47 -0800 Subject: [PATCH 048/130] Updates --- .../internals/AsyncKafkaConsumer.java | 3 +- .../events/ApplicationEventHandler.java | 1 + .../events/AsyncCommitApplicationEvent.java | 2 +- .../events/CommitApplicationEvent.java | 6 +- .../events/SyncCommitApplicationEvent.java | 5 +- .../internals/ConsumerNetworkThreadTest.java | 4 +- .../CoordinatorRequestManagerTest.java | 68 ++++++++----------- 7 files changed, 42 insertions(+), 47 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e2dd2d19aa0f3..1af7260231356 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 @@ -942,7 +942,8 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, timer); + final Map committedOffsets = applicationEventHandler.addAndGet(event, + timer); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { 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 867aa88742765..7535edf5970b3 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 @@ -108,6 +108,7 @@ public long maximumTimeToWait() { */ public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); + Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); return event.get(timer); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index 2472975ed4107..8190d18c2496d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -28,7 +28,7 @@ public class AsyncCommitApplicationEvent extends CommitApplicationEvent { public AsyncCommitApplicationEvent(final Map offsets, Timer timer) { - super(Type.COMMIT_ASYNC, timer, offsets); + super(offsets, Type.COMMIT_ASYNC, timer); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index 5da22be82f8ea..a6c182b09d06d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -30,9 +30,9 @@ public abstract class CommitApplicationEvent extends CompletableApplicationEvent */ private final Map offsets; - protected CommitApplicationEvent(final Type type, - final Timer timer, - final Map offsets) { + protected CommitApplicationEvent(final Map offsets, + final Type type, + final Timer timer) { super(type, timer); this.offsets = Collections.unmodifiableMap(offsets); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 71e5a8124c5e9..8e81a4102579f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -28,8 +28,9 @@ */ public class SyncCommitApplicationEvent extends CommitApplicationEvent { - public SyncCommitApplicationEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_SYNC, timer, offsets); + public SyncCommitApplicationEvent(final Map offsets, + final Timer timer) { + super(offsets, Type.COMMIT_SYNC, timer); } @Override 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 10b6edeb67cd5..2750825e07c78 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 @@ -290,8 +290,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new SyncCommitApplicationEvent(Collections.emptyMap(), timer)); - ApplicationEvent event2 = new SyncCommitApplicationEvent(Collections.emptyMap(), timer); + CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap(), timer)); + ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index e24e34972bc54..a3e3d22f42664 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -46,7 +45,6 @@ import static org.mockito.Mockito.verifyNoInteractions; public class CoordinatorRequestManagerTest { - private static final int DEFAULT_API_TIMEOUT_MS = 60000; private static final int RETRY_BACKOFF_MS = 500; private static final String GROUP_ID = "group-1"; private MockTime time; @@ -63,8 +61,7 @@ public void setup() { @Test public void testSuccessfulResponse() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); Optional coordinatorOpt = coordinatorManager.coordinator(); assertTrue(coordinatorOpt.isPresent()); @@ -72,7 +69,7 @@ public void testSuccessfulResponse() { assertEquals(node.host(), coordinatorOpt.get().host()); assertEquals(node.port(), coordinatorOpt.get().port()); - NetworkClientDelegate.PollResult pollResult = coordinatorManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult pollResult = coordinatorManager.poll(time.milliseconds()); assertEquals(Collections.emptyList(), pollResult.unsentRequests); } @@ -80,44 +77,41 @@ public void testSuccessfulResponse() { public void testMarkCoordinatorUnknown() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); assertTrue(coordinatorManager.coordinator().isPresent()); // It may take time for metadata to converge between after a coordinator has // been demoted. This can cause a tight loop in which FindCoordinator continues to // return node X while that node continues to reply with NOT_COORDINATOR. Hence we // still want to ensure a backoff after successfully finding the coordinator. - coordinatorManager.markCoordinatorUnknown("coordinator changed", timer.currentTimeMs()); - assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); + coordinatorManager.markCoordinatorUnknown("coordinator changed", time.milliseconds()); + assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); - timer.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); + time.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); - timer.sleep(RETRY_BACKOFF_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); + time.sleep(RETRY_BACKOFF_MS); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); assertTrue(coordinatorManager.coordinator().isPresent()); } @Test public void testBackoffAfterRetriableFailure() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS, timer); + expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS); verifyNoInteractions(backgroundEventHandler); - timer.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); + time.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); - timer.sleep(1); - expectFindCoordinatorRequest(coordinatorManager, Errors.NONE, timer); + time.sleep(1); + expectFindCoordinatorRequest(coordinatorManager, Errors.NONE); } @Test public void testPropagateAndBackoffAfterFatalError() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED, timer); + expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED); verify(backgroundEventHandler).add(argThat(backgroundEvent -> { if (!(backgroundEvent instanceof ErrorBackgroundEvent)) @@ -132,11 +126,11 @@ public void testPropagateAndBackoffAfterFatalError() { return groupAuthException.groupId().equals(GROUP_ID); })); - timer.sleep(RETRY_BACKOFF_MS - 1); - assertEquals(Collections.emptyList(), coordinatorManager.poll(timer.currentTimeMs()).unsentRequests); + time.sleep(RETRY_BACKOFF_MS - 1); + assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); - timer.sleep(1); - assertEquals(1, coordinatorManager.poll(timer.currentTimeMs()).unsentRequests.size()); + time.sleep(1); + assertEquals(1, coordinatorManager.poll(time.milliseconds()).unsentRequests.size()); assertEquals(Optional.empty(), coordinatorManager.coordinator()); } @@ -162,29 +156,27 @@ public void testFindCoordinatorResponseVersions() { @Test public void testNetworkTimeout() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); - Timer timer = time.timer(DEFAULT_API_TIMEOUT_MS); - NetworkClientDelegate.PollResult res = coordinatorManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = coordinatorManager.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); // Mimic a network timeout - res.unsentRequests.get(0).handler().onFailure(timer.currentTimeMs(), new TimeoutException()); + res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); // Sleep for exponential backoff - 1ms - timer.sleep(RETRY_BACKOFF_MS - 1); - NetworkClientDelegate.PollResult res2 = coordinatorManager.poll(timer.currentTimeMs()); + time.sleep(RETRY_BACKOFF_MS - 1); + NetworkClientDelegate.PollResult res2 = coordinatorManager.poll(this.time.milliseconds()); assertEquals(0, res2.unsentRequests.size()); - timer.sleep(1); - res2 = coordinatorManager.poll(timer.currentTimeMs()); + time.sleep(1); + res2 = coordinatorManager.poll(time.milliseconds()); assertEquals(1, res2.unsentRequests.size()); } private void expectFindCoordinatorRequest( CoordinatorRequestManager coordinatorManager, - Errors error, - Timer timer + Errors error ) { - NetworkClientDelegate.PollResult res = coordinatorManager.poll(timer.currentTimeMs()); + NetworkClientDelegate.PollResult res = coordinatorManager.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); @@ -219,12 +211,12 @@ private ClientResponse buildResponse( new RequestHeader(ApiKeys.FIND_COORDINATOR, findCoordinatorRequest.version(), "", 1), request.handler(), node.idString(), - request.timer().currentTimeMs(), - request.timer().currentTimeMs(), + time.milliseconds(), + time.milliseconds(), false, null, null, findCoordinatorResponse ); } -} +} \ No newline at end of file From c2b9b5216e04becb70985b61cfd61f91cc2429e0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:43:14 -0800 Subject: [PATCH 049/130] Updates --- .../consumer/internals/CoordinatorRequestManagerTest.java | 2 +- .../consumer/internals/TopicMetadataRequestManagerTest.java | 2 +- .../internals/events/ApplicationEventProcessorTest.java | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index a3e3d22f42664..d7ad1b55738c4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -219,4 +219,4 @@ private ClientResponse buildResponse( findCoordinatorResponse ); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index 031ecfdd0ac5f..c7b23150602c4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -298,4 +298,4 @@ private static Collection hardFailureExceptionProvider() { Arguments.of(new NetworkException("retriable-exception"))); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 425d4e06f83f3..235d8a84309cd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -40,7 +40,6 @@ import java.util.concurrent.CompletableFuture; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -76,7 +75,7 @@ public void setup() { heartbeatRequestManager = mock(HeartbeatRequestManager.class); membershipManager = mock(MembershipManager.class); requestManagers = new RequestManagers( - logContext, + new LogContext(), offsetsRequestManager, topicMetadataRequestManager, fetchRequestManager, From f4466443875bd17fe85cfb8a79d3e656456621db Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:43:58 -0800 Subject: [PATCH 050/130] Ugh --- .../internals/events/ApplicationEventProcessorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 235d8a84309cd..524e00daade85 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -75,7 +75,7 @@ public void setup() { heartbeatRequestManager = mock(HeartbeatRequestManager.class); membershipManager = mock(MembershipManager.class); requestManagers = new RequestManagers( - new LogContext(), + logContext, offsetsRequestManager, topicMetadataRequestManager, fetchRequestManager, @@ -85,7 +85,7 @@ public void setup() { Optional.of(membershipManager) ); processor = new ApplicationEventProcessor( - logContext, + new LogContext(), applicationEventQueue, requestManagers, metadata From 9940cf17fb99534caf536eb4a5f9cc03ed6b554d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:46:59 -0800 Subject: [PATCH 051/130] Update ConsumerNetworkThreadTest.java --- .../internals/ConsumerNetworkThreadTest.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) 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 2750825e07c78..70a74155779e4 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 @@ -170,9 +170,8 @@ public void testSyncCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { - Timer timer = time.timer(1000); Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, timer); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); @@ -181,8 +180,7 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - Timer timer = time.timer(1000); - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(timer); + ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); @@ -193,8 +191,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - Timer timer = time.timer(1000); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(timer); + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -203,8 +200,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - Timer timer = time.timer(1000); - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(timer); + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); @@ -229,8 +225,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - Timer timer = time.timer(1000); - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", timer)); + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", time.timer(Long.MAX_VALUE))); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); } @@ -285,11 +280,11 @@ void testEnsureMetadataUpdateOnPoll() { @Test void testEnsureEventsAreCompleted() { - Timer timer = time.timer(1000); Node node = metadata.fetch().nodes().get(0); coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); + Timer timer = time.timer(1000); CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap(), timer)); ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); From c1bf7d97c83d3d33fcc4892e96974af30ab271fe Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 21 Feb 2024 16:53:02 -0800 Subject: [PATCH 052/130] Update CommitApplicationEvent.java --- .../consumer/internals/events/CommitApplicationEvent.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index a6c182b09d06d..4c0cf59d70d8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -30,9 +30,7 @@ public abstract class CommitApplicationEvent extends CompletableApplicationEvent */ private final Map offsets; - protected CommitApplicationEvent(final Map offsets, - final Type type, - final Timer timer) { + protected CommitApplicationEvent(final Map offsets, Type type, Timer timer) { super(type, timer); this.offsets = Collections.unmodifiableMap(offsets); From 8fa83c693059c71b858a05582022cffa0e2a4ef4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 22 Feb 2024 10:33:23 -0800 Subject: [PATCH 053/130] Proof of concept moving timeout logic to network thread --- .../internals/AsyncKafkaConsumer.java | 22 +++---- .../internals/ConsumerNetworkThread.java | 21 ++++++- .../events/ApplicationEventHandler.java | 11 +--- .../events/ApplicationEventProcessor.java | 13 +++- .../events/CompletableApplicationEvent.java | 5 -- .../internals/AsyncKafkaConsumerTest.java | 61 ++++++++----------- 6 files changed, 70 insertions(+), 63 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 1af7260231356..9caebe654e679 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 @@ -942,8 +942,7 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - timer); + final Map committedOffsets = applicationEventHandler.addAndGet(event); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -995,7 +994,7 @@ public List partitionsFor(String topic, Duration timeout) { wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent, timer); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1024,7 +1023,7 @@ public Map> listTopics(Duration timeout) { new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, timer); + return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); } finally { wakeupTrigger.clearTask(); } @@ -1103,7 +1102,7 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent, - timer); + listOffsetsEvent); return offsetAndTimestampMap .entrySet() .stream() @@ -1278,7 +1276,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1359,7 +1357,7 @@ public void commitSync(Map offsets, Duration SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, requestTimer); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); - ConsumerUtils.getResult(commitFuture, requestTimer); + ConsumerUtils.getResult(commitFuture); interceptors.onCommit(offsets); } finally { wakeupTrigger.clearTask(); @@ -1574,7 +1572,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer), timer); + applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer)); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1597,7 +1595,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer), timer); + applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer)); return true; } catch (TimeoutException e) { return false; @@ -1631,7 +1629,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { new FetchCommittedOffsetsApplicationEvent( initializingPartitions, timer); - final Map offsets = applicationEventHandler.addAndGet(event, timer); + final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { 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 aa352cd68a22e..626fc278eeac5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -128,7 +130,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - applicationEventProcessor.process(); + final List events = applicationEventProcessor.process(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -144,6 +146,23 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); + + events.stream() + .filter(e -> e instanceof CompletableApplicationEvent) + .map(e -> (CompletableApplicationEvent) e) + .filter(e -> e.deadlineMs() < currentTimeMs && !e.future().isDone()) + .forEach(e -> { + long diff = currentTimeMs - e.deadlineMs(); + + TimeoutException exception = new TimeoutException("Operation could not be completed"); + log.warn("Event {} had a deadline of {} but was not complete by {}, which is {} ms. past, completing with exception {}", + e, + e.deadlineMs(), + currentTimeMs, + diff, + exception.getMessage()); + e.future().completeExceptionally(exception); + }); } /** 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 7535edf5970b3..b82b17ea50b10 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 @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; @@ -97,20 +98,14 @@ public long maximumTimeToWait() { * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will * return the result value upon successful completion; otherwise throws an error. * - *

- * - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. - * * @param event A {@link CompletableApplicationEvent} created by the polling thread - * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { + public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); - Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return event.get(timer); + return ConsumerUtils.getResult(event.future()); } @Override 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 b3544d159b5ff..a790b98e27ceb 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 @@ -27,9 +27,9 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -62,8 +62,15 @@ public ApplicationEventProcessor(final LogContext logContext, * an event generates an error. In such cases, the processor will log an exception, but we do not want those * errors to be propagated to the caller. */ - public boolean process() { - return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); + public List process() { + List events = new ArrayList<>(); + + process((event, error) -> { + events.add(event); + error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); + }); + + return events; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 72d664b33832e..a7f23237c35d8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; import java.util.Objects; @@ -48,10 +47,6 @@ public long deadlineMs() { return deadlineMs; } - public T get(Timer timer) { - return ConsumerUtils.getResult(future, timer); - } - public void chain(final CompletableFuture providedFuture) { providedFuture.whenComplete((value, exception) -> { if (exception != null) { 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 66cbc54cdacae..7baffb92f9e80 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 @@ -324,7 +324,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -346,7 +346,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); } @Test @@ -354,7 +354,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsApplicationEvent.class))).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); @@ -592,9 +592,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(null).when(applicationEventHandler).addAndGet(any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); } @@ -637,7 +637,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -770,15 +770,14 @@ public void testBeginningOffsets() { Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -788,26 +787,23 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class), - any()); + any(ListOffsetsApplicationEvent.class)); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -842,12 +838,11 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -865,8 +860,7 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -875,13 +869,12 @@ public void testWakeupCommitted() { final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future(), timer); + return ConsumerUtils.getResult(event.future()); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1506,20 +1499,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } } @@ -1534,11 +1527,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } @Test @@ -1720,13 +1713,13 @@ private void completeCommitSyncApplicationEventSuccessfully() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { From 63b9546e919cbbce8626c30728d473d3792df2d3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 27 Feb 2024 13:29:45 -0800 Subject: [PATCH 054/130] Reverting, for the most part --- .../internals/AsyncKafkaConsumer.java | 60 ++++++------ .../internals/ConsumerNetworkThread.java | 21 +---- .../events/ApplicationEventHandler.java | 11 ++- .../events/ApplicationEventProcessor.java | 31 +++--- .../events/AsyncCommitApplicationEvent.java | 6 +- .../events/CommitApplicationEvent.java | 5 +- .../events/CompletableApplicationEvent.java | 17 ++-- .../events/CompletableBackgroundEvent.java | 1 - .../internals/events/EventProcessor.java | 2 + ...FetchCommittedOffsetsApplicationEvent.java | 16 +++- .../events/LeaveOnCloseApplicationEvent.java | 6 +- .../events/ListOffsetsApplicationEvent.java | 5 +- .../ResetPositionsApplicationEvent.java | 6 +- .../events/SyncCommitApplicationEvent.java | 17 +++- .../events/TopicMetadataApplicationEvent.java | 22 +++-- .../events/UnsubscribeApplicationEvent.java | 7 +- .../ValidatePositionsApplicationEvent.java | 6 +- .../internals/AsyncKafkaConsumerTest.java | 94 +++++++++++-------- .../internals/ConsumerNetworkThreadTest.java | 20 ++-- .../internals/ConsumerTestBuilder.java | 2 +- .../internals/FetchRequestManagerTest.java | 4 +- .../events/ApplicationEventProcessorTest.java | 18 ++-- 22 files changed, 196 insertions(+), 181 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 9caebe654e679..d4b461b0140d8 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 @@ -183,6 +183,7 @@ public BackgroundEventProcessor(final LogContext logContext, * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ + @Override public boolean process() { AtomicReference firstError = new AtomicReference<>(); @@ -767,8 +768,7 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - Timer timer = time.timer(Long.MAX_VALUE); - AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets, timer); + AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets); CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { @@ -936,13 +936,13 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event); + final Map committedOffsets = applicationEventHandler.addAndGet(event, + time.timer(timeout)); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -988,13 +988,12 @@ public List partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timer); + new TopicMetadataApplicationEvent(topic, timeout.toMillis()); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1018,12 +1017,11 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timer); + new TopicMetadataApplicationEvent(timeout.toMillis()); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); + return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); } finally { wakeupTrigger.clearTask(); } @@ -1091,18 +1089,16 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - Timer timer = time.timer(timeout); ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( timestampToSearch, - false, - timer); + false); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent); + listOffsetsEvent, + time.timer(timeout)); return offsetAndTimestampMap .entrySet() .stream() @@ -1276,7 +1271,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(), timer); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1354,10 +1349,10 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, requestTimer); + SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, timeout.toMillis()); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); - ConsumerUtils.getResult(commitFuture); + ConsumerUtils.getResult(commitFuture, requestTimer); interceptors.onCommit(offsets); } finally { wakeupTrigger.clearTask(); @@ -1468,13 +1463,13 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.isPresent()) { - Timer timer = time.timer(defaultApiTimeoutMs); - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(timer); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); applicationEventHandler.add(unsubscribeApplicationEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); + Timer timer = time.timer(Long.MAX_VALUE); try { - processBackgroundEvents(unsubscribeApplicationEvent.future(), timer); + processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1572,7 +1567,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer)); + applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(), timer); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1595,7 +1590,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer)); + applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(), timer); return true; } catch (TimeoutException e) { return false; @@ -1628,8 +1623,8 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsApplicationEvent event = new FetchCommittedOffsetsApplicationEvent( initializingPartitions, - timer); - final Map offsets = applicationEventHandler.addAndGet(event); + timer.remainingMs()); + final Map offsets = applicationEventHandler.addAndGet(event, timer); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { @@ -1814,17 +1809,20 @@ private void subscribeInternal(Collection topics, Optional T processBackgroundEvents(Future future, Timer timer) { + T processBackgroundEvents(EventProcessor eventProcessor, + Future future, + Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = backgroundEventProcessor.process(); + boolean hadEvents = eventProcessor.process(); try { if (future.isDone()) { 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 626fc278eeac5..aa352cd68a22e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,8 +20,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -130,7 +128,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - final List events = applicationEventProcessor.process(); + applicationEventProcessor.process(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -146,23 +144,6 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); - - events.stream() - .filter(e -> e instanceof CompletableApplicationEvent) - .map(e -> (CompletableApplicationEvent) e) - .filter(e -> e.deadlineMs() < currentTimeMs && !e.future().isDone()) - .forEach(e -> { - long diff = currentTimeMs - e.deadlineMs(); - - TimeoutException exception = new TimeoutException("Operation could not be completed"); - log.warn("Event {} had a deadline of {} but was not complete by {}, which is {} ms. past, completing with exception {}", - e, - e.deadlineMs(), - currentTimeMs, - diff, - exception.getMessage()); - e.future().completeExceptionally(exception); - }); } /** 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 b82b17ea50b10..7535edf5970b3 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 @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; @@ -98,14 +97,20 @@ public long maximumTimeToWait() { * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will * return the result value upon successful completion; otherwise throws an error. * + *

+ * + * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * * @param event A {@link CompletableApplicationEvent} created by the polling thread + * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event) { + public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); + Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return ConsumerUtils.getResult(event.future()); + return event.get(timer); } @Override 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 a790b98e27ceb..9e48b4de6daad 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 @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -62,15 +61,8 @@ public ApplicationEventProcessor(final LogContext logContext, * an event generates an error. In such cases, the processor will log an exception, but we do not want those * errors to be propagated to the caller. */ - public List process() { - List events = new ArrayList<>(); - - process((event, error) -> { - events.add(event); - error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); - }); - - return events; + public boolean process() { + return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override @@ -158,7 +150,7 @@ private void process(final AsyncCommitApplicationEvent event) { private void process(final SyncCommitApplicationEvent event) { CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = event.deadlineMs(); + long expirationTimeoutMs = getExpirationTimeForTimeout(event.retryTimeoutMs()); CompletableFuture commitResult = manager.commitSync(event.offsets(), expirationTimeoutMs); event.chain(commitResult); } @@ -170,7 +162,7 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = event.deadlineMs(); + long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); event.chain(manager.fetchOffsets(event.partitions(), expirationTimeMs)); } @@ -245,7 +237,7 @@ private void process(final ValidatePositionsApplicationEvent event) { private void process(final TopicMetadataApplicationEvent event) { final CompletableFuture>> future; - long expirationTimeMs = event.deadlineMs(); + long expirationTimeMs = getExpirationTimeForTimeout(event.getTimeoutMs()); if (event.isAllTopics()) { future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); } else { @@ -288,6 +280,19 @@ private void process(final LeaveOnCloseApplicationEvent event) { event.chain(future); } + /** + * @return Expiration time in milliseconds calculated with the current time plus the given + * timeout. Returns Long.MAX_VALUE if the expiration overflows it. + * Visible for testing. + */ + long getExpirationTimeForTimeout(final long timeoutMs) { + long expiration = System.currentTimeMillis() + timeoutMs; + if (expiration < 0) { + return Long.MAX_VALUE; + } + return expiration; + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index 8190d18c2496d..7a939ce3cfd16 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -18,8 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; - import java.util.Map; /** @@ -27,8 +25,8 @@ */ public class AsyncCommitApplicationEvent extends CommitApplicationEvent { - public AsyncCommitApplicationEvent(final Map offsets, Timer timer) { - super(offsets, Type.COMMIT_ASYNC, timer); + public AsyncCommitApplicationEvent(final Map offsets) { + super(offsets, Type.COMMIT_ASYNC); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index 4c0cf59d70d8a..69d969d7b0f46 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -30,8 +29,8 @@ public abstract class CommitApplicationEvent extends CompletableApplicationEvent */ private final Map offsets; - protected CommitApplicationEvent(final Map offsets, Type type, Timer timer) { - super(type, timer); + public CommitApplicationEvent(final Map offsets, Type type) { + super(type); this.offsets = Collections.unmodifiableMap(offsets); for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index a7f23237c35d8..365c620e0c0c0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,9 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; -import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -30,21 +30,18 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; - private final long deadlineMs; - protected CompletableApplicationEvent(Type type, Timer timer) { + protected CompletableApplicationEvent(Type type) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } - @Override public CompletableFuture future() { return future; } - public long deadlineMs() { - return deadlineMs; + public T get(Timer timer) { + return ConsumerUtils.getResult(future, timer); } public void chain(final CompletableFuture providedFuture) { @@ -65,19 +62,19 @@ public boolean equals(Object o) { CompletableApplicationEvent that = (CompletableApplicationEvent) o; - return future.equals(that.future) && deadlineMs == that.deadlineMs; + return future.equals(that.future); } @Override public int hashCode() { int result = super.hashCode(); - result = 31 * result + Objects.hash(future, deadlineMs); + result = 31 * result + future.hashCode(); return result; } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; + return super.toStringBase() + ", future=" + future; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 3c8f9670c154d..640ee6103af9b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -33,7 +33,6 @@ protected CompletableBackgroundEvent(Type type) { this.future = new CompletableFuture<>(); } - @Override public CompletableFuture future() { return future; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index d24d4eb260555..79a987e8a7aa9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -48,6 +48,8 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } + public abstract boolean process(); + protected abstract void process(T event); @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index 396ddc9338fb3..34b2d97705cd9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -31,16 +30,26 @@ public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicatio */ private final Set partitions; + /** + * Time until which the request will be retried if it fails with a retriable error. + */ + private final long timeoutMs; + public FetchCommittedOffsetsApplicationEvent(final Set partitions, - final Timer timer) { - super(Type.FETCH_COMMITTED_OFFSETS, timer); + final long timeoutMs) { + super(Type.FETCH_COMMITTED_OFFSETS); this.partitions = Collections.unmodifiableSet(partitions); + this.timeoutMs = timeoutMs; } public Set partitions() { return partitions; } + public long timeout() { + return timeoutMs; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -64,6 +73,7 @@ public String toString() { return getClass().getSimpleName() + "{" + toStringBase() + ", partitions=" + partitions + + ", timeout=" + timeoutMs + "ms" + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index ab91166e4c539..ee0b6ffa61c7d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent(Timer timer) { - super(Type.LEAVE_ON_CLOSE, timer); + public LeaveOnCloseApplicationEvent() { + super(Type.LEAVE_ON_CLOSE); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index 1baf966c2ecc9..2466d062726f8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; @@ -37,8 +36,8 @@ public class ListOffsetsApplicationEvent extends CompletableApplicationEvent timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps, Timer timer) { - super(Type.LIST_OFFSETS, timer); + public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps) { + super(Type.LIST_OFFSETS); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java index c296adbc68e95..5d9b07f9de05f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -26,7 +24,7 @@ */ public class ResetPositionsApplicationEvent extends CompletableApplicationEvent { - public ResetPositionsApplicationEvent(Timer timer) { - super(Type.RESET_POSITIONS, timer); + public ResetPositionsApplicationEvent() { + super(Type.RESET_POSITIONS); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 8e81a4102579f..43dfee6ab18b5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -18,8 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; - import java.util.Map; /** @@ -28,9 +26,19 @@ */ public class SyncCommitApplicationEvent extends CommitApplicationEvent { + /** + * Time to wait for a response, retrying on retriable errors. + */ + private final long retryTimeoutMs; + public SyncCommitApplicationEvent(final Map offsets, - final Timer timer) { - super(offsets, Type.COMMIT_SYNC, timer); + final long retryTimeoutMs) { + super(offsets, Type.COMMIT_SYNC); + this.retryTimeoutMs = retryTimeoutMs; + } + + public Long retryTimeoutMs() { + return retryTimeoutMs; } @Override @@ -38,6 +46,7 @@ public String toString() { return "SyncCommitApplicationEvent{" + toStringBase() + ", offsets=" + offsets() + + ", retryTimeout=" + retryTimeoutMs + "ms" + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index 1e72f5bb3c9bb..dd6f842cc2674 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; @@ -26,17 +25,20 @@ public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { private final String topic; private final boolean allTopics; + private final long timeoutMs; - public TopicMetadataApplicationEvent(final Timer timer) { - super(Type.TOPIC_METADATA, timer); + public TopicMetadataApplicationEvent(final long timeoutMs) { + super(Type.TOPIC_METADATA); this.topic = null; this.allTopics = true; + this.timeoutMs = timeoutMs; } - public TopicMetadataApplicationEvent(final String topic, final Timer timer) { - super(Type.TOPIC_METADATA, timer); + public TopicMetadataApplicationEvent(final String topic, final long timeoutMs) { + super(Type.TOPIC_METADATA); this.topic = topic; this.allTopics = false; + this.timeoutMs = timeoutMs; } public String topic() { @@ -47,11 +49,15 @@ public boolean isAllTopics() { return allTopics; } + public long getTimeoutMs() { + return timeoutMs; + } @Override public String toString() { return getClass().getSimpleName() + " {" + toStringBase() + ", topic=" + topic + - ", allTopics=" + allTopics + "}"; + ", allTopics=" + allTopics + + ", timeoutMs=" + timeoutMs + "}"; } @Override @@ -62,11 +68,11 @@ public boolean equals(Object o) { TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - return topic.equals(that.topic) && (allTopics == that.allTopics); + return topic.equals(that.topic) && (allTopics == that.allTopics) && (timeoutMs == that.timeoutMs); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics); + return Objects.hash(super.hashCode(), topic, allTopics, timeoutMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java index 85dc2d225fcce..a1ccb896fdf57 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -27,9 +25,8 @@ * leave group heartbeat, without waiting for any response or considering timeouts). */ public class UnsubscribeApplicationEvent extends CompletableApplicationEvent { - - public UnsubscribeApplicationEvent(final Timer timer) { - super(Type.UNSUBSCRIBE, timer); + public UnsubscribeApplicationEvent() { + super(Type.UNSUBSCRIBE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java index a74882a313ca9..3b093e0b68353 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -26,7 +24,7 @@ */ public class ValidatePositionsApplicationEvent extends CompletableApplicationEvent { - public ValidatePositionsApplicationEvent(Timer timer) { - super(Type.VALIDATE_POSITIONS, timer); + public ValidatePositionsApplicationEvent() { + super(Type.VALIDATE_POSITIONS); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 1675e2c47bbee..976677dec8628 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 @@ -325,7 +325,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -347,7 +347,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); } @Test @@ -355,7 +355,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class))).thenAnswer(invocation -> { + any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); @@ -596,9 +596,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any()); + doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); } @@ -641,7 +641,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class)); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -774,14 +774,15 @@ public void testBeginningOffsets() { Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -791,23 +792,26 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class)); + any(ListOffsetsApplicationEvent.class), + any()); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -842,11 +846,12 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -864,7 +869,8 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -873,12 +879,13 @@ public void testWakeupCommitted() { final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); + Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future()); + return ConsumerUtils.getResult(event.future(), timer); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1503,20 +1510,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } } @@ -1531,11 +1538,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test @@ -1579,7 +1586,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1606,14 +1613,16 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - consumer.processBackgroundEvents(future, timer); + try (EventProcessor processor = mock(EventProcessor.class)) { + consumer.processBackgroundEvents(processor, future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); + } } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1625,15 +1634,17 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - consumer.processBackgroundEvents(future, timer); + try (EventProcessor processor = mock(EventProcessor.class)) { + consumer.processBackgroundEvents(processor, future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); + } } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1649,10 +1660,12 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); + try (EventProcessor processor = mock(EventProcessor.class)) { + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); + } } private HashMap mockTopicPartitionOffset() { @@ -1717,13 +1730,13 @@ private void completeCommitSyncApplicationEventSuccessfully() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { @@ -1739,3 +1752,4 @@ private void forceCommitCallbackInvocation() { consumer.commitAsync(); } } + 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 70a74155779e4..a491df417de45 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 @@ -41,7 +41,6 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -154,7 +153,7 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>(), time.timer(100L)); + ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(AsyncCommitApplicationEvent.class)); @@ -162,7 +161,7 @@ public void testAsyncCommitEvent() { @Test public void testSyncCommitEvent() { - ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), time.timer(100L)); + ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), 100L); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(SyncCommitApplicationEvent.class)); @@ -171,7 +170,7 @@ public void testSyncCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, time.timer(1000)); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); @@ -180,7 +179,7 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(time.timer(1000)); + ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); @@ -191,7 +190,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(time.timer(1000)); + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -200,7 +199,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(time.timer(1000)); + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); @@ -225,7 +224,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", time.timer(Long.MAX_VALUE))); + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", Long.MAX_VALUE)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); } @@ -284,9 +283,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - Timer timer = time.timer(1000); - CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap(), timer)); - ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap(), timer); + CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap())); + ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 4ed001518cb7e..d6ae62950608c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -140,7 +140,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.fetchConfig = new FetchConfig(config); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.metrics = createMetrics(config, time); this.subscriptions = spy(createSubscriptionState(config, logContext)); 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 2024b693d473a..b01e11f452262 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 @@ -177,7 +177,7 @@ public class FetchRequestManagerTest { private int maxWaitMs = 0; private int fetchSize = 1000; private long retryBackoffMs = 100; - private int requestTimeoutMs = 30000; + private long requestTimeoutMs = 30000; private MockTime time = new MockTime(1); private SubscriptionState subscriptions; private ConsumerMetadata metadata; @@ -3592,7 +3592,7 @@ private void buildFetcher(MetricConfig metricConfig, metadata, time, retryBackoffMs, - requestTimeoutMs, + (int) requestTimeoutMs, Integer.MAX_VALUE); offsetFetcher = new OffsetFetcher(logContext, consumerNetworkClient, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 524e00daade85..8ea8cb7a729f5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -27,9 +27,6 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,6 +36,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -46,7 +44,6 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { - private Time time; private ApplicationEventProcessor processor; private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); private RequestManagers requestManagers; @@ -64,7 +61,6 @@ public class ApplicationEventProcessorTest { @BeforeEach @SuppressWarnings("unchecked") public void setup() { - time = new MockTime(); LogContext logContext = new LogContext(); offsetRequestManager = mock(OffsetsRequestManager.class); offsetsRequestManager = mock(OffsetsRequestManager.class); @@ -100,10 +96,18 @@ public void testPrepClosingCommitEvents() { verify(commitRequestManager).signalClose(); } + @Test + public void testExpirationCalculation() { + assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); + long timeout = processor.getExpirationTimeForTimeout(1000); + assertTrue(timeout > 0); + assertTrue(timeout < Long.MAX_VALUE); + } + @Test public void testPrepClosingLeaveGroupEvent() { - Timer timer = time.timer(Long.MAX_VALUE); - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(timer); + LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); From 29fdd33e501264cfc64cbb07080f05ef83de83f2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 27 Feb 2024 13:31:29 -0800 Subject: [PATCH 055/130] WIP --- .../internals/AsyncKafkaConsumer.java | 87 +++++--- .../internals/CommitRequestManager.java | 4 +- .../internals/ConsumerNetworkThread.java | 8 + .../internals/MembershipManagerImpl.java | 4 +- .../internals/events/ApplicationEvent.java | 50 +---- .../events/ApplicationEventHandler.java | 14 +- .../events/ApplicationEventProcessor.java | 23 +- .../events/ApplicationEventType.java | 24 ++ .../AssignmentChangeApplicationEvent.java | 32 +-- .../events/AsyncCommitApplicationEvent.java | 16 +- .../internals/events/BackgroundEvent.java | 44 +--- .../events/BackgroundEventHandler.java | 2 +- .../internals/events/BackgroundEventType.java | 21 ++ .../events/CommitApplicationEvent.java | 23 +- .../events/CommitOnCloseApplicationEvent.java | 11 +- .../events/CompletableApplicationEvent.java | 38 +--- .../events/CompletableBackgroundEvent.java | 33 +-- .../internals/events/CompletableEvent.java | 2 + .../internals/events/ConsumerEvent.java | 68 ++++++ ...balanceListenerCallbackCompletedEvent.java | 29 +-- ...rRebalanceListenerCallbackNeededEvent.java | 34 +-- .../events/ErrorBackgroundEvent.java | 29 +-- .../internals/events/EventProcessor.java | 79 ++++--- ...FetchCommittedOffsetsApplicationEvent.java | 43 +--- .../events/GroupMetadataUpdateEvent.java | 34 +-- .../events/LeaveOnCloseApplicationEvent.java | 14 +- .../events/ListOffsetsApplicationEvent.java | 36 +-- .../NewTopicsMetadataUpdateRequestEvent.java | 11 +- .../events/PollApplicationEvent.java | 29 +-- .../ResetPositionsApplicationEvent.java | 8 +- .../SubscriptionChangeApplicationEvent.java | 4 +- .../events/SyncCommitApplicationEvent.java | 26 +-- .../events/TopicMetadataApplicationEvent.java | 42 +--- .../events/UnsubscribeApplicationEvent.java | 9 +- .../ValidatePositionsApplicationEvent.java | 8 +- .../clients/consumer/KafkaConsumerTest.java | 1 + .../internals/AsyncKafkaConsumerTest.java | 210 +++++++----------- .../internals/ConsumerNetworkThreadTest.java | 20 +- .../internals/ConsumerTestBuilder.java | 2 +- .../internals/FetchRequestManagerTest.java | 4 +- .../HeartbeatRequestManagerTest.java | 33 ++- .../events/ApplicationEventProcessorTest.java | 18 +- 42 files changed, 491 insertions(+), 736 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerEvent.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 d4b461b0140d8..c5fa05790460b 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 @@ -183,7 +183,6 @@ public BackgroundEventProcessor(final LogContext logContext, * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ - @Override public boolean process() { AtomicReference firstError = new AtomicReference<>(); @@ -768,7 +767,8 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets); + Timer timer = time.timer(Long.MAX_VALUE); + AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets, timer); CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { @@ -936,13 +936,13 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - time.timer(timeout)); + final Map committedOffsets = applicationEventHandler.addAndGet(event); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -988,12 +988,13 @@ public List partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } + final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timeout.toMillis()); + new TopicMetadataApplicationEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + applicationEventHandler.addAndGet(topicMetadataApplicationEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1017,11 +1018,12 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } + final Timer timer = time.timer(timeout); final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timeout.toMillis()); + new TopicMetadataApplicationEvent(timer); wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(topicMetadataApplicationEvent); } finally { wakeupTrigger.clearTask(); } @@ -1089,16 +1091,18 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + Timer timer = time.timer(timeout); ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( timestampToSearch, - false); + false, + timer); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent, - time.timer(timeout)); + listOffsetsEvent); return offsetAndTimestampMap .entrySet() .stream() @@ -1266,22 +1271,21 @@ private void close(Duration timeout, boolean swallowException) { void prepareShutdown(final Timer timer, final AtomicReference firstException) { if (!groupMetadata.isPresent()) return; - maybeAutoCommitSync(autoCommitEnabled, timer, firstException); + + if (autoCommitEnabled) + maybeAutoCommitSync(timer); + applicationEventHandler.add(new CommitOnCloseApplicationEvent()); completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(timer)); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } // Visible for testing - void maybeAutoCommitSync(final boolean shouldAutoCommit, - final Timer timer, - final AtomicReference firstException) { - if (!shouldAutoCommit) - return; + void maybeAutoCommitSync(final Timer timer) { Map allConsumed = subscriptions.allConsumed(); log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed); try { @@ -1349,10 +1353,10 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, timeout.toMillis()); + SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, requestTimer); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); - ConsumerUtils.getResult(commitFuture, requestTimer); + ConsumerUtils.getResult(commitFuture); interceptors.onCommit(offsets); } finally { wakeupTrigger.clearTask(); @@ -1463,13 +1467,13 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); + Timer timer = time.timer(defaultApiTimeoutMs); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(timer); applicationEventHandler.add(unsubscribeApplicationEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); - Timer timer = time.timer(Long.MAX_VALUE); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); + processBackgroundEvents(unsubscribeApplicationEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1567,10 +1571,11 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(timer)); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); - if (cachedSubscriptionHasAllFetchPositions) return true; + if (cachedSubscriptionHasAllFetchPositions) + return true; // Reset positions using committed offsets retrieved from the group coordinator, for any // partitions which do not have a valid position and are not awaiting reset. This will @@ -1590,9 +1595,10 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(timer)); return true; } catch (TimeoutException e) { + log.debug("Timeout while updating fetch positions", e); return false; } } @@ -1623,8 +1629,8 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsApplicationEvent event = new FetchCommittedOffsetsApplicationEvent( initializingPartitions, - timer.remainingMs()); - final Map offsets = applicationEventHandler.addAndGet(event, timer); + timer); + final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { @@ -1648,7 +1654,12 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); - backgroundEventProcessor.process(); + + try { + backgroundEventProcessor.process(); + } finally { + backgroundEventProcessor.completeExpiredEvents(time.milliseconds()); + } // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as // in the previous implementation, because it will eventually involve group coordination @@ -1809,20 +1820,22 @@ private void subscribeInternal(Collection topics, Optional T processBackgroundEvents(EventProcessor eventProcessor, - Future future, - Timer timer) { + T processBackgroundEvents(Future future, Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = eventProcessor.process(); + boolean hadEvents; + try { + hadEvents = backgroundEventProcessor.process(); + } finally { + backgroundEventProcessor.completeExpiredEvents(time.milliseconds()); + } try { if (future.isDone()) { 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 9206783d561be..3625eb389841d 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 @@ -774,7 +774,7 @@ abstract class RetriableRequestState extends RequestState { * errors. If not present, the request is triggered without waiting for a response or * retrying. */ - private final Optional expirationTimeMs; + final Optional expirationTimeMs; /** * True if the request expiration time has been reached. This is set when validating the @@ -901,7 +901,7 @@ public OffsetFetchRequestState(final Set partitions, } public boolean sameRequest(final OffsetFetchRequestState request) { - return requestedPartitions.equals(request.requestedPartitions); + return requestedPartitions.equals(request.requestedPartitions) && expirationTimeMs.equals(request.expirationTimeMs); } public NetworkClientDelegate.UnsentRequest toUnsentRequest() { 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 aa352cd68a22e..f5e3d5aa69865 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -30,10 +32,14 @@ import java.io.Closeable; import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; @@ -144,6 +150,8 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); + + applicationEventProcessor.completeExpiredEvents(currentTimeMs); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index a9b0f3b94d831..876a4b75d3da3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -1288,7 +1289,8 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + Timer timer = time.timer(1000); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions, timer); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index ac7ccc56c55f2..78dd25f439a16 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -16,53 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import java.util.Objects; - /** - * This is the abstract definition of the events created by the KafkaConsumer API + * This is the abstract definition of the events created by the KafkaConsumer API on the user's + * application thread. */ -public abstract class ApplicationEvent { - - public enum Type { - COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, - LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, - UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, - COMMIT_ON_CLOSE, LEAVE_ON_CLOSE - } - - private final Type type; - - protected ApplicationEvent(Type type) { - this.type = Objects.requireNonNull(type); - } - - public Type type() { - return type; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ApplicationEvent that = (ApplicationEvent) o; - - return type == that.type; - } - - @Override - public int hashCode() { - return type.hashCode(); - } - - protected String toStringBase() { - return "type=" + type; - } +public abstract class ApplicationEvent extends ConsumerEvent { - @Override - public String toString() { - return "ApplicationEvent{" + - toStringBase() + - '}'; + protected ApplicationEvent(ApplicationEventType type) { + super(type); } } \ No newline at end of file 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 7535edf5970b3..b97e9d931cdad 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 @@ -17,12 +17,12 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -30,8 +30,6 @@ import java.time.Duration; import java.util.Objects; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; /** @@ -97,20 +95,14 @@ public long maximumTimeToWait() { * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will * return the result value upon successful completion; otherwise throws an error. * - *

- * - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. - * * @param event A {@link CompletableApplicationEvent} created by the polling thread - * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { + public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); - Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return event.get(timer); + return ConsumerUtils.getResult(event.future()); } @Override 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 9e48b4de6daad..9d7b31d396189 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -61,8 +61,8 @@ public ApplicationEventProcessor(final LogContext logContext, * an event generates an error. In such cases, the processor will log an exception, but we do not want those * errors to be propagated to the caller. */ - public boolean process() { - return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); + public void process() { + process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override @@ -150,7 +150,7 @@ private void process(final AsyncCommitApplicationEvent event) { private void process(final SyncCommitApplicationEvent event) { CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = getExpirationTimeForTimeout(event.retryTimeoutMs()); + long expirationTimeoutMs = event.deadlineMs(); CompletableFuture commitResult = manager.commitSync(event.offsets(), expirationTimeoutMs); event.chain(commitResult); } @@ -162,7 +162,7 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); + long expirationTimeMs = event.deadlineMs(); event.chain(manager.fetchOffsets(event.partitions(), expirationTimeMs)); } @@ -237,7 +237,7 @@ private void process(final ValidatePositionsApplicationEvent event) { private void process(final TopicMetadataApplicationEvent event) { final CompletableFuture>> future; - long expirationTimeMs = getExpirationTimeForTimeout(event.getTimeoutMs()); + long expirationTimeMs = event.deadlineMs(); if (event.isAllTopics()) { future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); } else { @@ -280,19 +280,6 @@ private void process(final LeaveOnCloseApplicationEvent event) { event.chain(future); } - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; - } - /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java new file mode 100644 index 0000000000000..759eabeb1e2fe --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +public enum ApplicationEventType { + COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, + UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, + COMMIT_ON_CLOSE, LEAVE_ON_CLOSE +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java index ccf7199f2606b..43058303e27db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java @@ -22,6 +22,8 @@ import java.util.Collections; import java.util.Map; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.ASSIGNMENT_CHANGE; + public class AssignmentChangeApplicationEvent extends ApplicationEvent { private final Map offsets; @@ -29,7 +31,7 @@ public class AssignmentChangeApplicationEvent extends ApplicationEvent { public AssignmentChangeApplicationEvent(final Map offsets, final long currentTimeMs) { - super(Type.ASSIGNMENT_CHANGE); + super(ASSIGNMENT_CHANGE); this.offsets = Collections.unmodifiableMap(offsets); this.currentTimeMs = currentTimeMs; } @@ -43,31 +45,7 @@ public long currentTimeMs() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - AssignmentChangeApplicationEvent that = (AssignmentChangeApplicationEvent) o; - - if (currentTimeMs != that.currentTimeMs) return false; - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - result = 31 * result + (int) (currentTimeMs ^ (currentTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "AssignmentChangeApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - ", currentTimeMs=" + currentTimeMs + - '}'; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets + ", currentTimeMs=" + currentTimeMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index 7a939ce3cfd16..c2145e756fb5e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -18,22 +18,18 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + import java.util.Map; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_ASYNC; + /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ public class AsyncCommitApplicationEvent extends CommitApplicationEvent { - public AsyncCommitApplicationEvent(final Map offsets) { - super(offsets, Type.COMMIT_ASYNC); - } - - @Override - public String toString() { - return "AsyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - '}'; + public AsyncCommitApplicationEvent(final Map offsets, Timer timer) { + super(COMMIT_ASYNC, timer, offsets); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index e5d522201ef0a..65823c6fb6e65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -18,50 +18,12 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import java.util.Objects; - /** * This is the abstract definition of the events created by the {@link ConsumerNetworkThread network thread}. */ -public abstract class BackgroundEvent { - - public enum Type { - ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE - } - - private final Type type; - - public BackgroundEvent(Type type) { - this.type = Objects.requireNonNull(type); - } - - public Type type() { - return type; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - BackgroundEvent that = (BackgroundEvent) o; - - return type == that.type; - } - - @Override - public int hashCode() { - return type.hashCode(); - } - - protected String toStringBase() { - return "type=" + type; - } +public abstract class BackgroundEvent extends ConsumerEvent { - @Override - public String toString() { - return "BackgroundEvent{" + - toStringBase() + - '}'; + protected BackgroundEvent(BackgroundEventType type) { + super(type); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 103493d25314f..48421484f1d3b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -26,7 +26,7 @@ /** * An event handler that receives {@link BackgroundEvent background events} from the * {@link ConsumerNetworkThread network thread} which are then made available to the application thread - * via the {@link BackgroundEventProcessor}. + * via an {@link EventProcessor}. */ public class BackgroundEventHandler { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java new file mode 100644 index 0000000000000..6ea26eea0efb7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +public enum BackgroundEventType { + ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index 69d969d7b0f46..dd1225f39856c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -29,8 +30,10 @@ public abstract class CommitApplicationEvent extends CompletableApplicationEvent */ private final Map offsets; - public CommitApplicationEvent(final Map offsets, Type type) { - super(type); + protected CommitApplicationEvent(final ApplicationEventType type, + final Timer timer, + final Map offsets) { + super(type, timer); this.offsets = Collections.unmodifiableMap(offsets); for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { @@ -44,21 +47,9 @@ public Map offsets() { return offsets; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CommitApplicationEvent that = (CommitApplicationEvent) o; - - return offsets.equals(that.offsets); - } @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java index 4cc07e945f9d2..135181db56f88 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java @@ -16,16 +16,11 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_ON_CLOSE; + public class CommitOnCloseApplicationEvent extends ApplicationEvent { public CommitOnCloseApplicationEvent() { - super(Type.COMMIT_ON_CLOSE); - } - - @Override - public String toString() { - return "CommitOnCloseApplicationEvent{" + - toStringBase() + - '}'; + super(COMMIT_ON_CLOSE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 365c620e0c0c0..f13b0acf632e8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; import java.util.concurrent.CompletableFuture; @@ -30,18 +29,22 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableApplicationEvent(Type type) { + protected CompletableApplicationEvent(ApplicationEventType type, Timer timer) { super(type); this.future = new CompletableFuture<>(); + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } + @Override public CompletableFuture future() { return future; } - public T get(Timer timer) { - return ConsumerUtils.getResult(future, timer); + @Override + public long deadlineMs() { + return deadlineMs; } public void chain(final CompletableFuture providedFuture) { @@ -54,33 +57,8 @@ public void chain(final CompletableFuture providedFuture) { }); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableApplicationEvent that = (CompletableApplicationEvent) o; - - return future.equals(that.future); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; - } - @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 640ee6103af9b..64f08656c505e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + import java.util.concurrent.CompletableFuture; /** @@ -27,43 +29,26 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableBackgroundEvent(Type type) { + protected CompletableBackgroundEvent(BackgroundEventType type, Timer timer) { super(type); this.future = new CompletableFuture<>(); + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } + @Override public CompletableFuture future() { return future; } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableBackgroundEvent that = (CompletableBackgroundEvent) o; - - return future.equals(that.future); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; + public long deadlineMs() { + return deadlineMs; } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 8fdcc20fa8363..2707230062c11 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -22,4 +22,6 @@ public interface CompletableEvent { CompletableFuture future(); + long deadlineMs(); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerEvent.java new file mode 100644 index 0000000000000..0bbd39f6f5b0b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerEvent.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.Uuid; + +import java.util.Objects; + +/** + * Application event with a result in the form of a future, that can be retrieved within a + * timeout based on completion. + * + * @param + */ +public abstract class ConsumerEvent> { + + private final TYPE type; + + private final Uuid id; + + protected ConsumerEvent(TYPE type) { + this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); + } + + public TYPE type() { + return type; + } + + public Uuid id() { + return id; + } + + @Override + public final boolean equals(Object o) { + return this == o; + } + + @Override + public final int hashCode() { + return Objects.hash(type, id); + } + + protected String toStringBase() { + return "type=" + type + ", id=" + id; + } + + @Override + public final String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index b260c6154ea5f..8f6c335c98242 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -24,6 +24,8 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED; + /** * Event that signifies that the application thread has executed the {@link ConsumerRebalanceListener} callback. If * the callback execution threw an error, it is included in the event should any event listener want to know. @@ -37,7 +39,7 @@ public class ConsumerRebalanceListenerCallbackCompletedEvent extends Application public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, CompletableFuture future, Optional error) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); + super(CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); this.methodName = Objects.requireNonNull(methodName); this.future = Objects.requireNonNull(future); this.error = Objects.requireNonNull(error); @@ -55,24 +57,6 @@ public Optional error() { return error; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - - return methodName == that.methodName && - future.equals(that.future) && - error.equals(that.error); - } - - @Override - public int hashCode() { - return Objects.hash(methodName, future, error); - } - @Override protected String toStringBase() { return super.toStringBase() + @@ -80,11 +64,4 @@ protected String toStringBase() { ", future=" + future + ", error=" + error; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 7b17c034abdbd..991f7c788b805 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -20,12 +20,15 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.time.Duration; import java.util.Collections; import java.util.Objects; import java.util.SortedSet; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED; + /** * Event that signifies that the network I/O thread wants to invoke one of the callback methods on the * {@link ConsumerRebalanceListener}. This event will be processed by the application thread when the next @@ -38,8 +41,9 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); + SortedSet partitions, + Timer timer) { + super(CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } @@ -52,36 +56,10 @@ public SortedSet partitions() { return partitions; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackNeededEvent that = (ConsumerRebalanceListenerCallbackNeededEvent) o; - - return methodName == that.methodName && partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + methodName.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", methodName=" + methodName + ", partitions=" + partitions; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 2945f22986b18..b15a434f87e60 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -18,12 +18,14 @@ import org.apache.kafka.common.KafkaException; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.ERROR; + public class ErrorBackgroundEvent extends BackgroundEvent { private final RuntimeException error; public ErrorBackgroundEvent(Throwable t) { - super(Type.ERROR); + super(ERROR); this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); } @@ -32,28 +34,7 @@ public RuntimeException error() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ErrorBackgroundEvent that = (ErrorBackgroundEvent) o; - - return error.equals(that.error); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + error.hashCode(); - return result; - } - - @Override - public String toString() { - return "ErrorBackgroundEvent{" + - toStringBase() + - ", error=" + error + - '}'; + public String toStringBase() { + return super.toStringBase() + ", error=" + error; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 79a987e8a7aa9..2efb49de46004 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -28,6 +29,8 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; /** * An {@link EventProcessor} is the means by which events produced by thread A are @@ -40,16 +43,16 @@ public abstract class EventProcessor implements Closeable { private final Logger log; private final BlockingQueue eventQueue; + private final List> completableEvents; private final IdempotentCloser closer; protected EventProcessor(final LogContext logContext, final BlockingQueue eventQueue) { this.log = logContext.logger(EventProcessor.class); this.eventQueue = eventQueue; this.closer = new IdempotentCloser(); + this.completableEvents = new LinkedList<>(); } - public abstract boolean process(); - protected abstract void process(T event); @Override @@ -69,7 +72,8 @@ protected interface ProcessHandler { protected boolean process(ProcessHandler processHandler) { closer.assertOpen("The processor was previously closed, so no further processing can occur"); - List events = drain(); + LinkedList events = new LinkedList<>(); + eventQueue.drainTo(events); if (events.isEmpty()) { log.trace("No events to process"); @@ -83,6 +87,10 @@ protected boolean process(ProcessHandler processHandler) { try { Objects.requireNonNull(event, "Attempted to process a null event"); log.trace("Processing event: {}", event); + + if (event instanceof CompletableEvent) + completableEvents.add((CompletableEvent) event); + process(event); processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { @@ -97,39 +105,56 @@ protected boolean process(ProcessHandler processHandler) { return true; } + public void completeExpiredEvents(long currentTimeMs) { + log.trace("Removing expired events"); + + Consumer> completeEvent = e -> { + log.debug("Completing event {} exceptionally since it expired {} ms ago", e, currentTimeMs - e.deadlineMs()); + CompletableFuture f = e.future(); + f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); + }; + + // Complete (exceptionally) any events that have recently passed their deadline. + completableEvents + .stream() + .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) + .forEach(completeEvent); + + // Remove any events that are complete. + completableEvents.removeIf(e -> e.future().isDone()); + log.trace("Finished removal of expired events"); + } + /** * It is possible for the consumer to close before complete processing all the events in the queue. In * this case, we need to throw an exception to notify the user the consumer is closed. */ private void closeInternal() { - log.trace("Closing event processor"); - List incompleteEvents = drain(); - - if (incompleteEvents.isEmpty()) - return; + log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); - KafkaException exception = new KafkaException("The consumer is closed"); + Consumer> completeEvent = e -> { + log.debug("Completing event {} exceptionally since the consumer is closing", e); + CompletableFuture f = e.future(); + f.completeExceptionally(new KafkaException("The consumer is closed")); + }; - // Check each of the events and if it has a Future that is incomplete, complete it exceptionally. - incompleteEvents + // Check each of the unprocessed events and if it has a Future that is incomplete, complete it exceptionally. + eventQueue .stream() .filter(e -> e instanceof CompletableEvent) - .map(e -> ((CompletableEvent) e).future()) - .filter(f -> !f.isDone()) - .forEach(f -> { - log.debug("Completing {} with exception {}", f, exception.getMessage()); - f.completeExceptionally(exception); - }); - - log.debug("Discarding {} events because the consumer is closing", incompleteEvents.size()); - } + .map(e -> (CompletableEvent) e) + .filter(e -> !e.future().isDone()) + .forEach(completeEvent); + eventQueue.clear(); + + // Check each of the completable events and if it has a Future that is incomplete, complete it exceptionally. + // In the case of shutdown, we don't take the deadline into consideration. + completableEvents + .stream() + .filter(e -> !e.future().isDone()) + .forEach(completeEvent); + completableEvents.clear(); - /** - * Moves all the events from the queue to the returned list. - */ - private List drain() { - LinkedList events = new LinkedList<>(); - eventQueue.drainTo(events); - return events; + log.debug("Finished removal of events that were unprocessed and/or unfinished"); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index 34b2d97705cd9..d81864aea174d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -18,11 +18,14 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; import java.util.Set; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.FETCH_COMMITTED_OFFSETS; + public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicationEvent> { /** @@ -30,50 +33,18 @@ public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicatio */ private final Set partitions; - /** - * Time until which the request will be retried if it fails with a retriable error. - */ - private final long timeoutMs; - public FetchCommittedOffsetsApplicationEvent(final Set partitions, - final long timeoutMs) { - super(Type.FETCH_COMMITTED_OFFSETS); + final Timer timer) { + super(FETCH_COMMITTED_OFFSETS, timer); this.partitions = Collections.unmodifiableSet(partitions); - this.timeoutMs = timeoutMs; } public Set partitions() { return partitions; } - public long timeout() { - return timeoutMs; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - FetchCommittedOffsetsApplicationEvent that = (FetchCommittedOffsetsApplicationEvent) o; - - return partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - ", partitions=" + partitions + - ", timeout=" + timeoutMs + "ms" + - '}'; + public String toStringBase() { + return super.toStringBase() + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 120e671724209..1c6a432b88321 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import java.util.Objects; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.GROUP_METADATA_UPDATE; /** * This event is sent by the {@link ConsumerNetworkThread consumer's network thread} to the application thread @@ -29,12 +29,11 @@ */ public class GroupMetadataUpdateEvent extends BackgroundEvent { - final private int memberEpoch; - final private String memberId; + private final int memberEpoch; + private final String memberId; - public GroupMetadataUpdateEvent(final int memberEpoch, - final String memberId) { - super(Type.GROUP_METADATA_UPDATE); + public GroupMetadataUpdateEvent(final int memberEpoch, final String memberId) { + super(GROUP_METADATA_UPDATE); this.memberEpoch = memberEpoch; this.memberId = memberId; } @@ -47,33 +46,10 @@ public String memberId() { return memberId; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - GroupMetadataUpdateEvent that = (GroupMetadataUpdateEvent) o; - return memberEpoch == that.memberEpoch && - Objects.equals(memberId, that.memberId); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), memberEpoch, memberId); - } - @Override public String toStringBase() { return super.toStringBase() + ", memberEpoch=" + memberEpoch + ", memberId='" + memberId + '\''; } - - @Override - public String toString() { - return "GroupMetadataUpdateEvent{" + - toStringBase() + - '}'; - } - } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index ee0b6ffa61c7d..792a18d649428 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -16,15 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.LEAVE_ON_CLOSE; + public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent() { - super(Type.LEAVE_ON_CLOSE); - } - @Override - public String toString() { - return "LeaveOnCloseApplicationEvent{" + - toStringBase() + - '}'; + public LeaveOnCloseApplicationEvent(Timer timer) { + super(LEAVE_ON_CLOSE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index 2466d062726f8..c74adc1cc2c00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -18,11 +18,14 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.LIST_OFFSETS; + /** * Event for retrieving partition offsets by performing a * {@link org.apache.kafka.common.requests.ListOffsetsRequest ListOffsetsRequest}. @@ -36,8 +39,8 @@ public class ListOffsetsApplicationEvent extends CompletableApplicationEvent timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps) { - super(Type.LIST_OFFSETS); + public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps, Timer timer) { + super(LIST_OFFSETS, timer); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } @@ -64,31 +67,10 @@ public boolean requireTimestamps() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ListOffsetsApplicationEvent that = (ListOffsetsApplicationEvent) o; - - if (requireTimestamps != that.requireTimestamps) return false; - return timestampsToSearch.equals(that.timestampsToSearch); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + timestampsToSearch.hashCode(); - result = 31 * result + (requireTimestamps ? 1 : 0); - return result; - } - - @Override - public String toString() { - return getClass().getSimpleName() + " {" + - toStringBase() + - ", timestampsToSearch=" + timestampsToSearch + ", " + - "requireTimestamps=" + requireTimestamps + '}'; + public String toStringBase() { + return super.toStringBase() + + ", timestampsToSearch=" + timestampsToSearch + + ", requireTimestamps=" + requireTimestamps; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java index c06a3a717dceb..8cc9d1c78de52 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java @@ -16,16 +16,11 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.NEW_TOPICS_METADATA_UPDATE; + public class NewTopicsMetadataUpdateRequestEvent extends ApplicationEvent { public NewTopicsMetadataUpdateRequestEvent() { - super(Type.NEW_TOPICS_METADATA_UPDATE); - } - - @Override - public String toString() { - return "NewTopicsMetadataUpdateRequestEvent{" + - toStringBase() + - '}'; + super(NEW_TOPICS_METADATA_UPDATE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java index b958f0ec41703..6b0460bff90f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.POLL; + public class PollApplicationEvent extends ApplicationEvent { private final long pollTimeMs; public PollApplicationEvent(final long pollTimeMs) { - super(Type.POLL); + super(POLL); this.pollTimeMs = pollTimeMs; } @@ -30,28 +32,7 @@ public long pollTimeMs() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - PollApplicationEvent that = (PollApplicationEvent) o; - - return pollTimeMs == that.pollTimeMs; - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + (int) (pollTimeMs ^ (pollTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "PollApplicationEvent{" + - toStringBase() + - ", pollTimeMs=" + pollTimeMs + - '}'; + public String toStringBase() { + return super.toStringBase() + ", pollTimeMs=" + pollTimeMs; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java index 5d9b07f9de05f..f7ffbe43e57ab 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java @@ -17,6 +17,10 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.RESET_POSITIONS; + /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -24,7 +28,7 @@ */ public class ResetPositionsApplicationEvent extends CompletableApplicationEvent { - public ResetPositionsApplicationEvent() { - super(Type.RESET_POSITIONS); + public ResetPositionsApplicationEvent(Timer timer) { + super(RESET_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java index 73fd15fb14408..a458782751e86 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.SUBSCRIPTION_CHANGE; + /** * Application event indicating that the subscription state has changed, triggered when a user * calls the subscribe API. This will make the consumer join a consumer group if not part of it @@ -25,6 +27,6 @@ public class SubscriptionChangeApplicationEvent extends ApplicationEvent { public SubscriptionChangeApplicationEvent() { - super(Type.SUBSCRIPTION_CHANGE); + super(SUBSCRIPTION_CHANGE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 43dfee6ab18b5..6472cc93225fc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -18,35 +18,25 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + import java.util.Map; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_SYNC; + /** * Event to commit offsets waiting for a response and retrying on expected retriable errors until * the timer expires. */ public class SyncCommitApplicationEvent extends CommitApplicationEvent { - /** - * Time to wait for a response, retrying on retriable errors. - */ - private final long retryTimeoutMs; - public SyncCommitApplicationEvent(final Map offsets, - final long retryTimeoutMs) { - super(offsets, Type.COMMIT_SYNC); - this.retryTimeoutMs = retryTimeoutMs; - } - - public Long retryTimeoutMs() { - return retryTimeoutMs; + final Timer timer) { + super(COMMIT_SYNC, timer, offsets); } @Override - public String toString() { - return "SyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - ", retryTimeout=" + retryTimeoutMs + "ms" + - '}'; + public String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index dd6f842cc2674..d01b0f0e0861a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -17,28 +17,28 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; -import java.util.Objects; + +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.TOPIC_METADATA; public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { + private final String topic; private final boolean allTopics; - private final long timeoutMs; - public TopicMetadataApplicationEvent(final long timeoutMs) { - super(Type.TOPIC_METADATA); + public TopicMetadataApplicationEvent(final Timer timer) { + super(TOPIC_METADATA, timer); this.topic = null; this.allTopics = true; - this.timeoutMs = timeoutMs; } - public TopicMetadataApplicationEvent(final String topic, final long timeoutMs) { - super(Type.TOPIC_METADATA); + public TopicMetadataApplicationEvent(final String topic, final Timer timer) { + super(TOPIC_METADATA, timer); this.topic = topic; this.allTopics = false; - this.timeoutMs = timeoutMs; } public String topic() { @@ -49,30 +49,8 @@ public boolean isAllTopics() { return allTopics; } - public long getTimeoutMs() { - return timeoutMs; - } - @Override - public String toString() { - return getClass().getSimpleName() + " {" + toStringBase() + - ", topic=" + topic + - ", allTopics=" + allTopics + - ", timeoutMs=" + timeoutMs + "}"; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof TopicMetadataApplicationEvent)) return false; - if (!super.equals(o)) return false; - - TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - - return topic.equals(that.topic) && (allTopics == that.allTopics) && (timeoutMs == that.timeoutMs); - } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics, timeoutMs); + public String toStringBase() { + return super.toStringBase() + ", topic=" + topic + ", allTopics=" + allTopics; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java index a1ccb896fdf57..c17cc6164a171 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java @@ -17,6 +17,10 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.UNSUBSCRIBE; + /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -25,8 +29,9 @@ * leave group heartbeat, without waiting for any response or considering timeouts). */ public class UnsubscribeApplicationEvent extends CompletableApplicationEvent { - public UnsubscribeApplicationEvent() { - super(Type.UNSUBSCRIBE); + + public UnsubscribeApplicationEvent(final Timer timer) { + super(UNSUBSCRIBE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java index 3b093e0b68353..6b39b83a41900 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java @@ -17,6 +17,10 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.VALIDATE_POSITIONS; + /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -24,7 +28,7 @@ */ public class ValidatePositionsApplicationEvent extends CompletableApplicationEvent { - public ValidatePositionsApplicationEvent() { - super(Type.VALIDATE_POSITIONS); + public ValidatePositionsApplicationEvent(Timer timer) { + super(VALIDATE_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 7dec7305d4db4..848851bb316a6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -986,6 +986,7 @@ public void testResetUsingAutoResetPolicy(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) public void testOffsetIsValidAfterSeek(GroupProtocol groupProtocol) { + Time time = new MockTime(1); SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST); ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); 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 976677dec8628..5ccd0068355cb 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 @@ -19,7 +19,6 @@ import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -56,6 +55,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.RetriableException; @@ -120,6 +120,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -138,12 +139,7 @@ @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { - private long retryBackoffMs = 100L; - private int defaultApiTimeoutMs = 1000; - private boolean autoCommitEnabled = true; - private AsyncKafkaConsumer consumer = null; - private final Time time = new MockTime(1); private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); @@ -196,32 +192,25 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { ); } - private AsyncKafkaConsumer newConsumer( - FetchBuffer fetchBuffer, - ConsumerInterceptors interceptors, - ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, - SubscriptionState subscriptions, - List assignors, - String groupId, - String clientId) { - return new AsyncKafkaConsumer<>( + private AsyncKafkaConsumer newConsumer(SubscriptionState subscriptions, boolean autoCommitEnabled) { + return new AsyncKafkaConsumer( new LogContext(), - clientId, + "client-id", new Deserializers<>(new StringDeserializer(), new StringDeserializer()), - fetchBuffer, + mock(FetchBuffer.class), fetchCollector, - interceptors, + mock(ConsumerInterceptors.class), time, applicationEventHandler, backgroundEventQueue, - rebalanceListenerInvoker, + mock(ConsumerRebalanceListenerInvoker.class), new Metrics(), subscriptions, metadata, - retryBackoffMs, - defaultApiTimeoutMs, - assignors, - groupId, + 100L, + 1000, + singletonList(new RoundRobinAssignor()), + "group-id", autoCommitEnabled); } @@ -305,6 +294,7 @@ private static Stream commitExceptionSupplier() { @Test public void testCommitAsyncWithFencedException() { consumer = newConsumer(); + completeCommitSyncApplicationEventSuccessfully(); final HashMap offsets = mockTopicPartitionOffset(); MockCommitCallback callback = new MockCommitCallback(); @@ -316,6 +306,17 @@ public void testCommitAsyncWithFencedException() { commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); + + // Close the consumer here as we know it will cause a FencedInstanceIdException to be thrown. + // If we get an error other than the FencedInstanceIdException, we'll raise a ruckus. + try { + consumer.close(); + } catch (KafkaException e) { + assertNotNull(e.getCause()); + assertInstanceOf(FencedInstanceIdException.class, e.getCause()); + } finally { + consumer = null; + } } @Test @@ -325,7 +326,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -347,7 +348,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); } @Test @@ -355,7 +356,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsApplicationEvent.class))).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); @@ -367,12 +368,14 @@ public void testCommittedExceptionThrown() { @Test public void testWakeupBeforeCallingPoll() { consumer = newConsumer(); + completeCommitSyncApplicationEventSuccessfully(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -394,6 +397,7 @@ public void testWakeupAfterEmptyFetch() { }).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -417,6 +421,7 @@ public void testWakeupAfterNonEmptyFetch() { }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -435,7 +440,7 @@ public void testCommitInRebalanceCallback() { doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions, time.timer(1000)); backgroundEventQueue.add(e); completeCommitSyncApplicationEventSuccessfully(); final AtomicBoolean callbackExecuted = new AtomicBoolean(false); @@ -472,6 +477,7 @@ public void testClearWakeupTriggerAfterPoll() { .when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -536,14 +542,8 @@ public void testCommitSyncLeaderEpochUpdate() { @Test public void testCommitAsyncLeaderEpochUpdate() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, true); + completeCommitSyncApplicationEventSuccessfully(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap topicPartitionOffsets = new HashMap<>(); @@ -596,9 +596,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(null).when(applicationEventHandler).addAndGet(any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); } @@ -606,14 +606,7 @@ public void testVerifyApplicationEventOnShutdown() { public void testPartitionRevocationOnClose() { MockRebalanceListener listener = new MockRebalanceListener(); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - mock(ConsumerInterceptors.class), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, true); consumer.subscribe(singleton("topic"), listener); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); @@ -628,20 +621,13 @@ public void testFailedPartitionRevocationOnClose() { // closing the consumer. ConsumerRebalanceListener listener = mock(ConsumerRebalanceListener.class); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, true); subscriptions.subscribe(singleton("topic"), Optional.of(listener)); TopicPartition tp = new TopicPartition("topic", 0); subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class)); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -664,37 +650,23 @@ public void testCompleteQuietly() { @Test public void testAutoCommitSyncEnabled() { + completeCommitSyncApplicationEventSuccessfully(); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - mock(ConsumerInterceptors.class), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, true); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(true, time.timer(100), null); + consumer.maybeAutoCommitSync(time.timer(100)); verify(applicationEventHandler).add(any(SyncCommitApplicationEvent.class)); } @Test public void testAutoCommitSyncDisabled() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - mock(ConsumerInterceptors.class), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, false); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(false, time.timer(100), null); verify(applicationEventHandler, never()).add(any(SyncCommitApplicationEvent.class)); } @@ -774,15 +746,14 @@ public void testBeginningOffsets() { Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -792,26 +763,23 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class), - any()); + any(ListOffsetsApplicationEvent.class)); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -846,12 +814,11 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -869,8 +836,7 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class)); } @Test @@ -879,13 +845,12 @@ public void testWakeupCommitted() { final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future(), timer); + return ConsumerUtils.getResult(event.future()); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -976,6 +941,7 @@ public void testNoInterceptorCommitAsyncFailed() { @Test public void testRefreshCommittedOffsetsSuccess() { consumer = newConsumer(); + completeCommitSyncApplicationEventSuccessfully(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L)); @@ -1305,7 +1271,7 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions, time.timer(1000)); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. @@ -1454,14 +1420,7 @@ public void testGroupIdOnlyWhitespaces() { @Test public void testEnsurePollEventSentOnConsumerPoll() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( - mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id"); + consumer = newConsumer(subscriptions, true); final TopicPartition tp = new TopicPartition("topic", 0); final List> records = singletonList( new ConsumerRecord<>("topic", 0, 2, "key1", "value1")); @@ -1510,20 +1469,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } } @@ -1538,11 +1497,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } @Test @@ -1586,7 +1545,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1613,16 +1572,14 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); - } + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1634,17 +1591,15 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); - } + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1660,12 +1615,10 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); - } + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); } private HashMap mockTopicPartitionOffset() { @@ -1730,13 +1683,13 @@ private void completeCommitSyncApplicationEventSuccessfully() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { @@ -1752,4 +1705,3 @@ private void forceCommitCallbackInvocation() { consumer.commitAsync(); } } - 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 a491df417de45..70a74155779e4 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 @@ -41,6 +41,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -153,7 +154,7 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>()); + ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>(), time.timer(100L)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(AsyncCommitApplicationEvent.class)); @@ -161,7 +162,7 @@ public void testAsyncCommitEvent() { @Test public void testSyncCommitEvent() { - ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), 100L); + ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), time.timer(100L)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(SyncCommitApplicationEvent.class)); @@ -170,7 +171,7 @@ public void testSyncCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true, time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); @@ -179,7 +180,7 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); + ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); @@ -190,7 +191,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -199,7 +200,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(time.timer(1000)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); @@ -224,7 +225,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", Long.MAX_VALUE)); + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", time.timer(Long.MAX_VALUE))); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); } @@ -283,8 +284,9 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap())); - ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap()); + Timer timer = time.timer(1000); + CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap(), timer)); + ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index d6ae62950608c..4ed001518cb7e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -140,7 +140,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.fetchConfig = new FetchConfig(config); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.metrics = createMetrics(config, time); this.subscriptions = spy(createSubscriptionState(config, logContext)); 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 b01e11f452262..2024b693d473a 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 @@ -177,7 +177,7 @@ public class FetchRequestManagerTest { private int maxWaitMs = 0; private int fetchSize = 1000; private long retryBackoffMs = 100; - private long requestTimeoutMs = 30000; + private int requestTimeoutMs = 30000; private MockTime time = new MockTime(1); private SubscriptionState subscriptions; private ConsumerMetadata metadata; @@ -3592,7 +3592,7 @@ private void buildFetcher(MetricConfig metricConfig, metadata, time, retryBackoffMs, - (int) requestTimeoutMs, + requestTimeoutMs, Integer.MAX_VALUE); offsetFetcher = new OffsetFetcher(logContext, consumerNetworkClient, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 5cf5b9e2d92c5..f86e4a867632d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventType; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; @@ -328,12 +329,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { @Test public void testConsumerGroupMetadataFirstUpdate() { final GroupMetadataUpdateEvent groupMetadataUpdateEvent = makeFirstGroupMetadataUpdate(memberId, memberEpoch); - - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - memberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertGroupMetadataUpdateEvent(groupMetadataUpdateEvent, memberEpoch, memberId); } @Test @@ -368,13 +364,9 @@ public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated request.handler().onComplete(responseWithMemberEpochUpdate); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); + assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - updatedMemberEpoch, - memberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertGroupMetadataUpdateEvent(groupMetadataUpdateEvent, updatedMemberEpoch, memberId); } @Test @@ -396,13 +388,9 @@ public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame request.handler().onComplete(responseWithMemberIdUpdate); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); + assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - updatedMemberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertGroupMetadataUpdateEvent(groupMetadataUpdateEvent, memberEpoch, updatedMemberId); } private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String memberId, final int memberEpoch) { @@ -416,7 +404,7 @@ private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String membe request.handler().onComplete(firstResponse); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent event = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, event.type()); + assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, event.type()); return (GroupMetadataUpdateEvent) event; } @@ -705,6 +693,13 @@ private void assertNoHeartbeat(HeartbeatRequestManager hrm) { assertEquals(0, pollResult.unsentRequests.size()); } + private void assertGroupMetadataUpdateEvent(GroupMetadataUpdateEvent event, + int expectedMemberEpoch, + String expectedMemberId) { + assertEquals(expectedMemberEpoch, event.memberEpoch()); + assertEquals(expectedMemberId, event.memberId()); + } + private void mockStableMember() { membershipManager.onSubscriptionUpdated(); // Heartbeat response without assignment to set the state to STABLE. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 8ea8cb7a729f5..524e00daade85 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -27,6 +27,9 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -36,7 +39,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -44,6 +46,7 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { + private Time time; private ApplicationEventProcessor processor; private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); private RequestManagers requestManagers; @@ -61,6 +64,7 @@ public class ApplicationEventProcessorTest { @BeforeEach @SuppressWarnings("unchecked") public void setup() { + time = new MockTime(); LogContext logContext = new LogContext(); offsetRequestManager = mock(OffsetsRequestManager.class); offsetsRequestManager = mock(OffsetsRequestManager.class); @@ -96,18 +100,10 @@ public void testPrepClosingCommitEvents() { verify(commitRequestManager).signalClose(); } - @Test - public void testExpirationCalculation() { - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE)); - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); - long timeout = processor.getExpirationTimeForTimeout(1000); - assertTrue(timeout > 0); - assertTrue(timeout < Long.MAX_VALUE); - } - @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(); + Timer timer = time.timer(Long.MAX_VALUE); + LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); From 92d106293b98a669652200854d1df2418d809aa7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 27 Feb 2024 13:38:19 -0800 Subject: [PATCH 056/130] Updates to events --- .../internals/CommitRequestManager.java | 4 +- .../internals/events/ApplicationEvent.java | 51 +++++++++++++++++-- .../events/ApplicationEventType.java | 24 --------- .../AssignmentChangeApplicationEvent.java | 2 +- .../events/AsyncCommitApplicationEvent.java | 2 +- .../internals/events/BackgroundEvent.java | 47 +++++++++++++++-- .../internals/events/BackgroundEventType.java | 21 -------- .../events/CommitApplicationEvent.java | 2 +- .../events/CommitOnCloseApplicationEvent.java | 2 +- .../events/CompletableApplicationEvent.java | 2 +- .../events/CompletableBackgroundEvent.java | 2 +- ...balanceListenerCallbackCompletedEvent.java | 2 +- ...rRebalanceListenerCallbackNeededEvent.java | 2 +- .../events/ErrorBackgroundEvent.java | 2 +- ...FetchCommittedOffsetsApplicationEvent.java | 2 +- .../events/GroupMetadataUpdateEvent.java | 2 +- .../events/LeaveOnCloseApplicationEvent.java | 2 +- .../events/ListOffsetsApplicationEvent.java | 2 +- .../NewTopicsMetadataUpdateRequestEvent.java | 2 +- .../events/PollApplicationEvent.java | 2 +- .../ResetPositionsApplicationEvent.java | 2 +- .../SubscriptionChangeApplicationEvent.java | 2 +- .../events/SyncCommitApplicationEvent.java | 2 +- .../events/TopicMetadataApplicationEvent.java | 2 +- .../events/UnsubscribeApplicationEvent.java | 2 +- .../ValidatePositionsApplicationEvent.java | 2 +- .../HeartbeatRequestManagerTest.java | 8 +-- 27 files changed, 119 insertions(+), 78 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java 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 3625eb389841d..9206783d561be 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 @@ -774,7 +774,7 @@ abstract class RetriableRequestState extends RequestState { * errors. If not present, the request is triggered without waiting for a response or * retrying. */ - final Optional expirationTimeMs; + private final Optional expirationTimeMs; /** * True if the request expiration time has been reached. This is set when validating the @@ -901,7 +901,7 @@ public OffsetFetchRequestState(final Set partitions, } public boolean sameRequest(final OffsetFetchRequestState request) { - return requestedPartitions.equals(request.requestedPartitions) && expirationTimeMs.equals(request.expirationTimeMs); + return requestedPartitions.equals(request.requestedPartitions); } public NetworkClientDelegate.UnsentRequest toUnsentRequest() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index 78dd25f439a16..9dba061ded3e8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -16,13 +16,58 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.Uuid; + +import java.util.Objects; + /** * This is the abstract definition of the events created by the KafkaConsumer API on the user's * application thread. */ -public abstract class ApplicationEvent extends ConsumerEvent { +public abstract class ApplicationEvent { + + public enum Type { + COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, + LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, + UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, + COMMIT_ON_CLOSE, LEAVE_ON_CLOSE + } + + private final Type type; + + private final Uuid id; + + protected ApplicationEvent(Type type) { + this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); + } + + public Type type() { + return type; + } + + public Uuid id() { + return id; + } + + @Override + public final boolean equals(Object o) { + return this == o; + } + + @Override + public final int hashCode() { + return Objects.hash(type, id); + } + + protected String toStringBase() { + return "type=" + type + ", id=" + id; + } - protected ApplicationEvent(ApplicationEventType type) { - super(type); + @Override + public final String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java deleted file mode 100644 index 759eabeb1e2fe..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventType.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -public enum ApplicationEventType { - COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, - LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, - UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, - COMMIT_ON_CLOSE, LEAVE_ON_CLOSE -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java index 43058303e27db..e94d38639f45a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java @@ -22,7 +22,7 @@ import java.util.Collections; import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.ASSIGNMENT_CHANGE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.ASSIGNMENT_CHANGE; public class AssignmentChangeApplicationEvent extends ApplicationEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java index c2145e756fb5e..014ed80d4e225 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java @@ -22,7 +22,7 @@ import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_ASYNC; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.COMMIT_ASYNC; /** * Event to commit offsets without waiting for a response, so the request won't be retried. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index 65823c6fb6e65..f7de8dad7117f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -17,13 +17,54 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.common.Uuid; + +import java.util.Objects; /** * This is the abstract definition of the events created by the {@link ConsumerNetworkThread network thread}. */ -public abstract class BackgroundEvent extends ConsumerEvent { +public abstract class BackgroundEvent { + + public enum Type { + ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE + } + + private final Type type; + + private final Uuid id; + + protected BackgroundEvent(Type type) { + this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); + } + + public Type type() { + return type; + } + + public Uuid id() { + return id; + } + + @Override + public final boolean equals(Object o) { + return this == o; + } + + @Override + public final int hashCode() { + return Objects.hash(type, id); + } + + protected String toStringBase() { + return "type=" + type + ", id=" + id; + } - protected BackgroundEvent(BackgroundEventType type) { - super(type); + @Override + public final String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java deleted file mode 100644 index 6ea26eea0efb7..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventType.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -public enum BackgroundEventType { - ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java index dd1225f39856c..eb9f7fe939812 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java @@ -30,7 +30,7 @@ public abstract class CommitApplicationEvent extends CompletableApplicationEvent */ private final Map offsets; - protected CommitApplicationEvent(final ApplicationEventType type, + protected CommitApplicationEvent(final Type type, final Timer timer, final Map offsets) { super(type, timer); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java index 135181db56f88..fdd2a812a1eba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_ON_CLOSE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.COMMIT_ON_CLOSE; public class CommitOnCloseApplicationEvent extends ApplicationEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index f13b0acf632e8..d993e159ecda7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -31,7 +31,7 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im private final CompletableFuture future; private final long deadlineMs; - protected CompletableApplicationEvent(ApplicationEventType type, Timer timer) { + protected CompletableApplicationEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 64f08656c505e..fb1a98e47baa1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -31,7 +31,7 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; private final long deadlineMs; - protected CompletableBackgroundEvent(BackgroundEventType type, Timer timer) { + protected CompletableBackgroundEvent(Type type, Timer timer) { super(type); this.future = new CompletableFuture<>(); this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index 8f6c335c98242..2bd834231426d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -24,7 +24,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED; /** * Event that signifies that the application thread has executed the {@link ConsumerRebalanceListener} callback. If diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 991f7c788b805..ea2f442804c87 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -27,7 +27,7 @@ import java.util.Objects; import java.util.SortedSet; -import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEvent.Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED; /** * Event that signifies that the network I/O thread wants to invoke one of the callback methods on the diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index b15a434f87e60..24b1c680daef7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.KafkaException; -import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.ERROR; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEvent.Type.ERROR; public class ErrorBackgroundEvent extends BackgroundEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java index d81864aea174d..519345e450a31 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java @@ -24,7 +24,7 @@ import java.util.Map; import java.util.Set; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.FETCH_COMMITTED_OFFSETS; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.FETCH_COMMITTED_OFFSETS; public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicationEvent> { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 1c6a432b88321..9712e5972301c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import static org.apache.kafka.clients.consumer.internals.events.BackgroundEventType.GROUP_METADATA_UPDATE; +import static org.apache.kafka.clients.consumer.internals.events.BackgroundEvent.Type.GROUP_METADATA_UPDATE; /** * This event is sent by the {@link ConsumerNetworkThread consumer's network thread} to the application thread diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java index 792a18d649428..f2533e3e3fc5c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.utils.Timer; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.LEAVE_ON_CLOSE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.LEAVE_ON_CLOSE; public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java index c74adc1cc2c00..74207cd2a5d0c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java @@ -24,7 +24,7 @@ import java.util.HashMap; import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.LIST_OFFSETS; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.LIST_OFFSETS; /** * Event for retrieving partition offsets by performing a diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java index 8cc9d1c78de52..da36451ae0145 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.NEW_TOPICS_METADATA_UPDATE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.NEW_TOPICS_METADATA_UPDATE; public class NewTopicsMetadataUpdateRequestEvent extends ApplicationEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java index 6b0460bff90f0..01abe6fedb706 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.POLL; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.POLL; public class PollApplicationEvent extends ApplicationEvent { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java index f7ffbe43e57ab..2c537fcb2cc71 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.utils.Timer; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.RESET_POSITIONS; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.RESET_POSITIONS; /** * Event for resetting offsets for all assigned partitions that require it. This is an diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java index a458782751e86..946a3e001873e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.SUBSCRIPTION_CHANGE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.SUBSCRIPTION_CHANGE; /** * Application event indicating that the subscription state has changed, triggered when a user diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java index 6472cc93225fc..746b9be1969e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java @@ -22,7 +22,7 @@ import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.COMMIT_SYNC; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.COMMIT_SYNC; /** * Event to commit offsets waiting for a response and retrying on expected retriable errors until diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java index d01b0f0e0861a..9e24f36ae25f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java @@ -22,7 +22,7 @@ import java.util.List; import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.TOPIC_METADATA; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.TOPIC_METADATA; public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java index c17cc6164a171..856131b1ab304 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.utils.Timer; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.UNSUBSCRIBE; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.UNSUBSCRIBE; /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java index 6b39b83a41900..7edc629961ca6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.utils.Timer; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEventType.VALIDATE_POSITIONS; +import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.VALIDATE_POSITIONS; /** * Event for validating offsets for all assigned partitions for which a leader change has been diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f86e4a867632d..70307ea77d4be 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEventType; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent.Type; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; @@ -364,7 +364,7 @@ public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated request.handler().onComplete(responseWithMemberEpochUpdate); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); + assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; assertGroupMetadataUpdateEvent(groupMetadataUpdateEvent, updatedMemberEpoch, memberId); } @@ -388,7 +388,7 @@ public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame request.handler().onComplete(responseWithMemberIdUpdate); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); + assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; assertGroupMetadataUpdateEvent(groupMetadataUpdateEvent, memberEpoch, updatedMemberId); } @@ -404,7 +404,7 @@ private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String membe request.handler().onComplete(firstResponse); assertEquals(1, backgroundEventQueue.size()); final BackgroundEvent event = backgroundEventQueue.poll(); - assertEquals(BackgroundEventType.GROUP_METADATA_UPDATE, event.type()); + assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, event.type()); return (GroupMetadataUpdateEvent) event; } From dfec9d17a3502b2bb0f49c615ed8d49b7e61d534 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 27 Feb 2024 13:40:29 -0800 Subject: [PATCH 057/130] Remove Timer from ConsumerRebalanceListenerCallbackNeededEvent --- .../clients/consumer/internals/MembershipManagerImpl.java | 4 +--- .../internals/events/CompletableBackgroundEvent.java | 4 ++-- .../events/ConsumerRebalanceListenerCallbackNeededEvent.java | 5 ++--- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 4 ++-- 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 876a4b75d3da3..a9b0f3b94d831 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -38,7 +38,6 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -1289,8 +1288,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - Timer timer = time.timer(1000); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions, timer); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index fb1a98e47baa1..484825e4bbc79 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -31,10 +31,10 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; private final long deadlineMs; - protected CompletableBackgroundEvent(Type type, Timer timer) { + protected CompletableBackgroundEvent(Type type) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); + this.deadlineMs = Long.MAX_VALUE; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index ea2f442804c87..165b52cc6de55 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -41,9 +41,8 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions, - Timer timer) { - super(CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); + SortedSet partitions) { + super(CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } 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 5ccd0068355cb..f428fe146f9b6 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 @@ -440,7 +440,7 @@ public void testCommitInRebalanceCallback() { doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions, time.timer(1000)); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); backgroundEventQueue.add(e); completeCommitSyncApplicationEventSuccessfully(); final AtomicBoolean callbackExecuted = new AtomicBoolean(false); @@ -1271,7 +1271,7 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions, time.timer(1000)); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. From 44757aae9db2038151505958d4a09ab2c3cfb894 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 29 Feb 2024 21:17:45 -0800 Subject: [PATCH 058/130] Updates to clean up more clumsiness --- .../internals/AsyncKafkaConsumer.java | 25 ++---- .../internals/ConsumerNetworkThread.java | 6 -- .../events/ApplicationEventProcessor.java | 83 +++++++++++++++++-- .../internals/events/AsyncCommitEvent.java | 2 - .../events/CompletableApplicationEvent.java | 1 - .../events/CompletableBackgroundEvent.java | 7 -- .../internals/events/CompletableEvent.java | 2 - .../internals/events/EventProcessor.java | 69 +++++---------- 8 files changed, 101 insertions(+), 94 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 165c5aa602156..38655126c88ae 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 @@ -770,8 +770,8 @@ public void commitAsync(Map offsets, OffsetCo acquireAndEnsureOpen(); try { Timer timer = time.timer(Long.MAX_VALUE); - AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets, timer); - CompletableFuture future = commit(asyncCommitEvent); + AsyncCommitEvent event = new AsyncCommitEvent(offsets, timer); + CompletableFuture future = commit(event); future.whenComplete((r, t) -> { if (t == null) { @@ -939,9 +939,7 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event); @@ -1574,8 +1572,7 @@ private boolean updateFetchPositions(final Timer timer) { applicationEventHandler.addAndGet(new ValidatePositionsEvent(timer)); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); - if (cachedSubscriptionHasAllFetchPositions) - return true; + if (cachedSubscriptionHasAllFetchPositions) return true; // Reset positions using committed offsets retrieved from the group coordinator, for any // partitions which do not have a valid position and are not awaiting reset. This will @@ -1651,12 +1648,7 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); - - try { - backgroundEventProcessor.process(); - } finally { - backgroundEventProcessor.completeExpiredEvents(time.milliseconds()); - } + backgroundEventProcessor.process(); // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as // in the previous implementation, because it will eventually involve group coordination @@ -1827,12 +1819,7 @@ T processBackgroundEvents(Future future, Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents; - try { - hadEvents = backgroundEventProcessor.process(); - } finally { - backgroundEventProcessor.completeExpiredEvents(time.milliseconds()); - } + boolean hadEvents = backgroundEventProcessor.process(); try { if (future.isDone()) { 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 f5e3d5aa69865..59e48af86d850 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,8 +20,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -32,14 +30,10 @@ import java.io.Closeable; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; 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 0caf9479b8eb5..28fd66b6a2a8c 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 @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; @@ -27,15 +28,19 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; +import java.util.Collection; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Supplier; /** @@ -47,6 +52,7 @@ public class ApplicationEventProcessor extends EventProcessor private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; + private final List> completableEvents; public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, @@ -56,6 +62,7 @@ public ApplicationEventProcessor(final LogContext logContext, this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; + this.completableEvents = new LinkedList<>(); } /** @@ -64,7 +71,12 @@ public ApplicationEventProcessor(final LogContext logContext, * errors to be propagated to the caller. */ public void process() { - process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); + process((event, error) -> { + if (event instanceof CompletableEvent) + completableEvents.add((CompletableApplicationEvent) event); + + error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); + }); } @SuppressWarnings({"CyclomaticComplexity"}) @@ -140,6 +152,39 @@ public void process(ApplicationEvent event) { } } + /** + * We "complete" any of the {@link CompletableApplicationEvent}s that are either expired or done. + * + *

+ * + * Note: this cleanup step should be called after the user has processed the {@link #process(ProcessHandler)} is executed. + * The reason for this is to emulate the behavior of the legacy consumer's handling of timeouts. The legacy + * consumer would + * difference between the so that we can perform this cleanup after any events t + * + * @param currentTimeMs Current time + */ + public void completeExpiredEvents(long currentTimeMs) { + log.trace("Removing expired events"); + + Consumer> completeEvent = e -> { + long pastDueMs = currentTimeMs - e.deadlineMs(); + log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); + CompletableFuture f = e.future(); + f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); + }; + + // First, complete (exceptionally) any events that have passed their deadline. + completableEvents + .stream() + .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) + .forEach(completeEvent); + + // Second, remove any events that are already done, just to make sure we don't hold references. + completableEvents.removeIf(e -> e.future().isDone()); + log.trace("Finished removal of expired events"); + } + private void process(final PollEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; @@ -165,8 +210,7 @@ private void process(final SyncCommitEvent event) { } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = event.deadlineMs(); - CompletableFuture future = manager.commitSync(event.offsets(), expirationTimeoutMs); + CompletableFuture future = manager.commitSync(event.offsets(), event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -177,8 +221,10 @@ private void process(final FetchCommittedOffsetsEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = event.deadlineMs(); - CompletableFuture> future = manager.fetchOffsets(event.partitions(), expirationTimeMs); + CompletableFuture> future = manager.fetchOffsets( + event.partitions(), + event.deadlineMs() + ); future.whenComplete(complete(event.future())); } @@ -200,9 +246,10 @@ private void process(final AssignmentChangeEvent event) { } private void process(final ListOffsetsEvent event) { - final CompletableFuture> future = - requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), - event.requireTimestamps()); + final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets( + event.timestampsToSearch(), + event.requireTimestamps() + ); future.whenComplete(complete(event.future())); } @@ -303,6 +350,26 @@ private void process(final LeaveOnCloseEvent event) { }; } + /** + * Check each of the {@link CompletableApplicationEvent completable events}, and for any that are + * incomplete, {@link CompletableFuture#completeExceptionally(Throwable) complete it exceptionally}. + * + *

+ * + * Note: because this is called in the context of {@link AsyncKafkaConsumer#close() closing consumer}, + * don't take the deadline into consideration, just close it regardless. + */ + @Override + protected void cancelIncompleteEvents() { + super.cancelIncompleteEvents(); + + completableEvents + .stream() + .filter(e -> !e.future().isDone()) + .forEach(INCOMPLETE_EVENT_CANCELLER); + completableEvents.clear(); + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index e9b331548019e..aadc3865cb892 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -22,8 +22,6 @@ import java.util.Map; -import static org.apache.kafka.clients.consumer.internals.events.ApplicationEvent.Type.COMMIT_ASYNC; - /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 88fb1c6a13eaf..de731c64431d7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -44,7 +44,6 @@ public CompletableFuture future() { return future; } - @Override public long deadlineMs() { return deadlineMs; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index bf0f48375f000..1a58515a5cbce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -27,12 +27,10 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; - private final long deadlineMs; protected CompletableBackgroundEvent(final Type type) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = Long.MAX_VALUE; } @Override @@ -40,11 +38,6 @@ public CompletableFuture future() { return future; } - @Override - public long deadlineMs() { - return deadlineMs; - } - @Override protected String toStringBase() { return super.toStringBase() + ", future=" + future; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 56ddd07219913..97559d8cb9be2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,6 +21,4 @@ public interface CompletableEvent { CompletableFuture future(); - - long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 2efb49de46004..bb96f0f4e658b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -18,14 +18,12 @@ import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; import java.io.Closeable; import java.util.LinkedList; -import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.concurrent.BlockingQueue; @@ -41,23 +39,36 @@ */ public abstract class EventProcessor implements Closeable { + protected final static Consumer> INCOMPLETE_EVENT_CANCELLER = e -> { + CompletableFuture f = e.future(); + f.cancel(true); + }; + private final Logger log; private final BlockingQueue eventQueue; - private final List> completableEvents; private final IdempotentCloser closer; protected EventProcessor(final LogContext logContext, final BlockingQueue eventQueue) { this.log = logContext.logger(EventProcessor.class); this.eventQueue = eventQueue; this.closer = new IdempotentCloser(); - this.completableEvents = new LinkedList<>(); } protected abstract void process(T event); @Override public void close() { - closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); + closer.close( + () -> { + try { + log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); + cancelIncompleteEvents(); + } finally { + log.debug("Finished removal of events that were unprocessed and/or unfinished"); + } + }, + () -> log.warn("The event processor was already closed") + ); } protected interface ProcessHandler { @@ -88,8 +99,6 @@ protected boolean process(ProcessHandler processHandler) { Objects.requireNonNull(event, "Attempted to process a null event"); log.trace("Processing event: {}", event); - if (event instanceof CompletableEvent) - completableEvents.add((CompletableEvent) event); process(event); processHandler.onProcess(event, Optional.empty()); @@ -105,56 +114,18 @@ protected boolean process(ProcessHandler processHandler) { return true; } - public void completeExpiredEvents(long currentTimeMs) { - log.trace("Removing expired events"); - - Consumer> completeEvent = e -> { - log.debug("Completing event {} exceptionally since it expired {} ms ago", e, currentTimeMs - e.deadlineMs()); - CompletableFuture f = e.future(); - f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); - }; - - // Complete (exceptionally) any events that have recently passed their deadline. - completableEvents - .stream() - .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) - .forEach(completeEvent); - - // Remove any events that are complete. - completableEvents.removeIf(e -> e.future().isDone()); - log.trace("Finished removal of expired events"); - } - /** * It is possible for the consumer to close before complete processing all the events in the queue. In - * this case, we need to throw an exception to notify the user the consumer is closed. + * this case, we need to {@link CompletableFuture#completeExceptionally(Throwable) (exceptionally) complete} + * any remaining events. */ - private void closeInternal() { - log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); - - Consumer> completeEvent = e -> { - log.debug("Completing event {} exceptionally since the consumer is closing", e); - CompletableFuture f = e.future(); - f.completeExceptionally(new KafkaException("The consumer is closed")); - }; - - // Check each of the unprocessed events and if it has a Future that is incomplete, complete it exceptionally. + protected void cancelIncompleteEvents() { eventQueue .stream() .filter(e -> e instanceof CompletableEvent) .map(e -> (CompletableEvent) e) .filter(e -> !e.future().isDone()) - .forEach(completeEvent); + .forEach(INCOMPLETE_EVENT_CANCELLER); eventQueue.clear(); - - // Check each of the completable events and if it has a Future that is incomplete, complete it exceptionally. - // In the case of shutdown, we don't take the deadline into consideration. - completableEvents - .stream() - .filter(e -> !e.future().isDone()) - .forEach(completeEvent); - completableEvents.clear(); - - log.debug("Finished removal of events that were unprocessed and/or unfinished"); } } From 84c8f8dfc5696a5bb4735948d9246b15908a7730 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 13:52:01 -0800 Subject: [PATCH 059/130] Update ApplicationEventProcessor.java --- .../events/ApplicationEventProcessor.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 28fd66b6a2a8c..7711076d211d5 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 @@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.MembershipManager; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; @@ -153,16 +154,14 @@ public void process(ApplicationEvent event) { } /** - * We "complete" any of the {@link CompletableApplicationEvent}s that are either expired or done. + * This method "completes" any {@link CompletableApplicationEvent}s that have expired or completed. This cleanup + * step should only be called after the {@link ConsumerNetworkThread network I/O thread} has made at least + * one call to {@link NetworkClientDelegate#poll(long, long) poll}. This is done to emulate the behavior of the + * legacy consumer's handling of timeouts. The legacy consumer makes at least one attempt to satisfy any network + * requests before checking if a timeout has expired. * - *

- * - * Note: this cleanup step should be called after the user has processed the {@link #process(ProcessHandler)} is executed. - * The reason for this is to emulate the behavior of the legacy consumer's handling of timeouts. The legacy - * consumer would - * difference between the so that we can perform this cleanup after any events t - * - * @param currentTimeMs Current time + * @param currentTimeMs Current time with which to compare against the + * {@link CompletableApplicationEvent#deadlineMs() expiration time} */ public void completeExpiredEvents(long currentTimeMs) { log.trace("Removing expired events"); @@ -180,7 +179,7 @@ public void completeExpiredEvents(long currentTimeMs) { .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) .forEach(completeEvent); - // Second, remove any events that are already done, just to make sure we don't hold references. + // Second, remove any events that are already complete, just to make sure we don't hold references. completableEvents.removeIf(e -> e.future().isDone()); log.trace("Finished removal of expired events"); } From b3519aab7af2c161b45def89a311c3c9acd345bc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 13:59:48 -0800 Subject: [PATCH 060/130] Reverting unnecessary changes --- .../internals/events/EventProcessor.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index bb96f0f4e658b..fc075c99879aa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -58,17 +58,7 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve @Override public void close() { - closer.close( - () -> { - try { - log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); - cancelIncompleteEvents(); - } finally { - log.debug("Finished removal of events that were unprocessed and/or unfinished"); - } - }, - () -> log.warn("The event processor was already closed") - ); + closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); } protected interface ProcessHandler { @@ -98,8 +88,6 @@ protected boolean process(ProcessHandler processHandler) { try { Objects.requireNonNull(event, "Attempted to process a null event"); log.trace("Processing event: {}", event); - - process(event); processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { @@ -114,6 +102,15 @@ protected boolean process(ProcessHandler processHandler) { return true; } + private void closeInternal() { + try { + log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); + cancelIncompleteEvents(); + } finally { + log.debug("Finished removal of events that were unprocessed and/or unfinished"); + } + } + /** * It is possible for the consumer to close before complete processing all the events in the queue. In * this case, we need to {@link CompletableFuture#completeExceptionally(Throwable) (exceptionally) complete} From 00eb0cb630b30e2c894cc1958addbb398198fe11 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 14:03:20 -0800 Subject: [PATCH 061/130] Reverting more code to make diffs less --- .../internals/events/EventProcessor.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index fc075c99879aa..10b4aa8fd16d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -24,6 +24,7 @@ import java.io.Closeable; import java.util.LinkedList; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.concurrent.BlockingQueue; @@ -73,8 +74,7 @@ protected interface ProcessHandler { protected boolean process(ProcessHandler processHandler) { closer.assertOpen("The processor was previously closed, so no further processing can occur"); - LinkedList events = new LinkedList<>(); - eventQueue.drainTo(events); + List events = drain(); if (events.isEmpty()) { log.trace("No events to process"); @@ -117,12 +117,21 @@ private void closeInternal() { * any remaining events. */ protected void cancelIncompleteEvents() { - eventQueue + List events = drain(); + events .stream() .filter(e -> e instanceof CompletableEvent) .map(e -> (CompletableEvent) e) .filter(e -> !e.future().isDone()) .forEach(INCOMPLETE_EVENT_CANCELLER); - eventQueue.clear(); } -} + + /** + * Moves all the events from the queue to the returned list. + */ + private List drain() { + LinkedList events = new LinkedList<>(); + eventQueue.drainTo(events); + return events; + } +} \ No newline at end of file From 528dc4f67d6964b852b6a3590064b3c792b3568d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 14:03:29 -0800 Subject: [PATCH 062/130] Update ApplicationEventProcessor.java --- .../consumer/internals/events/ApplicationEventProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 7711076d211d5..95a22a4c5e741 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -33,7 +33,6 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; From 036e33b28b8a81f4a3af316c263d9e2087fa075c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 14:05:45 -0800 Subject: [PATCH 063/130] Update EventProcessor.java --- .../internals/events/EventProcessor.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 10b4aa8fd16d1..159d8e44be656 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.internals.IdempotentCloser; @@ -29,7 +30,6 @@ import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; /** * An {@link EventProcessor} is the means by which events produced by thread A are @@ -40,7 +40,7 @@ */ public abstract class EventProcessor implements Closeable { - protected final static Consumer> INCOMPLETE_EVENT_CANCELLER = e -> { + protected final static java.util.function.Consumer> INCOMPLETE_EVENT_CANCELLER = e -> { CompletableFuture f = e.future(); f.cancel(true); }; @@ -102,19 +102,10 @@ protected boolean process(ProcessHandler processHandler) { return true; } - private void closeInternal() { - try { - log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); - cancelIncompleteEvents(); - } finally { - log.debug("Finished removal of events that were unprocessed and/or unfinished"); - } - } - /** - * It is possible for the consumer to close before complete processing all the events in the queue. In - * this case, we need to {@link CompletableFuture#completeExceptionally(Throwable) (exceptionally) complete} - * any remaining events. + * It is possible for the {@link Consumer#close() consumer to close} before completing the processing of all + * the events in the queue. In this case, we need to + * {@link CompletableFuture#completeExceptionally(Throwable) (exceptionally) complete} any remaining events. */ protected void cancelIncompleteEvents() { List events = drain(); @@ -126,6 +117,15 @@ protected void cancelIncompleteEvents() { .forEach(INCOMPLETE_EVENT_CANCELLER); } + private void closeInternal() { + try { + log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); + cancelIncompleteEvents(); + } finally { + log.debug("Finished removal of events that were unprocessed and/or unfinished"); + } + } + /** * Moves all the events from the queue to the returned list. */ @@ -134,4 +134,4 @@ private List drain() { eventQueue.drainTo(events); return events; } -} \ No newline at end of file +} From e8dfea3601cf580c6a9d11a5949f122fe087b227 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Mar 2024 14:12:04 -0800 Subject: [PATCH 064/130] Update ApplicationEventProcessor.java --- .../consumer/internals/events/ApplicationEventProcessor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 95a22a4c5e741..324b8e6959d00 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 @@ -72,6 +72,7 @@ public ApplicationEventProcessor(final LogContext logContext, */ public void process() { process((event, error) -> { + // Store any CompletableApplicationEvents so we can check them for expiration later. if (event instanceof CompletableEvent) completableEvents.add((CompletableApplicationEvent) event); @@ -318,7 +319,7 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event manager.consumerRebalanceListenerCallbackCompleted(event); } - private void process(final CommitOnCloseEvent event) { + private void process(@SuppressWarnings("unused") final CommitOnCloseEvent event) { if (!requestManagers.commitRequestManager.isPresent()) return; log.debug("Signal CommitRequestManager closing"); From 027b2328ff13e94746aae703861ae5aae7fe78b8 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Mar 2024 13:18:15 -0800 Subject: [PATCH 065/130] Updates --- .../consumer/internals/AsyncKafkaConsumer.java | 3 ++- .../internals/events/AsyncCommitEvent.java | 5 +++-- .../internals/ConsumerNetworkThreadTest.java | 14 +++++--------- 3 files changed, 10 insertions(+), 12 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 daf0a9861f1d0..330d9c9c29db3 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 @@ -766,7 +766,8 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - AsyncCommitEvent event = new AsyncCommitEvent(offsets); + Timer timer = time.timer(Long.MAX_VALUE); + AsyncCommitEvent event = new AsyncCommitEvent(offsets, timer); CompletableFuture future = commit(event); future.whenComplete((r, t) -> { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index c36f0534b3671..971129d5b86fd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Map; @@ -26,7 +27,7 @@ */ public class AsyncCommitEvent extends CommitEvent { - public AsyncCommitEvent(final Map offsets) { - super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE); + public AsyncCommitEvent(final Map offsets, final Timer timer) { + super(Type.COMMIT_ASYNC, offsets, timer); } } 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 415699886e339..6399e722685d4 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 @@ -86,11 +86,8 @@ public class ConsumerNetworkThreadTest { private OffsetsRequestManager offsetsRequestManager; private CommitRequestManager commitRequestManager; private CoordinatorRequestManager coordinatorRequestManager; - private HeartbeatRequestManager heartbeatRequestManager; - private MembershipManager memberhipsManager; private ConsumerNetworkThread consumerNetworkThread; private MockClient client; - private SubscriptionState subscriptions; @BeforeEach public void setup() { @@ -104,10 +101,7 @@ public void setup() { commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); offsetsRequestManager = testBuilder.offsetsRequestManager; coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); - heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new); - memberhipsManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); consumerNetworkThread = testBuilder.consumerNetworkThread; - subscriptions = testBuilder.subscriptions; consumerNetworkThread.initializeResources(); } @@ -154,7 +148,8 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitEvent(new HashMap<>()); + Timer timer = time.timer(Long.MAX_VALUE); + ApplicationEvent e = new AsyncCommitEvent(new HashMap<>(), timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(AsyncCommitEvent.class)); @@ -285,8 +280,9 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); - ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); + Timer timer = time.timer(Long.MAX_VALUE); + CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap(), timer)); + ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); From 1d4d6b0cfe24cd8d9ea4b52aab68e3f4372ca4ee Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Mar 2024 13:41:45 -0800 Subject: [PATCH 066/130] Updates --- .../consumer/internals/ConsumerNetworkThread.java | 7 +++---- .../consumer/internals/AsyncKafkaConsumerTest.java | 10 ++++------ .../consumer/internals/FetchRequestManagerTest.java | 3 +-- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 59e48af86d850..78418fcd68a36 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 @@ -172,8 +172,7 @@ void runOnce() { */ // Visible for testing static void runAtClose(final Collection> requestManagers, - final NetworkClientDelegate networkClientDelegate, - final Timer timer) { + final NetworkClientDelegate networkClientDelegate) { // These are the optional outgoing requests at the requestManagers.stream() .filter(Optional::isPresent) @@ -268,12 +267,12 @@ private void sendUnsentRequests(final Timer timer) { void cleanup() { log.trace("Closing the consumer network thread"); - Timer timer = time.timer(closeTimeout); try { - runAtClose(requestManagers.entries(), networkClientDelegate, timer); + runAtClose(requestManagers.entries(), networkClientDelegate); } catch (Exception e) { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { + Timer timer = time.timer(closeTimeout); sendUnsentRequests(timer); closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); 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 78b6bbae8e479..2b655dc1f9f9f 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 @@ -29,7 +29,6 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.RoundRobinAssignor; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; @@ -933,12 +932,11 @@ public void testNoWakeupInCloseCommit() { AtomicReference capturedEvent = new AtomicReference<>(); doAnswer(invocation -> { - ApplicationEvent event = invocation.getArgument(0); - if (event instanceof SyncCommitEvent) { - capturedEvent.set((SyncCommitEvent) event); - } + SyncCommitEvent event = invocation.getArgument(0); + capturedEvent.set(event); + event.future().complete(null); return null; - }).when(applicationEventHandler).add(any()); + }).when(applicationEventHandler).add(any(SyncCommitEvent.class)); consumer.close(Duration.ZERO); 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 2024b693d473a..ed0aeef85d8d1 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 @@ -365,11 +365,10 @@ public void testFetcherCloseClosesFetchSessionsInBroker() { final ArgumentCaptor argument = ArgumentCaptor.forClass(NetworkClientDelegate.UnsentRequest.class); - Timer timer = time.timer(Duration.ofSeconds(10)); // NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling // the close() method with a Timer will NOT send out the close session requests on close. The network // I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here. - ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer); + ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate); // the network is polled during the last state of clean up. networkClientDelegate.poll(time.timer(1)); // validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the From c00ffc239fb2ae2ced6e1f916f3714ed5ce9aa6c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Mar 2024 14:29:23 -0800 Subject: [PATCH 067/130] Updates to timeout values --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 ++-- .../clients/consumer/internals/ConsumerNetworkThreadTest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 330d9c9c29db3..357cb371a6fd5 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 @@ -766,7 +766,7 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(defaultApiTimeoutMs); AsyncCommitEvent event = new AsyncCommitEvent(offsets, timer); CompletableFuture future = commit(event); future.whenComplete((r, t) -> { @@ -1470,7 +1470,7 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.get().isPresent()) { - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(defaultApiTimeoutMs); UnsubscribeEvent event = new UnsubscribeEvent(timer); applicationEventHandler.add(event); log.info("Unsubscribing all topics or patterns and assigned partitions"); 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 6399e722685d4..ed0561d323130 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 @@ -148,7 +148,7 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(100); ApplicationEvent e = new AsyncCommitEvent(new HashMap<>(), timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); @@ -280,7 +280,7 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(100); CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap(), timer)); ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap(), timer); CompletableFuture future = new CompletableFuture<>(); From 25eea605f3973950df6db0a756b394d41d49a52f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Sun, 10 Mar 2024 07:29:24 -0700 Subject: [PATCH 068/130] Refactored the event processors and event reaper --- .../internals/AsyncKafkaConsumer.java | 81 ++++++------ .../internals/ConsumerNetworkThread.java | 54 +++++++- .../events/ApplicationEventHandler.java | 1 + .../events/ApplicationEventProcessor.java | 80 +----------- .../events/CompletableBackgroundEvent.java | 5 + .../internals/events/CompletableEvent.java | 2 + .../events/CompletableEventReaper.java | 115 ++++++++++++++++++ .../internals/events/EventProcessor.java | 110 +---------------- .../internals/ConsumerTestBuilder.java | 3 +- .../events/ApplicationEventProcessorTest.java | 4 - 10 files changed, 224 insertions(+), 231 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.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 357cb371a6fd5..1def7f7975685 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 @@ -47,6 +47,8 @@ import org.apache.kafka.clients.consumer.internals.events.CommitEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; @@ -100,6 +102,7 @@ import java.util.ConcurrentModificationException; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -165,17 +168,11 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { *

  • {@link ConsumerRebalanceListener} callbacks that are to be executed on the application thread
  • * */ - private class BackgroundEventProcessor extends EventProcessor { + private class BackgroundEventProcessor implements EventProcessor { - private final ApplicationEventHandler applicationEventHandler; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; - public BackgroundEventProcessor(final LogContext logContext, - final BlockingQueue backgroundEventQueue, - final ApplicationEventHandler applicationEventHandler, - final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { - super(logContext, backgroundEventQueue); - this.applicationEventHandler = applicationEventHandler; + public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { this.rebalanceListenerInvoker = rebalanceListenerInvoker; } @@ -188,22 +185,42 @@ public BackgroundEventProcessor(final LogContext logContext, public boolean process() { AtomicReference firstError = new AtomicReference<>(); - ProcessHandler processHandler = (event, error) -> { - if (error.isPresent()) { - KafkaException e = error.get(); + LinkedList events = new LinkedList<>(); + backgroundEventQueue.drainTo(events); - if (!firstError.compareAndSet(null, e)) { - log.warn("An error occurred when processing the event: {}", e.getMessage(), e); + if (events.isEmpty()) { + log.trace("No background events to process"); + return false; + } + + try { + log.trace("Starting processing of {} background event{}", events.size(), events.size() == 1 ? "" : "s"); + + for (BackgroundEvent event : events) { + try { + Objects.requireNonNull(event, "Attempted to process a null background event"); + + if (event instanceof CompletableEvent) + backgroundEventReaper.add((CompletableEvent) event); + + log.trace("Processing background event: {}", event); + process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + + if (!firstError.compareAndSet(null, e)) { + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); + } } } - }; - - boolean hadEvents = process(processHandler); + } finally { + log.trace("Completed processing background event(s)"); + } if (firstError.get() != null) throw firstError.get(); - return hadEvents; + return true; } @Override @@ -244,7 +261,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final KafkaConsumerMetrics kafkaConsumerMetrics; private Logger log; private final String clientId; + private final BlockingQueue backgroundEventQueue; private final BackgroundEventProcessor backgroundEventProcessor; + private final CompletableEventReaper backgroundEventReaper; private final Deserializers deserializers; /** @@ -311,6 +330,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { 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()); log.debug("Initializing the Kafka consumer"); @@ -375,7 +395,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, - applicationEventQueue, requestManagersSupplier); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, @@ -392,11 +411,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { new RebalanceCallbackMetricsManager(metrics) ); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, rebalanceListenerInvoker ); + this.backgroundEventReaper = new CompletableEventReaper(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -458,12 +475,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; this.interceptors = Objects.requireNonNull(interceptors); this.time = time; - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ); + this.backgroundEventQueue = backgroundEventQueue; + this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); + this.backgroundEventReaper = new CompletableEventReaper(logContext); this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; @@ -523,7 +537,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); - BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + this.backgroundEventQueue = new LinkedBlockingQueue<>(); BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( logContext, backgroundEventQueue @@ -563,7 +577,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, - applicationEventQueue, requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, @@ -572,12 +585,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ); + this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); + this.backgroundEventReaper = new CompletableEventReaper(logContext); } // auxiliary interface for testing @@ -1248,6 +1257,8 @@ private void close(Duration timeout, boolean swallowException) { swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", this::maybeInvokeCommitCallbacks, firstException); closeTimer.update(); + if (backgroundEventReaper != null && backgroundEventQueue != null) + backgroundEventReaper.reapIncomplete(backgroundEventQueue); closeQuietly(interceptors, "consumer interceptors", firstException); closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); closeQuietly(metrics, "consumer metrics", firstException); 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 78418fcd68a36..92e0238994f55 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -31,9 +33,11 @@ import java.io.Closeable; import java.time.Duration; import java.util.Collection; +import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; @@ -50,6 +54,8 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread"; private final Time time; private final Logger log; + private final BlockingQueue applicationEventQueue; + private final CompletableEventReaper applicationEventReaper; private final Supplier applicationEventProcessorSupplier; private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; @@ -63,12 +69,15 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { public ConsumerNetworkThread(LogContext logContext, Time time, + BlockingQueue applicationEventQueue, Supplier applicationEventProcessorSupplier, Supplier networkClientDelegateSupplier, Supplier requestManagersSupplier) { super(BACKGROUND_THREAD_NAME, true); this.time = time; this.log = logContext.logger(getClass()); + this.applicationEventQueue = applicationEventQueue; + this.applicationEventReaper = new CompletableEventReaper(logContext); this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; this.networkClientDelegateSupplier = networkClientDelegateSupplier; this.requestManagersSupplier = requestManagersSupplier; @@ -128,7 +137,7 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - applicationEventProcessor.process(); + processEvents(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -145,7 +154,11 @@ void runOnce() { .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); - applicationEventProcessor.completeExpiredEvents(currentTimeMs); + // "Complete" any events that have expired. This cleanup step should only be called after the network I/O + // thread has made at least one call to poll. This is done to emulate the behavior of the legacy consumer's + // handling of timeouts. The legacy consumer makes at least one attempt to satisfy any network requests + // before checking if a timeout has expired. + applicationEventReaper.reapExpired(currentTimeMs); } /** @@ -181,6 +194,41 @@ static void runAtClose(final Collection> requ .forEach(networkClientDelegate::addAll); } + /** + * Drains all available events from the queue, and then processes them in order. If any errors are thrown while + * processing the individual events, these are logged and skipped. + */ + protected void processEvents() { + closer.assertOpen("The processor was previously closed, so no further processing can occur"); + + LinkedList events = new LinkedList<>(); + applicationEventQueue.drainTo(events); + + if (events.isEmpty()) { + log.trace("No events to process"); + } + + try { + log.trace("Starting processing of {} event{}", events.size(), events.size() == 1 ? "" : "s"); + + for (ApplicationEvent event : events) { + try { + Objects.requireNonNull(event, "Attempted to process a null event"); + + if (event instanceof CompletableEvent) + applicationEventReaper.add((CompletableEvent) event); + + log.trace("Processing event: {}", event); + applicationEventProcessor.process(event); + } catch (Throwable t) { + log.warn("Error processing event {}", t.getMessage(), t); + } + } + } finally { + log.trace("Completed processing"); + } + } + public boolean isRunning() { return running; } @@ -274,9 +322,9 @@ void cleanup() { } finally { Timer timer = time.timer(closeTimeout); sendUnsentRequests(timer); + applicationEventReaper.reapIncomplete(applicationEventQueue); closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); - closeQuietly(applicationEventProcessor, "application event processor"); log.debug("Closed the consumer network thread"); } } 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 b97e9d931cdad..f1931ff292ad8 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 @@ -53,6 +53,7 @@ public ApplicationEventHandler(final LogContext logContext, this.applicationEventQueue = applicationEventQueue; this.networkThread = new ConsumerNetworkThread(logContext, time, + applicationEventQueue, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 324b8e6959d00..820911856117b 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 @@ -18,66 +18,41 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; -import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.MembershipManager; -import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; -import java.util.function.Consumer; import java.util.function.Supplier; /** * An {@link EventProcessor} that is created and executes in the {@link ConsumerNetworkThread network thread} * which processes {@link ApplicationEvent application events} generated by the application thread. */ -public class ApplicationEventProcessor extends EventProcessor { +public class ApplicationEventProcessor implements EventProcessor { private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; - private final List> completableEvents; public ApplicationEventProcessor(final LogContext logContext, - final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { - super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; - this.completableEvents = new LinkedList<>(); - } - - /** - * Process the events—if any—that were produced by the application thread. It is possible that when processing - * an event generates an error. In such cases, the processor will log an exception, but we do not want those - * errors to be propagated to the caller. - */ - public void process() { - process((event, error) -> { - // Store any CompletableApplicationEvents so we can check them for expiration later. - if (event instanceof CompletableEvent) - completableEvents.add((CompletableApplicationEvent) event); - - error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e)); - }); } @SuppressWarnings({"CyclomaticComplexity"}) @@ -153,37 +128,6 @@ public void process(ApplicationEvent event) { } } - /** - * This method "completes" any {@link CompletableApplicationEvent}s that have expired or completed. This cleanup - * step should only be called after the {@link ConsumerNetworkThread network I/O thread} has made at least - * one call to {@link NetworkClientDelegate#poll(long, long) poll}. This is done to emulate the behavior of the - * legacy consumer's handling of timeouts. The legacy consumer makes at least one attempt to satisfy any network - * requests before checking if a timeout has expired. - * - * @param currentTimeMs Current time with which to compare against the - * {@link CompletableApplicationEvent#deadlineMs() expiration time} - */ - public void completeExpiredEvents(long currentTimeMs) { - log.trace("Removing expired events"); - - Consumer> completeEvent = e -> { - long pastDueMs = currentTimeMs - e.deadlineMs(); - log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); - CompletableFuture f = e.future(); - f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); - }; - - // First, complete (exceptionally) any events that have passed their deadline. - completableEvents - .stream() - .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) - .forEach(completeEvent); - - // Second, remove any events that are already complete, just to make sure we don't hold references. - completableEvents.removeIf(e -> e.future().isDone()); - log.trace("Finished removal of expired events"); - } - private void process(final PollEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; @@ -349,33 +293,12 @@ private void process(final LeaveOnCloseEvent event) { }; } - /** - * Check each of the {@link CompletableApplicationEvent completable events}, and for any that are - * incomplete, {@link CompletableFuture#completeExceptionally(Throwable) complete it exceptionally}. - * - *

    - * - * Note: because this is called in the context of {@link AsyncKafkaConsumer#close() closing consumer}, - * don't take the deadline into consideration, just close it regardless. - */ - @Override - protected void cancelIncompleteEvents() { - super.cancelIncompleteEvents(); - - completableEvents - .stream() - .filter(e -> !e.future().isDone()) - .forEach(INCOMPLETE_EVENT_CANCELLER); - completableEvents.clear(); - } - /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, - final BlockingQueue applicationEventQueue, final Supplier requestManagersSupplier) { return new CachedSupplier() { @Override @@ -383,7 +306,6 @@ protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); return new ApplicationEventProcessor( logContext, - applicationEventQueue, requestManagers, metadata ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 1a58515a5cbce..ab444cada0f44 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -38,6 +38,11 @@ public CompletableFuture future() { return future; } + @Override + public long deadlineMs() { + return Long.MAX_VALUE; + } + @Override protected String toStringBase() { return super.toStringBase() + ", future=" + future; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 97559d8cb9be2..56ddd07219913 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,4 +21,6 @@ public interface CompletableEvent { CompletableFuture future(); + + long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java new file mode 100644 index 0000000000000..1c0187b33f518 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.function.Consumer; + +public class CompletableEventReaper { + + private final Logger log; + private final List> completableEvents; + + public CompletableEventReaper(LogContext logContext) { + this.log = logContext.logger(CompletableEventReaper.class); + this.completableEvents = new ArrayList<>(); + } + + public void add(CompletableEvent event) { + completableEvents.add(event); + } + + /** + * This method "completes" any {@link CompletableEvent}s that have expired. + * + * @param currentTimeMs Current time with which to compare against the + * {@link CompletableEvent#deadlineMs() expiration time} + */ + public void reapExpired(long currentTimeMs) { + log.trace("Reaping expired events"); + + Consumer> completeEvent = e -> { + long pastDueMs = currentTimeMs - e.deadlineMs(); + log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); + CompletableFuture f = e.future(); + f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); + }; + + // First, complete (exceptionally) any events that have passed their deadline. + completableEvents + .stream() + .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) + .forEach(completeEvent); + + // Second, remove any events that are already complete, just to make sure we don't hold references. + completableEvents.removeIf(e -> e.future().isDone()); + + log.trace("Finished reaping expired events"); + } + + /** + * It is possible for the {@link AsyncKafkaConsumer#close() consumer to close} before completing the processing of + * all the events in the queue. In this case, we need to {@link Future#cancel(boolean) cancel} any remaining events. + * + *

    + * + * Check each of the {@link #add(CompletableEvent) previously-added} {@link CompletableEvent completable events}, + * and for any that are incomplete, {@link Future#cancel(boolean) cancel} them. Also check the core event queue + * for any incomplete events and likewise cancel them. + * + *

    + * + * Note: because this is called in the context of {@link AsyncKafkaConsumer#close() closing consumer}, + * don't take the deadline into consideration, just close it regardless. + */ + public void reapIncomplete(BlockingQueue eventQueue) { + log.trace("Reaping incomplete events"); + + Consumer> completeEvent = e -> { + log.debug("Canceling event {} since the consumer is closing", e); + CompletableFuture f = e.future(); + f.cancel(true); + }; + + completableEvents + .stream() + .filter(e -> !e.future().isDone()) + .forEach(completeEvent); + completableEvents.clear(); + + LinkedList events = new LinkedList<>(); + eventQueue.drainTo(events); + events + .stream() + .filter(e -> e instanceof CompletableEvent) + .map(e -> (CompletableEvent) e) + .filter(e -> !e.future().isDone()) + .forEach(completeEvent); + + log.trace("Finished reaping incomplete events"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 159d8e44be656..c5c6f2132a90c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -16,20 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.internals.IdempotentCloser; -import org.apache.kafka.common.utils.LogContext; -import org.slf4j.Logger; - -import java.io.Closeable; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; -import java.util.Optional; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; /** * An {@link EventProcessor} is the means by which events produced by thread A are @@ -38,100 +25,7 @@ * communication channel is formed around {@link BlockingQueue a shared queue} into which thread A * enqueues events and thread B reads and processes those events. */ -public abstract class EventProcessor implements Closeable { - - protected final static java.util.function.Consumer> INCOMPLETE_EVENT_CANCELLER = e -> { - CompletableFuture f = e.future(); - f.cancel(true); - }; - - private final Logger log; - private final BlockingQueue eventQueue; - private final IdempotentCloser closer; - - protected EventProcessor(final LogContext logContext, final BlockingQueue eventQueue) { - this.log = logContext.logger(EventProcessor.class); - this.eventQueue = eventQueue; - this.closer = new IdempotentCloser(); - } - - protected abstract void process(T event); - - @Override - public void close() { - closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); - } - - protected interface ProcessHandler { - - void onProcess(T event, Optional error); - } - - /** - * Drains all available events from the queue, and then processes them in order. If any errors are thrown while - * processing the individual events, these are submitted to the given {@link ProcessHandler}. - */ - protected boolean process(ProcessHandler processHandler) { - closer.assertOpen("The processor was previously closed, so no further processing can occur"); - - List events = drain(); - - if (events.isEmpty()) { - log.trace("No events to process"); - return false; - } - - try { - log.trace("Starting processing of {} event{}", events.size(), events.size() == 1 ? "" : "s"); - - for (T event : events) { - try { - Objects.requireNonNull(event, "Attempted to process a null event"); - log.trace("Processing event: {}", event); - process(event); - processHandler.onProcess(event, Optional.empty()); - } catch (Throwable t) { - KafkaException error = ConsumerUtils.maybeWrapAsKafkaException(t); - processHandler.onProcess(event, Optional.of(error)); - } - } - } finally { - log.trace("Completed processing"); - } - - return true; - } - - /** - * It is possible for the {@link Consumer#close() consumer to close} before completing the processing of all - * the events in the queue. In this case, we need to - * {@link CompletableFuture#completeExceptionally(Throwable) (exceptionally) complete} any remaining events. - */ - protected void cancelIncompleteEvents() { - List events = drain(); - events - .stream() - .filter(e -> e instanceof CompletableEvent) - .map(e -> (CompletableEvent) e) - .filter(e -> !e.future().isDone()) - .forEach(INCOMPLETE_EVENT_CANCELLER); - } - - private void closeInternal() { - try { - log.debug("Removing unprocessed and/or unfinished events because the consumer is closing"); - cancelIncompleteEvents(); - } finally { - log.debug("Finished removal of events that were unprocessed and/or unfinished"); - } - } +public interface EventProcessor { - /** - * Moves all the events from the queue to the returned list. - */ - private List drain() { - LinkedList events = new LinkedList<>(); - eventQueue.drainTo(events); - return events; - } + void process(T event); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 5f7236e7aeca1..fb98dea926ae6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -270,7 +270,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA ); this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, - applicationEventQueue, requestManagers, metadata ) @@ -287,7 +286,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA @Override public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); - closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); } public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { @@ -303,6 +301,7 @@ public ConsumerNetworkThreadTestBuilder(Optional groupInfo) { this.consumerNetworkThread = new ConsumerNetworkThread( logContext, time, + applicationEventQueue, () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 134dc163f1575..83bc1b7a653e6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -36,7 +36,6 @@ import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -54,11 +53,9 @@ public class ApplicationEventProcessorTest { private MembershipManager membershipManager; @BeforeEach - @SuppressWarnings("unchecked") public void setup() { LogContext logContext = new LogContext(); ConsumerMetadata metadata = mock(ConsumerMetadata.class); - BlockingQueue applicationEventQueue = mock(BlockingQueue.class); OffsetsRequestManager offsetsRequestManager = mock(OffsetsRequestManager.class); TopicMetadataRequestManager topicMetadataRequestManager = mock(TopicMetadataRequestManager.class); FetchRequestManager fetchRequestManager = mock(FetchRequestManager.class); @@ -78,7 +75,6 @@ public void setup() { ); processor = new ApplicationEventProcessor( new LogContext(), - applicationEventQueue, requestManagers, metadata ); From 83cbf6f1b120add6704d881765b6bec235338e53 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:16:03 -0700 Subject: [PATCH 069/130] Updates --- .../internals/AsyncKafkaConsumer.java | 46 +++++++--------- .../internals/ConsumerNetworkThread.java | 54 ++++++------------- .../internals/MembershipManagerImpl.java | 10 +++- .../internals/events/CommitEvent.java | 14 +---- .../events/CompletableApplicationEvent.java | 13 ++--- .../events/CompletableBackgroundEvent.java | 19 +++++-- .../internals/events/CompletableEvent.java | 39 ++++++++++++++ .../events/CompletableEventReaper.java | 40 ++++++++++++-- ...rRebalanceListenerCallbackNeededEvent.java | 6 ++- .../internals/AsyncKafkaConsumerTest.java | 12 ++++- 10 files changed, 157 insertions(+), 96 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 1def7f7975685..844fd51096b9d 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 @@ -182,45 +182,34 @@ public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalance * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ - public boolean process() { + private boolean process() { AtomicReference firstError = new AtomicReference<>(); LinkedList events = new LinkedList<>(); backgroundEventQueue.drainTo(events); - if (events.isEmpty()) { - log.trace("No background events to process"); - return false; - } - - try { - log.trace("Starting processing of {} background event{}", events.size(), events.size() == 1 ? "" : "s"); - - for (BackgroundEvent event : events) { - try { - Objects.requireNonNull(event, "Attempted to process a null background event"); + for (BackgroundEvent event : events) { + try { + Objects.requireNonNull(event, "Attempted to process a null background event"); - if (event instanceof CompletableEvent) - backgroundEventReaper.add((CompletableEvent) event); + if (event instanceof CompletableEvent) + backgroundEventReaper.add((CompletableEvent) event); - log.trace("Processing background event: {}", event); - process(event); - } catch (Throwable t) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - if (!firstError.compareAndSet(null, e)) { - log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); - } - } + if (!firstError.compareAndSet(null, e)) + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); } - } finally { - log.trace("Completed processing background event(s)"); } + backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); + if (firstError.get() != null) throw firstError.get(); - return true; + return !events.isEmpty(); } @Override @@ -236,7 +225,6 @@ public void process(final BackgroundEvent event) { default: throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); - } } @@ -1674,8 +1662,12 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); - backgroundEventProcessor.process(); + try { + backgroundEventProcessor.process(); + } finally { + backgroundEventProcessor. + } // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as // in the previous implementation, because it will eventually involve group coordination // logic 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 92e0238994f55..54152b1a663b0 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 @@ -137,7 +137,22 @@ void runOnce() { // Process the events—if any—that were produced by the application thread. It is possible that when processing // an event generates an error. In such cases, the processor will log an exception, but we do not want those // errors to be propagated to the caller. - processEvents(); + LinkedList events = new LinkedList<>(); + applicationEventQueue.drainTo(events); + + for (ApplicationEvent event : events) { + try { + Objects.requireNonNull(event, "Attempted to process a null event"); + + if (event instanceof CompletableEvent) + applicationEventReaper.add((CompletableEvent) event); + + log.trace("Processing event: {}", event); + applicationEventProcessor.process(event); + } catch (Throwable t) { + log.warn("Error processing event {}", t.getMessage(), t); + } + } final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -158,7 +173,7 @@ void runOnce() { // thread has made at least one call to poll. This is done to emulate the behavior of the legacy consumer's // handling of timeouts. The legacy consumer makes at least one attempt to satisfy any network requests // before checking if a timeout has expired. - applicationEventReaper.reapExpired(currentTimeMs); + applicationEventReaper.reapExpiredAndCompleted(currentTimeMs); } /** @@ -194,41 +209,6 @@ static void runAtClose(final Collection> requ .forEach(networkClientDelegate::addAll); } - /** - * Drains all available events from the queue, and then processes them in order. If any errors are thrown while - * processing the individual events, these are logged and skipped. - */ - protected void processEvents() { - closer.assertOpen("The processor was previously closed, so no further processing can occur"); - - LinkedList events = new LinkedList<>(); - applicationEventQueue.drainTo(events); - - if (events.isEmpty()) { - log.trace("No events to process"); - } - - try { - log.trace("Starting processing of {} event{}", events.size(), events.size() == 1 ? "" : "s"); - - for (ApplicationEvent event : events) { - try { - Objects.requireNonNull(event, "Attempted to process a null event"); - - if (event instanceof CompletableEvent) - applicationEventReaper.add((CompletableEvent) event); - - log.trace("Processing event: {}", event); - applicationEventProcessor.process(event); - } catch (Throwable t) { - log.warn("Error processing event {}", t.getMessage(), t); - } - } - } finally { - log.trace("Completed processing"); - } - } - public boolean isRunning() { return running; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index c2bdd3f860991..64a72ce40b78c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -40,6 +40,7 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -1334,7 +1335,14 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + + // We don't yet have the concept of having an expiring callback, but we will likely want that eventually. + Timer timer = time.timer(Long.MAX_VALUE); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent( + methodName, + sortedPartitions, + timer + ); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 1da7b84039ab8..84e478d87918d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -32,26 +32,14 @@ public abstract class CommitEvent extends CompletableApplicationEvent { protected CommitEvent(final Type type, final Map offsets, final Timer timer) { super(type, timer); - this.offsets = validate(offsets); - } - - protected CommitEvent(final Type type, final Map offsets, final long deadlineMs) { - super(type, deadlineMs); - this.offsets = validate(offsets); - } - /** - * Validates the offsets are not negative and then returns the given offset map as - * {@link Collections#unmodifiableMap(Map) as unmodifiable}. - */ - private static Map validate(final Map offsets) { for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); } } - return Collections.unmodifiableMap(offsets); + this.offsets = Collections.unmodifiableMap(offsets); } public Map offsets() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index dae9e9f1017ba..9e688f7a78f78 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -36,13 +36,14 @@ protected CompletableApplicationEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); Objects.requireNonNull(timer); - this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); - } - protected CompletableApplicationEvent(final Type type, final long deadlineMs) { - super(type); - this.future = new CompletableFuture<>(); - this.deadlineMs = deadlineMs; + long currentTimeMs = timer.currentTimeMs(); + long remainingMs = timer.remainingMs(); + + if (currentTimeMs > Long.MAX_VALUE - remainingMs) + this.deadlineMs = Long.MAX_VALUE; + else + this.deadlineMs = currentTimeMs + remainingMs; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index ab444cada0f44..1e7b19248991f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -27,10 +30,20 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableBackgroundEvent(final Type type) { + protected CompletableBackgroundEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); + Objects.requireNonNull(timer); + + long currentTimeMs = timer.currentTimeMs(); + long remainingMs = timer.remainingMs(); + + if (currentTimeMs > Long.MAX_VALUE - remainingMs) + this.deadlineMs = Long.MAX_VALUE; + else + this.deadlineMs = currentTimeMs + remainingMs; } @Override @@ -40,11 +53,11 @@ public CompletableFuture future() { @Override public long deadlineMs() { - return Long.MAX_VALUE; + return deadlineMs; } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 56ddd07219913..7e3dd36e986e3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -17,10 +17,49 @@ package org.apache.kafka.clients.consumer.internals.events; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; +/** + * {@code CompletableEvent} is an interface that is used by both {@link CompletableApplicationEvent} and + * {@link CompletableBackgroundEvent} for common processing and logic. A {@code CompletableEvent} is one that + * allows the caller to get the {@link #future() future} related to the event and the event's + * {@link #deadlineMs() expiration timestamp}. + * + * @param Return type for the event when completed + */ public interface CompletableEvent { + /** + * Returns the {@link CompletableFuture future} associated with this event. Any event will have some related + * logic that is executed on its behalf. The event can complete in one of three ways: + * + *

      + *
    • + * Success: when the logic for the event completes successfully, the data generated by that event + * (if applicable) is passed to {@link CompletableFuture#complete(Object)}. In the case where the generic + * bound type is specified as {@link Void}, {@code null} is provided.
    • + *
    • + * Error: when the the event logic generates an error, the error is passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. + *
    • + *
    • + * Timeout: when the time spent executing the event logic exceeds the {@link #deadlineMs() deadline}, an + * instance of {@link TimeoutException} should be created and passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. + *
    • + *
    + * + * @return Future on which the caller may block or query for completion + */ CompletableFuture future(); + /** + * This is the deadline that represents the absolute wall clock time by which any event-specific execution should + * complete. This is not a timeout value. After this time has passed, + * {@link CompletableFuture#completeExceptionally(Throwable)} will be invoked with an instance of + * {@link TimeoutException}. + * + * @return + */ long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 1c0187b33f518..722675af3a067 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -24,11 +24,17 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; +import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.function.Consumer; +/** + * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed + * and making sure to reap them if they complete or pass their deadline. This is done so that we enforce an upper + * bound on the amount of time the event logic will execute. + */ public class CompletableEventReaper { private final Logger log; @@ -39,24 +45,47 @@ public CompletableEventReaper(LogContext logContext) { this.completableEvents = new ArrayList<>(); } + /** + * Adds a new {@link CompletableEvent event} to our list so that we can track it for later completion/expiration. + * + * @param event Event to track + */ public void add(CompletableEvent event) { - completableEvents.add(event); + completableEvents.add(Objects.requireNonNull(event)); } /** - * This method "completes" any {@link CompletableEvent}s that have expired. + * This method "completes" any {@link CompletableEvent}s that have either expired or completed normally. So this + * is a two-step process: + * + *
      + *
    1. + * For each tracked event which has exceeded its {@link CompletableEvent#deadlineMs() deadline}, an + * instance of {@link TimeoutException} is created and passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. + *
    2. + *
    3. + * For each tracked event of which its {@link CompletableEvent#future() future} is already in the + * * {@link CompletableFuture#isDone() done} state, it will be removed from the list of tracked events. + *
    4. + *
    + * + *

    + * + * This method should be called at regular intervals * * @param currentTimeMs Current time with which to compare against the * {@link CompletableEvent#deadlineMs() expiration time} */ - public void reapExpired(long currentTimeMs) { + public void reapExpiredAndCompleted(long currentTimeMs) { log.trace("Reaping expired events"); Consumer> completeEvent = e -> { + TimeoutException error = new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName())); long pastDueMs = currentTimeMs - e.deadlineMs(); log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); CompletableFuture f = e.future(); - f.completeExceptionally(new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName()))); + f.completeExceptionally(error); }; // First, complete (exceptionally) any events that have passed their deadline. @@ -65,7 +94,8 @@ public void reapExpired(long currentTimeMs) { .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) .forEach(completeEvent); - // Second, remove any events that are already complete, just to make sure we don't hold references. + // Second, remove any events that are already complete, just to make sure we don't hold references. This will + // include any events that finished successfully as well as any events we just completed exceptionally above. completableEvents.removeIf(e -> e.future().isDone()); log.trace("Finished reaping expired events"); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index ec101d9ee486b..86f13e6fa0a08 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.time.Duration; import java.util.Collections; @@ -38,8 +39,9 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(final ConsumerRebalanceListenerMethodName methodName, - final SortedSet partitions) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); + final SortedSet partitions, + final Timer timer) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } 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 2b655dc1f9f9f..bb08ad029c317 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 @@ -441,7 +441,11 @@ public void testCommitInRebalanceCallback() { doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( + ON_PARTITIONS_REVOKED, + sortedPartitions, + time.timer(Long.MAX_VALUE) + ); backgroundEventQueue.add(e); completeCommitSyncApplicationEventSuccessfully(); final AtomicBoolean callbackExecuted = new AtomicBoolean(false); @@ -1259,7 +1263,11 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( + methodName, + partitions, + time.timer(Long.MAX_VALUE) + ); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. From 339557f65f59641f3ec2cf6c1215461b2b463d0e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:16:21 -0700 Subject: [PATCH 070/130] Update AsyncKafkaConsumer.java --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 7 ++----- 1 file changed, 2 insertions(+), 5 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 844fd51096b9d..ce5a114325ae0 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 @@ -1663,11 +1663,8 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); - try { - backgroundEventProcessor.process(); - } finally { - backgroundEventProcessor. - } + backgroundEventProcessor.process(); + // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as // in the previous implementation, because it will eventually involve group coordination // logic From 22b03294e5a96529184c3da1ba1d7e699d0e270a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:22:40 -0700 Subject: [PATCH 071/130] Reverting code --- .../internals/AsyncKafkaConsumer.java | 56 +++++++++---------- .../internals/events/ListOffsetsEvent.java | 14 +++++ 2 files changed, 39 insertions(+), 31 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 ce5a114325ae0..05312e56508ed 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 @@ -100,7 +100,6 @@ import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -225,6 +224,7 @@ public void process(final BackgroundEvent event) { default: throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); + } } @@ -763,9 +763,9 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - Timer timer = time.timer(defaultApiTimeoutMs); - AsyncCommitEvent event = new AsyncCommitEvent(offsets, timer); - CompletableFuture future = commit(event); + Timer timer = time.timer(Long.MAX_VALUE); + AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets, timer); + CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { if (t == null) { @@ -989,10 +989,10 @@ public List partitionsFor(String topic, Duration timeout) { } final Timer timer = time.timer(timeout); - final TopicMetadataEvent event = new TopicMetadataEvent(topic, timer); - wakeupTrigger.setActiveTask(event.future()); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - Map> topicMetadata = applicationEventHandler.addAndGet(event); + Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1016,10 +1016,10 @@ public Map> listTopics(Duration timeout) { } final Timer timer = time.timer(timeout); - final AllTopicsMetadataEvent event = new AllTopicsMetadataEvent(timer); - wakeupTrigger.setActiveTask(event.future()); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timer); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(event); + return applicationEventHandler.addAndGet(topicMetadataEvent); } finally { wakeupTrigger.clearTask(); } @@ -1088,19 +1088,14 @@ public Map offsetsForTimes(Map offsetsByTimes = new HashMap<>(timestampsToSearch.size()); - for (Map.Entry entry : timestampsToSearch.entrySet()) - offsetsByTimes.put(entry.getKey(), null); - return offsetsByTimes; - } + if (timeout.toMillis() == 0L) + return listOffsetsEvent.emptyResult(); - return applicationEventHandler.addAndGet(event); + return applicationEventHandler.addAndGet(listOffsetsEvent); } finally { release(); } @@ -1142,11 +1137,11 @@ private Map beginningOrEndOffset(Collection timestamp)); Timer timer = time.timer(timeout); - ListOffsetsEvent event = new ListOffsetsEvent( + ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, false, timer); - Map offsetAndTimestampMap = applicationEventHandler.addAndGet(event); + Map offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); return offsetAndTimestampMap .entrySet() .stream() @@ -1354,11 +1349,11 @@ public void commitSync(Map offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - Timer timer = time.timer(timeout.toMillis()); - SyncCommitEvent event = new SyncCommitEvent(offsets, timer); - CompletableFuture future = commit(event); - wakeupTrigger.setActiveTask(future); - ConsumerUtils.getResult(future); + Timer requestTimer = time.timer(timeout.toMillis()); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, requestTimer); + CompletableFuture commitFuture = commit(syncCommitEvent); + wakeupTrigger.setActiveTask(commitFuture); + ConsumerUtils.getResult(commitFuture, requestTimer); interceptors.onCommit(offsets); } finally { wakeupTrigger.clearTask(); @@ -1469,13 +1464,13 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.get().isPresent()) { - Timer timer = time.timer(defaultApiTimeoutMs); - UnsubscribeEvent event = new UnsubscribeEvent(timer); - applicationEventHandler.add(event); + Timer timer = time.timer(Long.MAX_VALUE); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(timer); + applicationEventHandler.add(unsubscribeEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); try { - processBackgroundEvents(event.future(), timer); + processBackgroundEvents(unsubscribeEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1662,7 +1657,6 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); - backgroundEventProcessor.process(); // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 7593d547d3bbb..34da6b2b3a58a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.utils.Timer; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -42,6 +43,19 @@ public ListOffsetsEvent(final Map timestampToSearch, final this.requireTimestamps = requireTimestamps; } + /** + * Build result representing that no offsets were found as part of the current event. + * + * @return Map containing all the partitions the event was trying to get offsets for, and + * null {@link OffsetAndTimestamp} as value + */ + public Map emptyResult() { + HashMap offsetsByTimes = new HashMap<>(timestampsToSearch.size()); + for (Map.Entry entry : timestampsToSearch.entrySet()) + offsetsByTimes.put(entry.getKey(), null); + return offsetsByTimes; + } + public Map timestampsToSearch() { return timestampsToSearch; } From 02ccdc6283184f96e39ff832cfb09f65f32eca7c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:25:04 -0700 Subject: [PATCH 072/130] Update AsyncKafkaConsumer.java --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 05312e56508ed..6ae0a553cd34f 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 @@ -1088,7 +1088,10 @@ public Map offsetsForTimes(Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; From 223aadd595eb21b37a63ea6966540990c3c50e84 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:29:07 -0700 Subject: [PATCH 073/130] More general cleanup --- .../internals/AsyncKafkaConsumer.java | 74 +++++++++---------- .../internals/ConsumerNetworkThread.java | 3 - 2 files changed, 36 insertions(+), 41 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 6ae0a553cd34f..5feb8233afaca 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 @@ -175,42 +175,6 @@ public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalance this.rebalanceListenerInvoker = rebalanceListenerInvoker; } - /** - * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. - * It is possible that {@link ErrorEvent an error} - * could occur when processing the events. In such cases, the processor will take a reference to the first - * error, continue to process the remaining events, and then throw the first error that occurred. - */ - private boolean process() { - AtomicReference firstError = new AtomicReference<>(); - - LinkedList events = new LinkedList<>(); - backgroundEventQueue.drainTo(events); - - for (BackgroundEvent event : events) { - try { - Objects.requireNonNull(event, "Attempted to process a null background event"); - - if (event instanceof CompletableEvent) - backgroundEventReaper.add((CompletableEvent) event); - - process(event); - } catch (Throwable t) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - - if (!firstError.compareAndSet(null, e)) - log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); - } - } - - backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); - - if (firstError.get() != null) - throw firstError.get(); - - return !events.isEmpty(); - } - @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -1663,7 +1627,7 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); - backgroundEventProcessor.process(); + processBackgroundEvents(); // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as // in the previous implementation, because it will eventually involve group coordination @@ -1792,6 +1756,40 @@ private void subscribeInternal(Collection topics, Optional firstError = new AtomicReference<>(); + + LinkedList events = new LinkedList<>(); + backgroundEventQueue.drainTo(events); + + for (BackgroundEvent event : events) { + try { + if (event instanceof CompletableEvent) + backgroundEventReaper.add((CompletableEvent) event); + + backgroundEventProcessor.process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + + if (!firstError.compareAndSet(null, e)) + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); + } + } + + backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); + + if (firstError.get() != null) + throw firstError.get(); + + return !events.isEmpty(); + } + /** * This method can be used by cases where the caller has an event that needs to both block for completion but * also process background events. For some events, in order to fully process the associated logic, the @@ -1834,7 +1832,7 @@ T processBackgroundEvents(Future future, Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = backgroundEventProcessor.process(); + boolean hadEvents = processBackgroundEvents(); try { if (future.isDone()) { 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 54152b1a663b0..52e9c86cc2718 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 @@ -142,12 +142,9 @@ void runOnce() { for (ApplicationEvent event : events) { try { - Objects.requireNonNull(event, "Attempted to process a null event"); - if (event instanceof CompletableEvent) applicationEventReaper.add((CompletableEvent) event); - log.trace("Processing event: {}", event); applicationEventProcessor.process(event); } catch (Throwable t) { log.warn("Error processing event {}", t.getMessage(), t); From 001fbe90a09a51277d494d6ffaadf6f778a9bd58 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:34:50 -0700 Subject: [PATCH 074/130] Reverting unnecessary changes --- .../internals/events/ApplicationEventProcessor.java | 12 ++++-------- .../internals/events/CompletableEventReaper.java | 2 +- ...ConsumerRebalanceListenerCallbackNeededEvent.java | 4 +++- .../consumer/internals/events/ListOffsetsEvent.java | 1 + .../consumer/internals/events/UnsubscribeEvent.java | 1 + 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 820911856117b..f3fcdc46bcd2e 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 @@ -164,10 +164,7 @@ private void process(final FetchCommittedOffsetsEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture> future = manager.fetchOffsets( - event.partitions(), - event.deadlineMs() - ); + CompletableFuture> future = manager.fetchOffsets(event.partitions(), event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -189,10 +186,9 @@ private void process(final AssignmentChangeEvent event) { } private void process(final ListOffsetsEvent event) { - final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets( - event.timestampsToSearch(), - event.requireTimestamps() - ); + final CompletableFuture> future = + requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), + event.requireTimestamps()); future.whenComplete(complete(event.future())); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 722675af3a067..20ee3d694a779 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -66,7 +66,7 @@ public void add(CompletableEvent event) { * *

  • * For each tracked event of which its {@link CompletableEvent#future() future} is already in the - * * {@link CompletableFuture#isDone() done} state, it will be removed from the list of tracked events. + * {@link CompletableFuture#isDone() done} state, it will be removed from the list of tracked events. *
  • * * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 86f13e6fa0a08..2165d621a21eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -56,6 +56,8 @@ public SortedSet partitions() { @Override protected String toStringBase() { - return super.toStringBase() + ", methodName=" + methodName + ", partitions=" + partitions; + return super.toStringBase() + + ", methodName=" + methodName + + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 34da6b2b3a58a..e218705846e19 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -70,4 +70,5 @@ public String toStringBase() { ", timestampsToSearch=" + timestampsToSearch + ", requireTimestamps=" + requireTimestamps; } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index 3962c3fa1317e..0b988370014a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -32,3 +32,4 @@ public UnsubscribeEvent(final Timer timer) { super(Type.UNSUBSCRIBE, timer); } } + From 9f1dfbe72fe3dcae7b12bb96149df1f8f0a8dc0c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:41:59 -0700 Subject: [PATCH 075/130] Revert unnecessary changes --- .../internals/ConsumerNetworkThread.java | 7 +++-- .../internals/events/CommitEvent.java | 9 +++++- .../internals/ConsumerTestBuilder.java | 2 +- .../internals/FetchRequestManagerTest.java | 7 +++-- .../HeartbeatRequestManagerTest.java | 5 ++-- .../events/ApplicationEventProcessorTest.java | 29 ++++++++++++++----- 6 files changed, 40 insertions(+), 19 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 52e9c86cc2718..f68b92a202c54 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 @@ -197,7 +197,8 @@ void runOnce() { */ // Visible for testing static void runAtClose(final Collection> requestManagers, - final NetworkClientDelegate networkClientDelegate) { + final NetworkClientDelegate networkClientDelegate, + final Timer timer) { // These are the optional outgoing requests at the requestManagers.stream() .filter(Optional::isPresent) @@ -292,12 +293,12 @@ private void sendUnsentRequests(final Timer timer) { void cleanup() { log.trace("Closing the consumer network thread"); + Timer timer = time.timer(closeTimeout); try { - runAtClose(requestManagers.entries(), networkClientDelegate); + runAtClose(requestManagers.entries(), networkClientDelegate, timer); } catch (Exception e) { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { - Timer timer = time.timer(closeTimeout); sendUnsentRequests(timer); applicationEventReaper.reapIncomplete(applicationEventQueue); closeQuietly(requestManagers, "request managers"); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 84e478d87918d..889ab60de0ee0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -32,14 +32,21 @@ public abstract class CommitEvent extends CompletableApplicationEvent { protected CommitEvent(final Type type, final Map offsets, final Timer timer) { super(type, timer); + this.offsets = validate(offsets); + } + /** + * Validates the offsets are not negative and then returns the given offset map as + * {@link Collections#unmodifiableMap(Map) as unmodifiable}. + */ + private static Map validate(final Map offsets) { for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); } } - this.offsets = Collections.unmodifiableMap(offsets); + return Collections.unmodifiableMap(offsets); } public Map offsets() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index fb98dea926ae6..4ac402972c926 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -141,7 +141,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.fetchConfig = new FetchConfig(config); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.metrics = createMetrics(config, time); this.subscriptions = spy(createSubscriptionState(config, logContext)); 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 ed0aeef85d8d1..b01e11f452262 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 @@ -177,7 +177,7 @@ public class FetchRequestManagerTest { private int maxWaitMs = 0; private int fetchSize = 1000; private long retryBackoffMs = 100; - private int requestTimeoutMs = 30000; + private long requestTimeoutMs = 30000; private MockTime time = new MockTime(1); private SubscriptionState subscriptions; private ConsumerMetadata metadata; @@ -365,10 +365,11 @@ public void testFetcherCloseClosesFetchSessionsInBroker() { final ArgumentCaptor argument = ArgumentCaptor.forClass(NetworkClientDelegate.UnsentRequest.class); + Timer timer = time.timer(Duration.ofSeconds(10)); // NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling // the close() method with a Timer will NOT send out the close session requests on close. The network // I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here. - ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate); + ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer); // the network is polled during the last state of clean up. networkClientDelegate.poll(time.timer(1)); // validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the @@ -3591,7 +3592,7 @@ private void buildFetcher(MetricConfig metricConfig, metadata, time, retryBackoffMs, - requestTimeoutMs, + (int) requestTimeoutMs, Integer.MAX_VALUE); offsetFetcher = new OffsetFetcher(logContext, consumerNetworkClient, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 9c59ca41e9af7..8e05e505be471 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -66,7 +66,6 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MAX_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MS; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -177,7 +176,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); - assertInstanceOf(ConsumerGroupHeartbeatRequest.Builder.class, request.requestBuilder()); + assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); ConsumerGroupHeartbeatRequest heartbeatRequest = (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); @@ -312,7 +311,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); - assertInstanceOf(ConsumerGroupHeartbeatRequest.Builder.class, request.requestBuilder()); + assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); ConsumerGroupHeartbeatRequest heartbeatRequest = (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 83bc1b7a653e6..f7f798c38fa08 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -36,6 +36,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -45,9 +46,17 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { - private final Time time = new MockTime(1); private ApplicationEventProcessor processor; + private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); + private RequestManagers requestManagers; + private ConsumerMetadata metadata = mock(ConsumerMetadata.class); + private NetworkClientDelegate networkClientDelegate = mock(NetworkClientDelegate.class); + private OffsetsRequestManager offsetRequestManager; + private OffsetsRequestManager offsetsRequestManager; + private TopicMetadataRequestManager topicMetadataRequestManager; + private FetchRequestManager fetchRequestManager; + private CoordinatorRequestManager coordinatorRequestManager; private CommitRequestManager commitRequestManager; private HeartbeatRequestManager heartbeatRequestManager; private MembershipManager membershipManager; @@ -55,11 +64,11 @@ public class ApplicationEventProcessorTest { @BeforeEach public void setup() { LogContext logContext = new LogContext(); - ConsumerMetadata metadata = mock(ConsumerMetadata.class); - OffsetsRequestManager offsetsRequestManager = mock(OffsetsRequestManager.class); - TopicMetadataRequestManager topicMetadataRequestManager = mock(TopicMetadataRequestManager.class); - FetchRequestManager fetchRequestManager = mock(FetchRequestManager.class); - CoordinatorRequestManager coordinatorRequestManager = mock(CoordinatorRequestManager.class); + offsetRequestManager = mock(OffsetsRequestManager.class); + offsetsRequestManager = mock(OffsetsRequestManager.class); + topicMetadataRequestManager = mock(TopicMetadataRequestManager.class); + fetchRequestManager = mock(FetchRequestManager.class); + coordinatorRequestManager = mock(CoordinatorRequestManager.class); commitRequestManager = mock(CommitRequestManager.class); heartbeatRequestManager = mock(HeartbeatRequestManager.class); membershipManager = mock(MembershipManager.class); @@ -82,7 +91,7 @@ public void setup() { @Test public void testPrepClosingCommitEvents() { - List results = Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class)); + List results = mockCommitResults(); doReturn(new NetworkClientDelegate.PollResult(100, results)).when(commitRequestManager).pollOnClose(); processor.process(new CommitOnCloseEvent()); verify(commitRequestManager).signalClose(); @@ -90,7 +99,7 @@ public void testPrepClosingCommitEvents() { @Test public void testPrepClosingLeaveGroupEvent() { - Timer timer = time.timer(Long.MAX_VALUE); + Timer timer = time.timer(100); LeaveOnCloseEvent event = new LeaveOnCloseEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); @@ -98,4 +107,8 @@ public void testPrepClosingLeaveGroupEvent() { verify(membershipManager).leaveGroup(); assertTrue(event.future().isDone()); } + + private List mockCommitResults() { + return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class)); + } } From b055cef05627b072c1f2853545d13cc8b4d3b10e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:45:04 -0700 Subject: [PATCH 076/130] Updates to remove unnecessary changes --- .../internals/ConsumerNetworkThreadTest.java | 18 ++++++++++++++---- .../events/ApplicationEventProcessorTest.java | 3 ++- 2 files changed, 16 insertions(+), 5 deletions(-) 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 ed0561d323130..821cd8eb6cd30 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 @@ -86,8 +86,11 @@ public class ConsumerNetworkThreadTest { private OffsetsRequestManager offsetsRequestManager; private CommitRequestManager commitRequestManager; private CoordinatorRequestManager coordinatorRequestManager; + private HeartbeatRequestManager heartbeatRequestManager; + private MembershipManager memberhipsManager; private ConsumerNetworkThread consumerNetworkThread; private MockClient client; + private SubscriptionState subscriptions; @BeforeEach public void setup() { @@ -101,7 +104,10 @@ public void setup() { commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); offsetsRequestManager = testBuilder.offsetsRequestManager; coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); + heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new); + memberhipsManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); consumerNetworkThread = testBuilder.consumerNetworkThread; + subscriptions = testBuilder.subscriptions; consumerNetworkThread.initializeResources(); } @@ -167,7 +173,8 @@ public void testSyncCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsEvent(timestamps, true, time.timer(1000)); + Timer timer = time.timer(100); + ApplicationEvent e = new ListOffsetsEvent(timestamps, true, timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); @@ -176,7 +183,8 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - ApplicationEvent e = new ResetPositionsEvent(time.timer(1000)); + Timer timer = time.timer(100); + ResetPositionsEvent e = new ResetPositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); @@ -187,7 +195,8 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ApplicationEvent event = new ResetPositionsEvent(time.timer(1000)); + Timer timer = time.timer(100); + ResetPositionsEvent event = new ResetPositionsEvent(timer); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -221,7 +230,8 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataEvent("topic", time.timer(Long.MAX_VALUE))); + Timer timer = time.timer(Long.MAX_VALUE); + applicationEventsQueue.add(new TopicMetadataEvent("topic", timer)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index f7f798c38fa08..57e6351b74682 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -62,6 +62,7 @@ public class ApplicationEventProcessorTest { private MembershipManager membershipManager; @BeforeEach + @SuppressWarnings("unchecked") public void setup() { LogContext logContext = new LogContext(); offsetRequestManager = mock(OffsetsRequestManager.class); @@ -72,7 +73,7 @@ public void setup() { commitRequestManager = mock(CommitRequestManager.class); heartbeatRequestManager = mock(HeartbeatRequestManager.class); membershipManager = mock(MembershipManager.class); - RequestManagers requestManagers = new RequestManagers( + requestManagers = new RequestManagers( logContext, offsetsRequestManager, topicMetadataRequestManager, From cb75d3f62dc4024babfba00dc8c2eecb31d252b2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:51:06 -0700 Subject: [PATCH 077/130] Revert unnecessary changes --- .../internals/AsyncKafkaConsumerTest.java | 39 ++++++++++--------- .../internals/ConsumerNetworkThreadTest.java | 3 +- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index bb08ad029c317..d415a280a67a7 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 @@ -29,6 +29,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.RoundRobinAssignor; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; @@ -140,7 +141,12 @@ @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { + private long retryBackoffMs = 100L; + private int defaultApiTimeoutMs = 1000; + private boolean autoCommitEnabled = true; + private AsyncKafkaConsumer consumer = null; + private final Time time = new MockTime(1); private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); @@ -369,7 +375,7 @@ public void testCommittedExceptionThrown() { @Test public void testWakeupBeforeCallingPoll() { consumer = newConsumer(); - completeCommitSyncApplicationEventSuccessfully(); +// completeCommitSyncApplicationEventSuccessfully(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); @@ -441,11 +447,7 @@ public void testCommitInRebalanceCallback() { doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( - ON_PARTITIONS_REVOKED, - sortedPartitions, - time.timer(Long.MAX_VALUE) - ); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions, time.timer(Long.MAX_VALUE)); backgroundEventQueue.add(e); completeCommitSyncApplicationEventSuccessfully(); final AtomicBoolean callbackExecuted = new AtomicBoolean(false); @@ -708,13 +710,15 @@ public void testCompleteQuietly() { AtomicReference exception = new AtomicReference<>(); CompletableFuture future = CompletableFuture.completedFuture(null); consumer = newConsumer(); - assertDoesNotThrow(() -> consumer.completeQuietly(() -> future.get(0, TimeUnit.MILLISECONDS), "test", exception)); + assertDoesNotThrow(() -> consumer.completeQuietly(() -> { + future.get(0, TimeUnit.MILLISECONDS); + }, "test", exception)); assertNull(exception.get()); assertDoesNotThrow(() -> consumer.completeQuietly(() -> { throw new KafkaException("Test exception"); }, "test", exception)); - assertInstanceOf(KafkaException.class, exception.get()); + assertTrue(exception.get() instanceof KafkaException); } @Test @@ -831,7 +835,8 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Set partitions = mockTopicPartitionOffset().keySet(); Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); - doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(ListOffsetsEvent.class)); + doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( + any(ListOffsetsEvent.class)); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); @@ -936,11 +941,12 @@ public void testNoWakeupInCloseCommit() { AtomicReference capturedEvent = new AtomicReference<>(); doAnswer(invocation -> { - SyncCommitEvent event = invocation.getArgument(0); - capturedEvent.set(event); - event.future().complete(null); + ApplicationEvent event = invocation.getArgument(0); + if (event instanceof SyncCommitEvent) { + capturedEvent.set((SyncCommitEvent) event); + } return null; - }).when(applicationEventHandler).add(any(SyncCommitEvent.class)); + }).when(applicationEventHandler).add(any()); consumer.close(Duration.ZERO); @@ -1263,11 +1269,7 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent( - methodName, - partitions, - time.timer(Long.MAX_VALUE) - ); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions, time.timer(Long.MAX_VALUE)); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. @@ -1685,3 +1687,4 @@ private void forceCommitCallbackInvocation() { consumer.commitAsync(); } } + 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 821cd8eb6cd30..21a1544db26ef 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 @@ -205,7 +205,8 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ApplicationEvent e = new ValidatePositionsEvent(time.timer(1000)); + Timer timer = time.timer(100); + ValidatePositionsEvent e = new ValidatePositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); From e893f0f75497cb7722e2aaf8f8c1d9ee5b87c3b3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Mar 2024 15:54:27 -0700 Subject: [PATCH 078/130] More cleanup --- .../kafka/clients/consumer/internals/events/ErrorEvent.java | 3 --- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 1 - 2 files changed, 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java index 49af6d2bd447b..5e6d822382348 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java @@ -18,15 +18,12 @@ import org.apache.kafka.common.KafkaException; -import java.util.Objects; - public class ErrorEvent extends BackgroundEvent { private final RuntimeException error; public ErrorEvent(Throwable t) { super(Type.ERROR); - Objects.requireNonNull(t); this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); } 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 d415a280a67a7..7307844e2c710 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 @@ -375,7 +375,6 @@ public void testCommittedExceptionThrown() { @Test public void testWakeupBeforeCallingPoll() { consumer = newConsumer(); -// completeCommitSyncApplicationEventSuccessfully(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); From 53a4ae24499a79a1c77b01e6ba2c984322eb9e1b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 12 Mar 2024 17:22:50 -0700 Subject: [PATCH 079/130] Comments changes --- .../consumer/internals/events/CompletableEventReaper.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 20ee3d694a779..eb2a47f56b217 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -31,8 +31,8 @@ import java.util.function.Consumer; /** - * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed - * and making sure to reap them if they complete or pass their deadline. This is done so that we enforce an upper + * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed, + * making sure to reap them if they complete normally or pass their deadline. This is done so that we enforce an upper * bound on the amount of time the event logic will execute. */ public class CompletableEventReaper { @@ -72,7 +72,7 @@ public void add(CompletableEvent event) { * *

    * - * This method should be called at regular intervals + * This method should be called at regular intervals, based upon the needs of the resource that owns the reaper. * * @param currentTimeMs Current time with which to compare against the * {@link CompletableEvent#deadlineMs() expiration time} From 968a535482c323da8cbf23bfa41d1dd1ad5c0dfc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 1 Apr 2024 16:49:59 -0700 Subject: [PATCH 080/130] Updates to include more unit tests --- .../internals/AsyncKafkaConsumer.java | 30 ++- .../internals/ConsumerNetworkThread.java | 24 +- .../internals/events/CompletableEvent.java | 15 +- .../events/CompletableEventReaper.java | 60 +++-- .../events/CompletableEventReaperTest.java | 210 ++++++++++++++++++ 5 files changed, 301 insertions(+), 38 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.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 5feb8233afaca..48dd4e70dd9c5 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 @@ -47,7 +47,7 @@ import org.apache.kafka.clients.consumer.internals.events.CommitEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; @@ -215,7 +215,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final String clientId; private final BlockingQueue backgroundEventQueue; private final BackgroundEventProcessor backgroundEventProcessor; - private final CompletableEventReaper backgroundEventReaper; + private final CompletableEventReaper> backgroundEventReaper; private final Deserializers deserializers; /** @@ -365,7 +365,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.backgroundEventProcessor = new BackgroundEventProcessor( rebalanceListenerInvoker ); - this.backgroundEventReaper = new CompletableEventReaper(logContext); + this.backgroundEventReaper = new CompletableEventReaper<>(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -429,7 +429,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.time = time; this.backgroundEventQueue = backgroundEventQueue; this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); - this.backgroundEventReaper = new CompletableEventReaper(logContext); + this.backgroundEventReaper = new CompletableEventReaper<>(logContext); this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; @@ -538,7 +538,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { networkClientDelegateSupplier, requestManagersSupplier); this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); - this.backgroundEventReaper = new CompletableEventReaper(logContext); + this.backgroundEventReaper = new CompletableEventReaper<>(logContext); } // auxiliary interface for testing @@ -1207,8 +1207,20 @@ private void close(Duration timeout, boolean swallowException) { swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", this::maybeInvokeCommitCallbacks, firstException); closeTimer.update(); - if (backgroundEventReaper != null && backgroundEventQueue != null) - backgroundEventReaper.reapIncomplete(backgroundEventQueue); + + if (backgroundEventReaper != null && backgroundEventQueue != null) { + // Copy over the completable events to a separate list, then reap any incomplete + // events on that list. + LinkedList allEvents = new LinkedList<>(); + backgroundEventQueue.drainTo(allEvents); + List> completableEvents = allEvents + .stream() + .filter(e -> e instanceof CompletableBackgroundEvent) + .map(e -> (CompletableBackgroundEvent) e) + .collect(Collectors.toList()); + backgroundEventReaper.reapIncomplete(completableEvents); + } + closeQuietly(interceptors, "consumer interceptors", firstException); closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); closeQuietly(metrics, "consumer metrics", firstException); @@ -1770,8 +1782,8 @@ private boolean processBackgroundEvents() { for (BackgroundEvent event : events) { try { - if (event instanceof CompletableEvent) - backgroundEventReaper.add((CompletableEvent) event); + if (event instanceof CompletableBackgroundEvent) + backgroundEventReaper.add((CompletableBackgroundEvent) event); backgroundEventProcessor.process(event); } catch (Throwable t) { 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 f68b92a202c54..27b99d4136c28 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,7 +20,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; @@ -39,6 +39,7 @@ import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; import static org.apache.kafka.common.utils.Utils.closeQuietly; @@ -55,7 +56,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private final Time time; private final Logger log; private final BlockingQueue applicationEventQueue; - private final CompletableEventReaper applicationEventReaper; + private final CompletableEventReaper> applicationEventReaper; private final Supplier applicationEventProcessorSupplier; private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; @@ -77,7 +78,7 @@ public ConsumerNetworkThread(LogContext logContext, this.time = time; this.log = logContext.logger(getClass()); this.applicationEventQueue = applicationEventQueue; - this.applicationEventReaper = new CompletableEventReaper(logContext); + this.applicationEventReaper = new CompletableEventReaper<>(logContext); this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; this.networkClientDelegateSupplier = networkClientDelegateSupplier; this.requestManagersSupplier = requestManagersSupplier; @@ -142,8 +143,8 @@ void runOnce() { for (ApplicationEvent event : events) { try { - if (event instanceof CompletableEvent) - applicationEventReaper.add((CompletableEvent) event); + if (event instanceof CompletableApplicationEvent) + applicationEventReaper.add((CompletableApplicationEvent) event); applicationEventProcessor.process(event); } catch (Throwable t) { @@ -300,7 +301,18 @@ void cleanup() { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { sendUnsentRequests(timer); - applicationEventReaper.reapIncomplete(applicationEventQueue); + + // Copy over the completable events to a separate list, then reap any incomplete + // events on that list. + LinkedList allEvents = new LinkedList<>(); + applicationEventQueue.drainTo(allEvents); + List> completableEvents = allEvents + .stream() + .filter(e -> e instanceof CompletableApplicationEvent) + .map(e -> (CompletableApplicationEvent) e) + .collect(Collectors.toList()); + applicationEventReaper.reapIncomplete(completableEvents); + closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); log.debug("Closed the consumer network thread"); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 7e3dd36e986e3..97a16759d9495 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import java.util.concurrent.TimeoutException; /** @@ -31,7 +33,7 @@ public interface CompletableEvent { /** * Returns the {@link CompletableFuture future} associated with this event. Any event will have some related - * logic that is executed on its behalf. The event can complete in one of three ways: + * logic that is executed on its behalf. The event can complete in one of the following ways: * *

      *
    • @@ -47,9 +49,16 @@ public interface CompletableEvent { * instance of {@link TimeoutException} should be created and passed to * {@link CompletableFuture#completeExceptionally(Throwable)}. *
    • + *
    • + * Cancelled: when an event remains incomplete when the consumer closes, the future will be + * {@link CompletableFuture#cancel(boolean) cancelled}. Attempts to {@link Future#get() get the result} + * of the processing will throw a {@link CancellationException}. + *
    • *
    * * @return Future on which the caller may block or query for completion + * + * @see CompletableEventReaper */ CompletableFuture future(); @@ -59,7 +68,9 @@ public interface CompletableEvent { * {@link CompletableFuture#completeExceptionally(Throwable)} will be invoked with an instance of * {@link TimeoutException}. * - * @return + * @return Absolute time for event to be completed + * + * @see CompletableEventReaper */ long deadlineMs(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index eb2a47f56b217..b2204cdcbd4ba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -22,36 +22,45 @@ import org.slf4j.Logger; import java.util.ArrayList; -import java.util.LinkedList; +import java.util.Collection; import java.util.List; import java.util.Objects; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.function.Consumer; +import java.util.function.Predicate; /** * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed, * making sure to reap them if they complete normally or pass their deadline. This is done so that we enforce an upper * bound on the amount of time the event logic will execute. */ -public class CompletableEventReaper { +public class CompletableEventReaper> { private final Logger log; - private final List> completableEvents; + + /** + * List of tracked events that we are candidates to expire or cancel when reviewed. + */ + private final List tracked; + + /** + * {@link Predicate} to check if a {@link Future} is {@link Future#isDone() done}. + */ + private final Predicate doneFilter = e -> e.future().isDone(); public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); - this.completableEvents = new ArrayList<>(); + this.tracked = new ArrayList<>(); } /** - * Adds a new {@link CompletableEvent event} to our list so that we can track it for later completion/expiration. + * Adds a new {@link CompletableEvent event} to track for later completion/expiration. * * @param event Event to track */ - public void add(CompletableEvent event) { - completableEvents.add(Objects.requireNonNull(event)); + public void add(T event) { + tracked.add(Objects.requireNonNull(event, "Event to track must be non-null")); } /** @@ -88,15 +97,16 @@ public void reapExpiredAndCompleted(long currentTimeMs) { f.completeExceptionally(error); }; - // First, complete (exceptionally) any events that have passed their deadline. - completableEvents + // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. + tracked .stream() - .filter(e -> !e.future().isDone() && currentTimeMs > e.deadlineMs()) + .filter(not(doneFilter)) + .filter(e -> currentTimeMs > e.deadlineMs()) .forEach(completeEvent); // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. - completableEvents.removeIf(e -> e.future().isDone()); + tracked.removeIf(doneFilter); log.trace("Finished reaping expired events"); } @@ -115,31 +125,39 @@ public void reapExpiredAndCompleted(long currentTimeMs) { * * Note: because this is called in the context of {@link AsyncKafkaConsumer#close() closing consumer}, * don't take the deadline into consideration, just close it regardless. + * + * @param events Events from a queue that have not yet been tracked that also need to be reviewed */ - public void reapIncomplete(BlockingQueue eventQueue) { + public void reapIncomplete(Collection events) { log.trace("Reaping incomplete events"); + Objects.requireNonNull(events, "Event queue to reap must be non-null"); + Consumer> completeEvent = e -> { log.debug("Canceling event {} since the consumer is closing", e); CompletableFuture f = e.future(); f.cancel(true); }; - completableEvents + tracked .stream() - .filter(e -> !e.future().isDone()) + .filter(not(doneFilter)) .forEach(completeEvent); - completableEvents.clear(); + tracked.clear(); - LinkedList events = new LinkedList<>(); - eventQueue.drainTo(events); events .stream() - .filter(e -> e instanceof CompletableEvent) - .map(e -> (CompletableEvent) e) - .filter(e -> !e.future().isDone()) + .filter(not(doneFilter)) .forEach(completeEvent); log.trace("Finished reaping incomplete events"); } + + int size() { + return tracked.size(); + } + + private Predicate not(Predicate p) { + return p.negate(); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java new file mode 100644 index 0000000000000..c9a8529766af9 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CancellationException; +import java.util.concurrent.LinkedBlockingQueue; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CompletableEventReaperTest { + + private final LogContext logContext = new LogContext(); + private final Time time = new MockTime(0, 0, 0); + private final CompletableEventReaper> reaper = new CompletableEventReaper<>(logContext); + + @Test + public void testExpired() { + // Add a new event to the reaper. + Timer timer = time.timer(100); + UnsubscribeEvent event = new UnsubscribeEvent(timer); + reaper.add(event); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // Sleep for at least 1 ms. *more* than the timeout so that the event is considered expired. + time.sleep(timer.timeoutMs() + 1); + timer.update(time.milliseconds()); + assertEquals(0, timer.remainingMs()); + + // However, until we actually invoke the reaper, the event isn't complete and is still being tracked. + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // Call the reaper and validate that the event is now "done" (expired), the correct exception type is + // thrown, and the event is no longer tracked. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertTrue(event.future().isDone()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testCompleted() { + // Add a new event to the reaper. + Timer timer = time.timer(100); + UnsubscribeEvent event = new UnsubscribeEvent(timer); + reaper.add(event); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // We'll cause the event to be completed normally. Note that because we haven't called the reaper, the + // event is still being tracked. + event.future().complete(null); + assertTrue(event.future().isDone()); + assertEquals(1, reaper.size()); + + // To ensure we don't accidentally expire an event that completed normally, sleep past the timeout. + time.sleep(timer.timeoutMs() + 1); + timer.update(time.milliseconds()); + assertEquals(0, timer.remainingMs()); + + // Call the reaper and validate that the event is not considered expired, but is still no longer tracked. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertTrue(event.future().isDone()); + assertNull(ConsumerUtils.getResult(event.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testCompletedAndExpired() { + // Add two events to the reaper. One event will be completed, the other we will let expire. + Timer timer = time.timer(100); + UnsubscribeEvent event1 = new UnsubscribeEvent(timer); + UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + reaper.add(event1); + reaper.add(event2); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertFalse(event1.future().isDone()); + assertFalse(event2.future().isDone()); + assertEquals(2, reaper.size()); + + // We'll cause the first event to be completed normally, but then sleep past the timer deadline. + event1.future().complete(null); + assertTrue(event1.future().isDone()); + + time.sleep(timer.timeoutMs() + 1); + timer.update(time.milliseconds()); + assertEquals(0, timer.remainingMs()); + + // Though the first event is completed, it's still being tracked, along with the second expired event. + assertEquals(2, reaper.size()); + + // Validate that the first (completed) event is not expired, but the second one is expired. In either case, + // both should be completed and neither should be tracked anymore. + reaper.reapExpiredAndCompleted(time.milliseconds()); + assertTrue(event1.future().isDone()); + assertTrue(event2.future().isDone()); + assertNull(ConsumerUtils.getResult(event1.future())); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testIncompleteQueue() { + // Add two events to the queue. + BlockingQueue> queue = new LinkedBlockingQueue<>(); + Timer timer = time.timer(100); + UnsubscribeEvent event1 = new UnsubscribeEvent(timer); + UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + queue.add(event1); + queue.add(event2); + + // Complete one of our events, just to make sure it isn't inadvertently canceled. + event1.future().complete(null); + + // In this test, our events aren't tracked in the reaper, just in the queue. + assertEquals(0, reaper.size()); + assertEquals(2, queue.size()); + + // Go ahead and reap the incomplete from the queue. + reaper.reapIncomplete(queue); + + // The first event was completed, so we didn't cancel it in the reaper. + assertTrue(event1.future().isDone()); + assertFalse(event1.future().isCancelled()); + assertNull(ConsumerUtils.getResult(event1.future())); + + // The second event was incomplete, so it was canceled. + assertTrue(event2.future().isDone()); + assertTrue(event2.future().isCancelled()); + assertThrows(CancellationException.class, () -> ConsumerUtils.getResult(event2.future())); + + // Because the events aren't tracked in the reaper *and* the queue isn't drained as part of the + // cancellation process, our data structures should both be the same as above. + assertEquals(0, reaper.size()); + assertEquals(2, queue.size()); + } + + @Test + public void testIncompleteTracked() { + // This queue is just here to test the case where the queue is empty. + BlockingQueue> queue = new LinkedBlockingQueue<>(); + + // Add two events for the reaper to track. + Timer timer = time.timer(100); + UnsubscribeEvent event1 = new UnsubscribeEvent(timer); + UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + reaper.add(event1); + reaper.add(event2); + + // Complete one of our events, just to make sure it isn't inadvertently canceled. + event1.future().complete(null); + + // In this test, our events are tracked exclusively in the reaper, not the queue. + assertEquals(2, reaper.size()); + + // Go ahead and reap the incomplete events. Both sets should be zero after that. + reaper.reapIncomplete(queue); + assertEquals(0, reaper.size()); + assertEquals(0, queue.size()); + + // The first event was completed, so we didn't cancel it in the reaper. + assertTrue(event1.future().isDone()); + assertFalse(event1.future().isCancelled()); + assertNull(ConsumerUtils.getResult(event1.future())); + + // The second event was incomplete, so it was canceled. + assertTrue(event2.future().isDone()); + assertTrue(event2.future().isCancelled()); + assertThrows(CancellationException.class, () -> ConsumerUtils.getResult(event2.future())); + } +} From c4ff61b9f5a8dd96e635f48303674748b35e4777 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 1 Apr 2024 17:09:28 -0700 Subject: [PATCH 081/130] Updated EventProcessor JavaDoc --- .../internals/events/EventProcessor.java | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index c5c6f2132a90c..1c0bb0305989e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -19,13 +19,23 @@ import java.util.concurrent.BlockingQueue; /** - * An {@link EventProcessor} is the means by which events produced by thread A are - * processed by thread B. By definition, threads A and B run in parallel to - * each other, so a mechanism is needed with which to receive and process the events from the other thread. That - * communication channel is formed around {@link BlockingQueue a shared queue} into which thread A - * enqueues events and thread B reads and processes those events. + * An {@code EventProcessor} is the means by which events are processed, the meaning of which is left + * intentionally loose. This is in large part to keep the {@code EventProcessor} focused on what it means to process + * the events, and not linking itself too closely with the rest of the surrounding application. + * + *

    + * + * The {@code EventProcessor} is envisaged as a stateless service that acts as a conduit, receiving an event and + * dispatching to another block of code to process. The semantic meaning of each event is different, so the + * {@code EventProcessor} will need to interact with other parts of the system that maintain state. The + * implementation should not be concerned with the mechanism by which an event arrived for processing. While the + * events are shuffled around the consumer subsystem by means of {@link BlockingQueue shared queues}, it should + * be considered an anti-pattern to need to know how it arrived or what happens after its is processed. */ public interface EventProcessor { + /** + * Process an event that is received. + */ void process(T event); } From df7008682abc2c6a75d85742dc1cbe0859922169 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 15:55:50 -0700 Subject: [PATCH 082/130] Refactoring for less diff noise --- .../internals/AsyncKafkaConsumer.java | 29 ++++--- .../events/ApplicationEventProcessor.java | 6 +- .../events/CompletableEventReaper.java | 31 +++---- .../internals/AsyncKafkaConsumerTest.java | 83 ++++++++++++++++--- 4 files changed, 103 insertions(+), 46 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 76c89ed83b9dd..34da0258d3cc8 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 @@ -956,7 +956,8 @@ public List partitionsFor(String topic, Duration timeout) { final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); + Map> topicMetadata = + applicationEventHandler.addAndGet(topicMetadataEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1059,8 +1060,9 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection offsetAndTimestampMap; + // If timeout is set to zero return empty immediately; otherwise try to get the results + // and throw timeout exception if it cannot complete in time. if (timeout.isZero()) { - // Return an empty results but also send a request to update the highwatermark. applicationEventHandler.add(listOffsetsEvent); return listOffsetsEvent.emptyResults(); } + + Map offsetAndTimestampMap; offsetAndTimestampMap = applicationEventHandler.addAndGet( listOffsetsEvent); return offsetAndTimestampMap.entrySet() @@ -1448,8 +1452,10 @@ private void updatePatternSubscription(Cluster cluster) { final Set topicsToSubscribe = cluster.topics().stream() .filter(subscriptions::matchesSubscribedPattern) .collect(Collectors.toSet()); - if (subscriptions.subscribeFromPattern(topicsToSubscribe)) + if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { + applicationEventHandler.add(new SubscriptionChangeEvent()); metadata.requestUpdateForNewTopics(); + } } @Override @@ -1598,7 +1604,6 @@ private boolean updateFetchPositions(final Timer timer) { applicationEventHandler.addAndGet(new ResetPositionsEvent(timer)); return true; } catch (TimeoutException e) { - log.debug("Timeout while updating fetch positions", e); return false; } } @@ -1654,11 +1659,9 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); + maybeUpdateSubscriptionMetadata(); processBackgroundEvents(); - // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as - // in the previous implementation, because it will eventually involve group coordination - // logic return updateFetchPositions(timer); } @@ -1737,8 +1740,8 @@ private void subscribeInternal(Pattern pattern, Optional BiConsumer complete(final CompletableFuture f) { + private BiConsumer complete(final CompletableFuture b) { return (value, exception) -> { if (exception != null) - f.completeExceptionally(exception); + b.completeExceptionally(exception); else - f.complete(value); + b.complete(value); }; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index b2204cdcbd4ba..bf9c16550d20e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -25,10 +25,10 @@ import java.util.Collection; import java.util.List; import java.util.Objects; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.function.Consumer; -import java.util.function.Predicate; /** * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed, @@ -44,11 +44,6 @@ public class CompletableEventReaper> { */ private final List tracked; - /** - * {@link Predicate} to check if a {@link Future} is {@link Future#isDone() done}. - */ - private final Predicate doneFilter = e -> e.future().isDone(); - public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); this.tracked = new ArrayList<>(); @@ -89,7 +84,7 @@ public void add(T event) { public void reapExpiredAndCompleted(long currentTimeMs) { log.trace("Reaping expired events"); - Consumer> completeEvent = e -> { + Consumer> timeoutEvent = e -> { TimeoutException error = new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName())); long pastDueMs = currentTimeMs - e.deadlineMs(); log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); @@ -100,13 +95,13 @@ public void reapExpiredAndCompleted(long currentTimeMs) { // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. tracked .stream() - .filter(not(doneFilter)) + .filter(e -> !e.future().isDone()) .filter(e -> currentTimeMs > e.deadlineMs()) - .forEach(completeEvent); + .forEach(timeoutEvent); // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. - tracked.removeIf(doneFilter); + tracked.removeIf(e -> e.future().isDone()); log.trace("Finished reaping expired events"); } @@ -133,22 +128,22 @@ public void reapIncomplete(Collection events) { Objects.requireNonNull(events, "Event queue to reap must be non-null"); - Consumer> completeEvent = e -> { + Consumer> cancelEvent = e -> { log.debug("Canceling event {} since the consumer is closing", e); CompletableFuture f = e.future(); - f.cancel(true); + f.completeExceptionally(new CancellationException("Canceling event since the consumer is closing")); }; tracked .stream() - .filter(not(doneFilter)) - .forEach(completeEvent); + .filter(e -> !e.future().isDone()) + .forEach(cancelEvent); tracked.clear(); events .stream() - .filter(not(doneFilter)) - .forEach(completeEvent); + .filter(e -> !e.future().isDone()) + .forEach(cancelEvent); log.trace("Finished reaping incomplete events"); } @@ -156,8 +151,4 @@ public void reapIncomplete(Collection events) { int size() { return tracked.size(); } - - private Predicate not(Predicate p) { - return p.negate(); - } } 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 59c62374fb270..9bb6a3261cb10 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -198,25 +199,33 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { ); } - private AsyncKafkaConsumer newConsumer(SubscriptionState subscriptions, boolean autoCommitEnabled) { + private AsyncKafkaConsumer newConsumer( + FetchBuffer fetchBuffer, + ConsumerInterceptors interceptors, + ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, + SubscriptionState subscriptions, + List assignors, + String groupId, + String clientId, + boolean autoCommitEnabled) { return new AsyncKafkaConsumer( new LogContext(), - "client-id", + clientId, new Deserializers<>(new StringDeserializer(), new StringDeserializer()), - mock(FetchBuffer.class), + fetchBuffer, fetchCollector, - mock(ConsumerInterceptors.class), + interceptors, time, applicationEventHandler, backgroundEventQueue, - mock(ConsumerRebalanceListenerInvoker.class), + rebalanceListenerInvoker, new Metrics(), subscriptions, metadata, 100L, 1000, - singletonList(new RoundRobinAssignor()), - "group-id", + assignors, + groupId, autoCommitEnabled); } @@ -547,7 +556,15 @@ public void testCommitSyncLeaderEpochUpdate() { @Test public void testCommitAsyncLeaderEpochUpdate() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, true); + consumer = newConsumer( + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); completeCommitSyncApplicationEventSuccessfully(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); @@ -677,7 +694,15 @@ public void testVerifyApplicationEventOnShutdown() { public void testPartitionRevocationOnClose() { MockRebalanceListener listener = new MockRebalanceListener(); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, true); + consumer = newConsumer( + mock(FetchBuffer.class), + mock(ConsumerInterceptors.class), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); consumer.subscribe(singleton("topic"), listener); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); @@ -692,7 +717,15 @@ public void testFailedPartitionRevocationOnClose() { // closing the consumer. ConsumerRebalanceListener listener = mock(ConsumerRebalanceListener.class); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, true); + consumer = newConsumer( + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); subscriptions.subscribe(singleton("topic"), Optional.of(listener)); TopicPartition tp = new TopicPartition("topic", 0); subscriptions.assignFromSubscribed(singleton(tp)); @@ -723,7 +756,15 @@ public void testCompleteQuietly() { public void testAutoCommitSyncEnabled() { completeCommitSyncApplicationEventSuccessfully(); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, true); + consumer = newConsumer( + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); @@ -734,7 +775,15 @@ public void testAutoCommitSyncEnabled() { @Test public void testAutoCommitSyncDisabled() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, false); + consumer = newConsumer( + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + false); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); @@ -1428,7 +1477,15 @@ public void testGroupIdOnlyWhitespaces() { @Test public void testEnsurePollEventSentOnConsumerPoll() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer(subscriptions, true); + consumer = newConsumer( + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); final TopicPartition tp = new TopicPartition("topic", 0); final List> records = singletonList( new ConsumerRecord<>("topic", 0, 2, "key1", "value1")); From 83790d47e2c1dc7d0c1485d5958630483049eef2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:09:45 -0700 Subject: [PATCH 083/130] Trying more to clean up diffs --- .../internals/AsyncKafkaConsumerTest.java | 46 +++++++++++++------ 1 file changed, 31 insertions(+), 15 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 9bb6a3261cb10..3c7014ea81e14 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 @@ -126,6 +126,7 @@ import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeast; @@ -147,7 +148,7 @@ public class AsyncKafkaConsumerTest { private AsyncKafkaConsumer consumer = null; - private final Time time = new MockTime(1); + private Time time = new MockTime(1); private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); @@ -208,7 +209,7 @@ private AsyncKafkaConsumer newConsumer( String groupId, String clientId, boolean autoCommitEnabled) { - return new AsyncKafkaConsumer( + return new AsyncKafkaConsumer<>( new LogContext(), clientId, new Deserializers<>(new StringDeserializer(), new StringDeserializer()), @@ -222,8 +223,8 @@ private AsyncKafkaConsumer newConsumer( new Metrics(), subscriptions, metadata, - 100L, - 1000, + retryBackoffMs, + defaultApiTimeoutMs, assignors, groupId, autoCommitEnabled); @@ -758,7 +759,7 @@ public void testAutoCommitSyncEnabled() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); consumer = newConsumer( mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerInterceptors.class), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, singletonList(new RoundRobinAssignor()), @@ -777,7 +778,7 @@ public void testAutoCommitSyncDisabled() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); consumer = newConsumer( mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerInterceptors.class), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, singletonList(new RoundRobinAssignor()), @@ -865,7 +866,16 @@ public void testBeginningOffsets() { consumer = newConsumer(); Map expectedOffsets = mockOffsetAndTimestamp(); - doReturn(expectedOffsets).when(applicationEventHandler).addAndGet(any()); + // Don't use the instance's Timer as it auto-increments and will break the test. + Time time = new MockTime(); + when(applicationEventHandler.addAndGet(any(ListOffsetsEvent.class))).thenAnswer(invocation -> { + ListOffsetsEvent event = invocation.getArgument(0); + Timer timer = time.timer(event.deadlineMs() - time.milliseconds()); + if (timer.remainingMs() == 0) { + fail("Timer duration should not be zero."); + } + return expectedOffsets; + }); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(expectedOffsets.keySet(), Duration.ofMillis(1))); @@ -1478,14 +1488,14 @@ public void testGroupIdOnlyWhitespaces() { public void testEnsurePollEventSentOnConsumerPoll() { SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); consumer = newConsumer( - mock(FetchBuffer.class), - new ConsumerInterceptors<>(Collections.emptyList()), - mock(ConsumerRebalanceListenerInvoker.class), - subscriptions, - singletonList(new RoundRobinAssignor()), - "group-id", - "client-id", - autoCommitEnabled); + mock(FetchBuffer.class), + new ConsumerInterceptors<>(Collections.emptyList()), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + singletonList(new RoundRobinAssignor()), + "group-id", + "client-id", + autoCommitEnabled); final TopicPartition tp = new TopicPartition("topic", 0); final List> records = singletonList( new ConsumerRecord<>("topic", 0, 2, "key1", "value1")); @@ -1600,6 +1610,8 @@ public void testLongPollWaitIsLimited() { @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { consumer = newConsumer(); + + // Don't use the instance's Timer as it auto-increments. Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); @@ -1634,6 +1646,8 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { @Test public void testProcessBackgroundEventsWithoutDelay() { consumer = newConsumer(); + + // Don't use the instance's Timer as it auto-increments. Time time = new MockTime(); Timer timer = time.timer(1000); @@ -1654,6 +1668,8 @@ public void testProcessBackgroundEventsWithoutDelay() { @Test public void testProcessBackgroundEventsTimesOut() throws Exception { consumer = newConsumer(); + + // Don't use the instance's Timer as it auto-increments. Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); From 7289abf80bae5e3258ca82f7a4ba255beb1c5cc7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:23:20 -0700 Subject: [PATCH 084/130] Reverting some MockTime related tweaks --- .../consumer/internals/AsyncKafkaConsumerTest.java | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 3c7014ea81e14..f47e8abb4555f 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 @@ -148,7 +148,7 @@ public class AsyncKafkaConsumerTest { private AsyncKafkaConsumer consumer = null; - private Time time = new MockTime(1); + private Time time = new MockTime(0); private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); @@ -337,6 +337,7 @@ public void testCommitAsyncWithFencedException() { @Test public void testCommitted() { + time = new MockTime(1); consumer = newConsumer(); Map topicPartitionOffsets = mockTopicPartitionOffset(); completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); @@ -866,8 +867,6 @@ public void testBeginningOffsets() { consumer = newConsumer(); Map expectedOffsets = mockOffsetAndTimestamp(); - // Don't use the instance's Timer as it auto-increments and will break the test. - Time time = new MockTime(); when(applicationEventHandler.addAndGet(any(ListOffsetsEvent.class))).thenAnswer(invocation -> { ListOffsetsEvent event = invocation.getArgument(0); Timer timer = time.timer(event.deadlineMs() - time.milliseconds()); @@ -1610,9 +1609,6 @@ public void testLongPollWaitIsLimited() { @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { consumer = newConsumer(); - - // Don't use the instance's Timer as it auto-increments. - Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); CountDownLatch latch = new CountDownLatch(3); @@ -1646,9 +1642,6 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { @Test public void testProcessBackgroundEventsWithoutDelay() { consumer = newConsumer(); - - // Don't use the instance's Timer as it auto-increments. - Time time = new MockTime(); Timer timer = time.timer(1000); // Create a future that is already completed. @@ -1668,9 +1661,6 @@ public void testProcessBackgroundEventsWithoutDelay() { @Test public void testProcessBackgroundEventsTimesOut() throws Exception { consumer = newConsumer(); - - // Don't use the instance's Timer as it auto-increments. - Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); From 5eeeea256ed992925bcc8d9fd03bb7ecd0c13f90 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:27:49 -0700 Subject: [PATCH 085/130] More reverts --- .../internals/AsyncKafkaConsumerTest.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index f47e8abb4555f..e70c749d0f608 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 @@ -207,8 +207,7 @@ private AsyncKafkaConsumer newConsumer( SubscriptionState subscriptions, List assignors, String groupId, - String clientId, - boolean autoCommitEnabled) { + String clientId) { return new AsyncKafkaConsumer<>( new LogContext(), clientId, @@ -565,8 +564,7 @@ public void testCommitAsyncLeaderEpochUpdate() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - autoCommitEnabled); + "client-id"); completeCommitSyncApplicationEventSuccessfully(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); @@ -703,8 +701,7 @@ public void testPartitionRevocationOnClose() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - autoCommitEnabled); + "client-id"); consumer.subscribe(singleton("topic"), listener); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); @@ -726,8 +723,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - autoCommitEnabled); + "client-id"); subscriptions.subscribe(singleton("topic"), Optional.of(listener)); TopicPartition tp = new TopicPartition("topic", 0); subscriptions.assignFromSubscribed(singleton(tp)); @@ -765,8 +761,7 @@ public void testAutoCommitSyncEnabled() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - autoCommitEnabled); + "client-id"); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); @@ -776,6 +771,7 @@ public void testAutoCommitSyncEnabled() { @Test public void testAutoCommitSyncDisabled() { + autoCommitEnabled = false; SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); consumer = newConsumer( mock(FetchBuffer.class), @@ -784,8 +780,7 @@ public void testAutoCommitSyncDisabled() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - false); + "client-id"); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); @@ -1493,8 +1488,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { subscriptions, singletonList(new RoundRobinAssignor()), "group-id", - "client-id", - autoCommitEnabled); + "client-id"); final TopicPartition tp = new TopicPartition("topic", 0); final List> records = singletonList( new ConsumerRecord<>("topic", 0, 2, "key1", "value1")); From b520fbd6867c079375194c0b41b868871e051a96 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:37:54 -0700 Subject: [PATCH 086/130] More reverts --- .../internals/AsyncKafkaConsumer.java | 8 +++-- .../internals/events/EventProcessor.java | 7 +++- .../internals/AsyncKafkaConsumerTest.java | 32 +++++++++++-------- 3 files changed, 31 insertions(+), 16 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 34da0258d3cc8..66d8b77a0ea6b 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 @@ -958,6 +958,7 @@ public List partitionsFor(String topic, Duration timeout) { try { Map> topicMetadata = applicationEventHandler.addAndGet(topicMetadataEvent); + return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { wakeupTrigger.clearTask(); @@ -1061,6 +1062,7 @@ public Map offsetsForTimes(Map T processBackgroundEvents(Future future, Timer timer) { + T processBackgroundEvents(EventProcessor eventProcessor, + Future future, + Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 1c0bb0305989e..13129126dcb8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -32,10 +32,15 @@ * events are shuffled around the consumer subsystem by means of {@link BlockingQueue shared queues}, it should * be considered an anti-pattern to need to know how it arrived or what happens after its is processed. */ -public interface EventProcessor { +public interface EventProcessor extends AutoCloseable { /** * Process an event that is received. */ void process(T event); + + @Override + default void close() { + // Do nothing by default... + } } 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 e70c749d0f608..19e5fa9118137 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 @@ -1597,7 +1597,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1623,14 +1623,16 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - consumer.processBackgroundEvents(future, timer); + try (EventProcessor processor = mock(EventProcessor.class)) { + consumer.processBackgroundEvents(processor, future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); + } } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1641,15 +1643,17 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - consumer.processBackgroundEvents(future, timer); + try (EventProcessor processor = mock(EventProcessor.class)) { + consumer.processBackgroundEvents(processor, future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); + } } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1664,10 +1668,12 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); + try (EventProcessor processor = mock(EventProcessor.class)) { + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); + } } private Map mockTopicPartitionOffset() { From 211f2975b996e8dbb8413d9d9271d8e6f0e21f00 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:44:38 -0700 Subject: [PATCH 087/130] Yet still more refactoring --- .../internals/AsyncKafkaConsumer.java | 73 ++++++++++--------- 1 file changed, 37 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 66d8b77a0ea6b..eed6317b74e31 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 @@ -175,6 +175,40 @@ public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalance this.rebalanceListenerInvoker = rebalanceListenerInvoker; } + /** + * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. + * It is possible that {@link ErrorEvent an error} + * could occur when processing the events. In such cases, the processor will take a reference to the first + * error, continue to process the remaining events, and then throw the first error that occurred. + */ + public boolean process() { + AtomicReference firstError = new AtomicReference<>(); + + LinkedList events = new LinkedList<>(); + backgroundEventQueue.drainTo(events); + + for (BackgroundEvent event : events) { + try { + if (event instanceof CompletableBackgroundEvent) + backgroundEventReaper.add((CompletableBackgroundEvent) event); + + backgroundEventProcessor.process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + + if (!firstError.compareAndSet(null, e)) + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); + } + } + + backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); + + if (firstError.get() != null) + throw firstError.get(); + + return !events.isEmpty(); + } + @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -1662,7 +1696,7 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); maybeUpdateSubscriptionMetadata(); - processBackgroundEvents(); + backgroundEventProcessor.process(); return updateFetchPositions(timer); } @@ -1788,40 +1822,6 @@ private void subscribeInternal(Collection topics, Optional firstError = new AtomicReference<>(); - - LinkedList events = new LinkedList<>(); - backgroundEventQueue.drainTo(events); - - for (BackgroundEvent event : events) { - try { - if (event instanceof CompletableBackgroundEvent) - backgroundEventReaper.add((CompletableBackgroundEvent) event); - - backgroundEventProcessor.process(event); - } catch (Throwable t) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - - if (!firstError.compareAndSet(null, e)) - log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); - } - } - - backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); - - if (firstError.get() != null) - throw firstError.get(); - - return !events.isEmpty(); - } - /** * This method can be used by cases where the caller has an event that needs to both block for completion but * also process background events. For some events, in order to fully process the associated logic, the @@ -1854,6 +1854,7 @@ private boolean processBackgroundEvents() { * execution of the rebalancing logic. The rebalancing logic cannot complete until the * {@link ConsumerRebalanceListener} callback is performed. * + * @param eventProcessor Event processor that contains the queue of events to process * @param future Event that contains a {@link CompletableFuture}; it is on this future that the * application thread will wait for completion * @param timer Overall timer that bounds how long to wait for the event to complete @@ -1866,7 +1867,7 @@ T processBackgroundEvents(EventProcessor eventProcessor, log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = processBackgroundEvents(); + boolean hadEvents = backgroundEventProcessor.process(); try { if (future.isDone()) { From 26d369ba8158786775a8249b25734008fac310e7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 11 Apr 2024 16:54:59 -0700 Subject: [PATCH 088/130] Reverting the reverts as it broke other stuff :( --- .../internals/AsyncKafkaConsumer.java | 74 +++++++++---------- .../internals/AsyncKafkaConsumerTest.java | 6 +- 2 files changed, 40 insertions(+), 40 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index eed6317b74e31..14c7344b10e77 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 @@ -175,40 +175,6 @@ public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalance this.rebalanceListenerInvoker = rebalanceListenerInvoker; } - /** - * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. - * It is possible that {@link ErrorEvent an error} - * could occur when processing the events. In such cases, the processor will take a reference to the first - * error, continue to process the remaining events, and then throw the first error that occurred. - */ - public boolean process() { - AtomicReference firstError = new AtomicReference<>(); - - LinkedList events = new LinkedList<>(); - backgroundEventQueue.drainTo(events); - - for (BackgroundEvent event : events) { - try { - if (event instanceof CompletableBackgroundEvent) - backgroundEventReaper.add((CompletableBackgroundEvent) event); - - backgroundEventProcessor.process(event); - } catch (Throwable t) { - KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); - - if (!firstError.compareAndSet(null, e)) - log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); - } - } - - backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); - - if (firstError.get() != null) - throw firstError.get(); - - return !events.isEmpty(); - } - @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -1696,7 +1662,7 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); maybeInvokeCommitCallbacks(); maybeUpdateSubscriptionMetadata(); - backgroundEventProcessor.process(); + process(backgroundEventProcessor); return updateFetchPositions(timer); } @@ -1822,6 +1788,40 @@ private void subscribeInternal(Collection topics, Optional processor) { + AtomicReference firstError = new AtomicReference<>(); + + LinkedList events = new LinkedList<>(); + backgroundEventQueue.drainTo(events); + + for (BackgroundEvent event : events) { + try { + if (event instanceof CompletableBackgroundEvent) + backgroundEventReaper.add((CompletableBackgroundEvent) event); + + processor.process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + + if (!firstError.compareAndSet(null, e)) + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); + } + } + + backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); + + if (firstError.get() != null) + throw firstError.get(); + + return !events.isEmpty(); + } + /** * This method can be used by cases where the caller has an event that needs to both block for completion but * also process background events. For some events, in order to fully process the associated logic, the @@ -1861,13 +1861,13 @@ private void subscribeInternal(Collection topics, Optional T processBackgroundEvents(EventProcessor eventProcessor, + T processBackgroundEvents(EventProcessor eventProcessor, Future future, Timer timer) { log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = backgroundEventProcessor.process(); + boolean hadEvents = process(eventProcessor); try { if (future.isDone()) { 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 19e5fa9118137..92dc97d6525bf 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 @@ -1623,7 +1623,7 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { + try (EventProcessor processor = mock(EventProcessor.class)) { consumer.processBackgroundEvents(processor, future, timer); // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. @@ -1643,7 +1643,7 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - try (EventProcessor processor = mock(EventProcessor.class)) { + try (EventProcessor processor = mock(EventProcessor.class)) { consumer.processBackgroundEvents(processor, future, timer); // Because we didn't need to perform a timed get, we should still have every last millisecond @@ -1668,7 +1668,7 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { + try (EventProcessor processor = mock(EventProcessor.class)) { assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); // Because we forced our mocked future to continuously time out, we should have no time remaining. From 7f2d2318b49ba003f8971955c8353a69df9fa906 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:33:17 -0700 Subject: [PATCH 089/130] Added a brief comment about the case where close() is called from within the constructor --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index fe3d9bc409e4f..d8058519da955 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 @@ -1231,6 +1231,8 @@ private void close(Duration timeout, boolean swallowException) { closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed shutting down network thread", firstException); closeTimer.update(); + // close() can be called from inside one of the constructors. In that case, it's possible that neither + // the reaper nor the background event queue were constructed, so check them first to avoid NPE. if (backgroundEventReaper != null && backgroundEventQueue != null) { // Copy over the completable events to a separate list, then reap any incomplete // events on that list. From 0c646adce87f72eeaf02ec1fa9f3b377defa0b6f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:39:58 -0700 Subject: [PATCH 090/130] Minor refactoring of comments and code in ConsumerNetworkThread per PR review --- .../internals/ConsumerNetworkThread.java | 51 +++++++++++-------- 1 file changed, 29 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index 27b99d4136c28..fc840f63097bf 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 @@ -135,22 +135,7 @@ void initializeResources() { * */ void runOnce() { - // Process the events—if any—that were produced by the application thread. It is possible that when processing - // an event generates an error. In such cases, the processor will log an exception, but we do not want those - // errors to be propagated to the caller. - LinkedList events = new LinkedList<>(); - applicationEventQueue.drainTo(events); - - for (ApplicationEvent event : events) { - try { - if (event instanceof CompletableApplicationEvent) - applicationEventReaper.add((CompletableApplicationEvent) event); - - applicationEventProcessor.process(event); - } catch (Throwable t) { - log.warn("Error processing event {}", t.getMessage(), t); - } - } + processApplicationEvents(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -167,10 +152,34 @@ void runOnce() { .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); - // "Complete" any events that have expired. This cleanup step should only be called after the network I/O - // thread has made at least one call to poll. This is done to emulate the behavior of the legacy consumer's - // handling of timeouts. The legacy consumer makes at least one attempt to satisfy any network requests - // before checking if a timeout has expired. + reapExpiredApplicationEvents(currentTimeMs); + } + + /** + * Process the events—if any—that were produced by the application thread. + */ + private void processApplicationEvents() { + LinkedList events = new LinkedList<>(); + applicationEventQueue.drainTo(events); + + for (ApplicationEvent event : events) { + try { + if (event instanceof CompletableApplicationEvent) + applicationEventReaper.add((CompletableApplicationEvent) event); + + applicationEventProcessor.process(event); + } catch (Throwable t) { + log.warn("Error processing event {}", t.getMessage(), t); + } + } + } + + /** + * "Complete" any events that have expired. This cleanup step should only be called after the network I/O + * thread has made at least one call to {@link NetworkClientDelegate#poll(long, long) poll} so that each event + * is given least one attempt to satisfy any network requests before checking if a timeout has expired. + */ + private void reapExpiredApplicationEvents(long currentTimeMs) { applicationEventReaper.reapExpiredAndCompleted(currentTimeMs); } @@ -302,8 +311,6 @@ void cleanup() { } finally { sendUnsentRequests(timer); - // Copy over the completable events to a separate list, then reap any incomplete - // events on that list. LinkedList allEvents = new LinkedList<>(); applicationEventQueue.drainTo(allEvents); List> completableEvents = allEvents From fd48d1f6d94f4829ad5931aa856999c455374cc1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:42:30 -0700 Subject: [PATCH 091/130] Update clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java Co-authored-by: Bruno Cadonna --- .../internals/events/CompletableEventReaper.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index bf9c16550d20e..495b5bbbce879 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -93,12 +93,10 @@ public void reapExpiredAndCompleted(long currentTimeMs) { }; // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. - tracked - .stream() - .filter(e -> !e.future().isDone()) - .filter(e -> currentTimeMs > e.deadlineMs()) - .forEach(timeoutEvent); - + tracked.stream() + .filter(e -> !e.future().isDone()) + .filter(e -> currentTimeMs > e.deadlineMs()) + .forEach(timeoutEvent); // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. tracked.removeIf(e -> e.future().isDone()); From 68beee56446bd04f73befefb516621cafdd7cf71 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:43:58 -0700 Subject: [PATCH 092/130] Made reapIncomplete's stream processing code consistent with reapExpiredAndCompleted --- .../internals/events/CompletableEventReaper.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 495b5bbbce879..06b7e7a4eb97c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -132,16 +132,14 @@ public void reapIncomplete(Collection events) { f.completeExceptionally(new CancellationException("Canceling event since the consumer is closing")); }; - tracked - .stream() - .filter(e -> !e.future().isDone()) - .forEach(cancelEvent); + tracked.stream() + .filter(e -> !e.future().isDone()) + .forEach(cancelEvent); tracked.clear(); - events - .stream() - .filter(e -> !e.future().isDone()) - .forEach(cancelEvent); + events.stream() + .filter(e -> !e.future().isDone()) + .forEach(cancelEvent); log.trace("Finished reaping incomplete events"); } From 337d1f7d4615a91e30148b5c71e4c3b61284e107 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:49:30 -0700 Subject: [PATCH 093/130] Added calculateDeadlineMs to CompletableEvent for reuse by implementations --- .../events/CompletableApplicationEvent.java | 11 +-------- .../events/CompletableBackgroundEvent.java | 11 +-------- .../internals/events/CompletableEvent.java | 23 +++++++++++++++++++ 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 9e688f7a78f78..c79a9c1cc1587 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.utils.Timer; -import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -35,15 +34,7 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im protected CompletableApplicationEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); - Objects.requireNonNull(timer); - - long currentTimeMs = timer.currentTimeMs(); - long remainingMs = timer.remainingMs(); - - if (currentTimeMs > Long.MAX_VALUE - remainingMs) - this.deadlineMs = Long.MAX_VALUE; - else - this.deadlineMs = currentTimeMs + remainingMs; + this.deadlineMs = CompletableEvent.calculateDeadlineMs(timer); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 1e7b19248991f..4a80952fd6736 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.utils.Timer; -import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -35,15 +34,7 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl protected CompletableBackgroundEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); - Objects.requireNonNull(timer); - - long currentTimeMs = timer.currentTimeMs(); - long remainingMs = timer.remainingMs(); - - if (currentTimeMs > Long.MAX_VALUE - remainingMs) - this.deadlineMs = Long.MAX_VALUE; - else - this.deadlineMs = currentTimeMs + remainingMs; + this.deadlineMs = CompletableEvent.calculateDeadlineMs(timer); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 97a16759d9495..240969d8cb3b8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import java.util.Objects; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -73,4 +76,24 @@ public interface CompletableEvent { * @see CompletableEventReaper */ long deadlineMs(); + + /** + * Calculate the deadline timestamp based on the given timer. + * + * @param timer Timer from which to use the {@link Timer#currentTimeMs()} and {@link Timer#remainingMs()}} to + * calculate the deadline + * + * @return Absolute time for event to be completed + */ + static long calculateDeadlineMs(final Timer timer) { + Objects.requireNonNull(timer); + + long currentTimeMs = timer.currentTimeMs(); + long remainingMs = timer.remainingMs(); + + if (currentTimeMs > Long.MAX_VALUE - remainingMs) + return Long.MAX_VALUE; + else + return currentTimeMs + remainingMs; + } } From 74a7ed78cc69f0d28bd18139b90c28468058e111 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 10:53:38 -0700 Subject: [PATCH 094/130] Removed EventProcessor.close() as it is no longer needed --- .../internals/events/EventProcessor.java | 7 +---- .../internals/AsyncKafkaConsumerTest.java | 29 +++++++++---------- 2 files changed, 14 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 13129126dcb8a..1c0bb0305989e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -32,15 +32,10 @@ * events are shuffled around the consumer subsystem by means of {@link BlockingQueue shared queues}, it should * be considered an anti-pattern to need to know how it arrived or what happens after its is processed. */ -public interface EventProcessor extends AutoCloseable { +public interface EventProcessor { /** * Process an event that is received. */ void process(T event); - - @Override - default void close() { - // Do nothing by default... - } } 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 80e6ae286acda..921c830ee90f0 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 @@ -1750,12 +1750,11 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + EventProcessor processor = mock(EventProcessor.class); + consumer.processBackgroundEvents(processor, future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); - } + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); } /** @@ -1770,13 +1769,12 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + EventProcessor processor = mock(EventProcessor.class); + consumer.processBackgroundEvents(processor, future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); - } + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); } /** @@ -1795,12 +1793,11 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); + EventProcessor processor = mock(EventProcessor.class); + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); - } + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); } private Map mockTopicPartitionOffset() { From 92c2d859d0b13281ae92be3b7accc3d44f9d9e67 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 11:03:10 -0700 Subject: [PATCH 095/130] Removed the need for explicit Long.MAX_VALUE-based Timer objects for those CompletableEvents which don't have a hard timeout --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 3 +-- .../consumer/internals/MembershipManagerImpl.java | 9 +-------- .../consumer/internals/events/AsyncCommitEvent.java | 4 ++-- .../clients/consumer/internals/events/CommitEvent.java | 6 +++++- .../internals/events/CompletableApplicationEvent.java | 6 +++++- .../internals/events/CompletableBackgroundEvent.java | 6 ++---- .../ConsumerRebalanceListenerCallbackNeededEvent.java | 6 ++---- .../consumer/internals/AsyncKafkaConsumerTest.java | 4 ++-- .../consumer/internals/ConsumerNetworkThreadTest.java | 8 +++----- 9 files changed, 23 insertions(+), 29 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 d8058519da955..c3f26b13060bf 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 @@ -730,8 +730,7 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - Timer timer = time.timer(Long.MAX_VALUE); - AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets, timer); + AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets); lastPendingAsyncCommit = commit(asyncCommitEvent).whenComplete((r, t) -> { if (t == null) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 0367109bc2b26..a7b32caa9284b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -40,7 +40,6 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -1340,13 +1339,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - // We don't yet have the concept of having an expiring callback, but we will likely want that eventually. - Timer timer = time.timer(Long.MAX_VALUE); - CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent( - methodName, - sortedPartitions, - timer - ); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index 971129d5b86fd..e9739f3ad721c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -27,7 +27,7 @@ */ public class AsyncCommitEvent extends CommitEvent { - public AsyncCommitEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_ASYNC, offsets, timer); + public AsyncCommitEvent(final Map offsets) { + super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 889ab60de0ee0..1e265856a6019 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -31,7 +31,11 @@ public abstract class CommitEvent extends CompletableApplicationEvent { private final Map offsets; protected CommitEvent(final Type type, final Map offsets, final Timer timer) { - super(type, timer); + this(type, offsets, CompletableEvent.calculateDeadlineMs(timer)); + } + + public CommitEvent(final Type type, Map offsets, final long deadlineMs) { + super(type, deadlineMs); this.offsets = validate(offsets); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index c79a9c1cc1587..531f127bde1a2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -32,9 +32,13 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im private final long deadlineMs; protected CompletableApplicationEvent(final Type type, final Timer timer) { + this(type, CompletableEvent.calculateDeadlineMs(timer)); + } + + protected CompletableApplicationEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = CompletableEvent.calculateDeadlineMs(timer); + this.deadlineMs = deadlineMs; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 4a80952fd6736..716d7f7d30691 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - import java.util.concurrent.CompletableFuture; /** @@ -31,10 +29,10 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; private final long deadlineMs; - protected CompletableBackgroundEvent(final Type type, final Timer timer) { + protected CompletableBackgroundEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); - this.deadlineMs = CompletableEvent.calculateDeadlineMs(timer); + this.deadlineMs = deadlineMs; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 2165d621a21eb..ecb9eedab22c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.time.Duration; import java.util.Collections; @@ -39,9 +38,8 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final SortedSet partitions; public ConsumerRebalanceListenerCallbackNeededEvent(final ConsumerRebalanceListenerMethodName methodName, - final SortedSet partitions, - final Timer timer) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, timer); + final SortedSet partitions) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, Long.MAX_VALUE); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } 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 921c830ee90f0..91ea7a2b20232 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 @@ -464,7 +464,7 @@ public void testCommitInRebalanceCallback() { doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.add(tp); - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions, time.timer(Long.MAX_VALUE)); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, sortedPartitions); backgroundEventQueue.add(e); completeCommitSyncApplicationEventSuccessfully(); final AtomicBoolean callbackExecuted = new AtomicBoolean(false); @@ -1459,7 +1459,7 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); for (ConsumerRebalanceListenerMethodName methodName : methodNames) { - CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions, time.timer(Long.MAX_VALUE)); + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); backgroundEventQueue.add(e); // This will trigger the background event queue to process our background event message. 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 48eaca81789d6..091009064deef 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 @@ -156,8 +156,7 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - Timer timer = time.timer(100); - ApplicationEvent e = new AsyncCommitEvent(new HashMap<>(), timer); + ApplicationEvent e = new AsyncCommitEvent(new HashMap<>()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(AsyncCommitEvent.class)); @@ -294,9 +293,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - Timer timer = time.timer(100); - CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap(), timer)); - ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap(), timer); + CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); + ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); From 2981ebed7df2f2577d483ea260b002a4aa331100 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 25 Apr 2024 11:06:44 -0700 Subject: [PATCH 096/130] Removed superfluous import --- .../clients/consumer/internals/events/AsyncCommitEvent.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index e9739f3ad721c..c36f0534b3671 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Map; From 1e2144b0650b2de7acb0a53d03709ba5d484d66d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Apr 2024 11:15:01 -0700 Subject: [PATCH 097/130] Updates to use long instead of Timer to indicate the deadline --- .../internals/AsyncKafkaConsumer.java | 26 ++++----- .../events/AbstractTopicMetadataEvent.java | 5 +- .../events/AllTopicsMetadataEvent.java | 6 +-- .../events/CompletableApplicationEvent.java | 9 ++-- .../events/CompletableBackgroundEvent.java | 3 ++ .../internals/events/CompletableEvent.java | 54 +++++++++++++++---- .../events/FetchCommittedOffsetsEvent.java | 5 +- .../internals/events/LeaveOnCloseEvent.java | 6 +-- .../internals/events/ListOffsetsEvent.java | 5 +- .../internals/events/ResetPositionsEvent.java | 6 +-- .../internals/events/TopicMetadataEvent.java | 6 +-- .../internals/events/UnsubscribeEvent.java | 6 +-- .../events/ValidatePositionsEvent.java | 6 +-- .../internals/ConsumerNetworkThreadTest.java | 16 +++--- .../events/ApplicationEventProcessorTest.java | 5 +- .../events/CompletableEventReaperTest.java | 17 +++--- 16 files changed, 96 insertions(+), 85 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 48b8774b87e69..07143092d87df 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 @@ -134,6 +134,7 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.refreshCommittedOffsets; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.apache.kafka.common.utils.Utils.closeQuietly; import static org.apache.kafka.common.utils.Utils.isBlank; import static org.apache.kafka.common.utils.Utils.join; @@ -904,10 +905,9 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event); @@ -956,8 +956,7 @@ public List partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); - final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { Map> topicMetadata = @@ -985,8 +984,7 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); - final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timer); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { return applicationEventHandler.addAndGet(topicMetadataEvent); @@ -1057,10 +1055,9 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - Timer timer = time.timer(timeout); ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, - timer, + calculateDeadlineMs(time, timeout), false); // If timeout is set to zero return empty immediately; otherwise try to get the results @@ -1282,7 +1278,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseEvent(timer)); + applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer))); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1501,7 +1497,7 @@ public void unsubscribe() { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.get().isPresent()) { Timer timer = time.timer(Long.MAX_VALUE); - UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(timer); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer)); applicationEventHandler.add(unsubscribeEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); @@ -1614,7 +1610,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsEvent(timer)); + applicationEventHandler.addAndGet(new ValidatePositionsEvent(calculateDeadlineMs(timer))); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1643,7 +1639,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsEvent(timer)); + applicationEventHandler.addAndGet(new ResetPositionsEvent(calculateDeadlineMs(timer))); return true; } catch (TimeoutException e) { return false; @@ -1676,7 +1672,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsEvent event = new FetchCommittedOffsetsEvent( initializingPartitions, - timer); + calculateDeadlineMs(timer)); final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index 3347002cc6fea..9621e34ef5b94 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -17,14 +17,13 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { - protected AbstractTopicMetadataEvent(final Type type, final Timer timer) { - super(type, timer); + protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) { + super(type, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java index bda18e642105b..8fe1702c85bd6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - public class AllTopicsMetadataEvent extends AbstractTopicMetadataEvent { - public AllTopicsMetadataEvent(final Timer timer) { - super(Type.ALL_TOPICS_METADATA, timer); + public AllTopicsMetadataEvent(final long deadlineMs) { + super(Type.ALL_TOPICS_METADATA, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 531f127bde1a2..dffac12902177 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - import java.util.concurrent.CompletableFuture; /** @@ -31,10 +29,9 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im private final CompletableFuture future; private final long deadlineMs; - protected CompletableApplicationEvent(final Type type, final Timer timer) { - this(type, CompletableEvent.calculateDeadlineMs(timer)); - } - + /** + * Note: the {@code deadlineMs} is the future time of expiration, not a timeout. + */ protected CompletableApplicationEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 716d7f7d30691..d02010496e545 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -29,6 +29,9 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; private final long deadlineMs; + /** + * Note: the {@code deadlineMs} is the future time of expiration, not a timeout. + */ protected CompletableBackgroundEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 240969d8cb3b8..017ab67755c4a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -16,14 +16,17 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import java.util.Objects; +import java.time.Duration; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeoutException; +import static java.util.Objects.requireNonNull; + /** * {@code CompletableEvent} is an interface that is used by both {@link CompletableApplicationEvent} and * {@link CompletableBackgroundEvent} for common processing and logic. A {@code CompletableEvent} is one that @@ -78,22 +81,53 @@ public interface CompletableEvent { long deadlineMs(); /** - * Calculate the deadline timestamp based on the given timer. + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and {@link Timer#remainingMs()}. * - * @param timer Timer from which to use the {@link Timer#currentTimeMs()} and {@link Timer#remainingMs()}} to - * calculate the deadline + * @param timer Timer * - * @return Absolute time for event to be completed + * @return Absolute time by which event should be completed */ static long calculateDeadlineMs(final Timer timer) { - Objects.requireNonNull(timer); + requireNonNull(timer); + return calculateDeadlineMs(timer.currentTimeMs(), timer.remainingMs()); + } + + /** + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and {@link Duration#toMillis()}. + * + * @param time Time + * @param duration Duration + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final Time time, final Duration duration) { + return calculateDeadlineMs(requireNonNull(time).milliseconds(), requireNonNull(duration).toMillis()); + } - long currentTimeMs = timer.currentTimeMs(); - long remainingMs = timer.remainingMs(); + /** + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and timeout. + * + * @param time Time + * @param timeoutMs Timeout, in milliseconds + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final Time time, final long timeoutMs) { + return calculateDeadlineMs(requireNonNull(time).milliseconds(), timeoutMs); + } - if (currentTimeMs > Long.MAX_VALUE - remainingMs) + /** + * Calculate the deadline timestamp based on the current time and timeout. + * + * @param currentTimeMs Current time, in milliseconds + * @param timeoutMs Timeout, in milliseconds + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final long currentTimeMs, final long timeoutMs) { + if (currentTimeMs > Long.MAX_VALUE - timeoutMs) return Long.MAX_VALUE; else - return currentTimeMs + remainingMs; + return currentTimeMs + timeoutMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java index 980a8f1104261..785736791a7c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -31,8 +30,8 @@ public class FetchCommittedOffsetsEvent extends CompletableApplicationEvent partitions; - public FetchCommittedOffsetsEvent(final Set partitions, final Timer timer) { - super(Type.FETCH_COMMITTED_OFFSETS, timer); + public FetchCommittedOffsetsEvent(final Set partitions, final long deadlineMs) { + super(Type.FETCH_COMMITTED_OFFSETS, deadlineMs); this.partitions = Collections.unmodifiableSet(partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java index e77b4dfb2893c..647265a1500c8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - public class LeaveOnCloseEvent extends CompletableApplicationEvent { - public LeaveOnCloseEvent(final Timer timer) { - super(Type.LEAVE_ON_CLOSE, timer); + public LeaveOnCloseEvent(final long deadlineMs) { + super(Type.LEAVE_ON_CLOSE, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 3df4719a7b065..8ae2f1ea57612 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; @@ -38,9 +37,9 @@ public class ListOffsetsEvent extends CompletableApplicationEvent timestampToSearch, - Timer timer, + long deadlineMs, boolean requireTimestamps) { - super(Type.LIST_OFFSETS, timer); + super(Type.LIST_OFFSETS, deadlineMs); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java index 65893b62ecaa5..86dbb80c0f0ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -26,7 +24,7 @@ */ public class ResetPositionsEvent extends CompletableApplicationEvent { - public ResetPositionsEvent(final Timer timer) { - super(Type.RESET_POSITIONS, timer); + public ResetPositionsEvent(final long deadlineMs) { + super(Type.RESET_POSITIONS, deadlineMs); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java index 33e1270ce6040..9758ae0efa0f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java @@ -16,16 +16,14 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - import java.util.Objects; public class TopicMetadataEvent extends AbstractTopicMetadataEvent { private final String topic; - public TopicMetadataEvent(final String topic, final Timer timer) { - super(Type.TOPIC_METADATA, timer); + public TopicMetadataEvent(final String topic, final long deadlineMs) { + super(Type.TOPIC_METADATA, deadlineMs); this.topic = Objects.requireNonNull(topic); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index 0b988370014a5..327feaa22f69b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -28,8 +26,8 @@ */ public class UnsubscribeEvent extends CompletableApplicationEvent { - public UnsubscribeEvent(final Timer timer) { - super(Type.UNSUBSCRIBE, timer); + public UnsubscribeEvent(final long deadlineMs) { + super(Type.UNSUBSCRIBE, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java index 21e7f3cf6eba1..a93ff9859a58e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -26,7 +24,7 @@ */ public class ValidatePositionsEvent extends CompletableApplicationEvent { - public ValidatePositionsEvent(final Timer timer) { - super(Type.VALIDATE_POSITIONS, timer); + public ValidatePositionsEvent(final long deadlineMs) { + super(Type.VALIDATE_POSITIONS, deadlineMs); } } \ No newline at end of file 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 091009064deef..7004b1b0c7d81 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 @@ -62,6 +62,7 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -175,8 +176,7 @@ public void testSyncCommitEvent() { @ValueSource(booleans = {true, false}) public void testListOffsetsEventIsProcessed(boolean requireTimestamp) { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - Timer timer = time.timer(100); - ApplicationEvent e = new ListOffsetsEvent(timestamps, timer, requireTimestamp); + ApplicationEvent e = new ListOffsetsEvent(timestamps, calculateDeadlineMs(time, 100), requireTimestamp); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); @@ -185,8 +185,7 @@ public void testListOffsetsEventIsProcessed(boolean requireTimestamp) { @Test public void testResetPositionsEventIsProcessed() { - Timer timer = time.timer(100); - ResetPositionsEvent e = new ResetPositionsEvent(timer); + ResetPositionsEvent e = new ResetPositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); @@ -197,8 +196,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - Timer timer = time.timer(100); - ResetPositionsEvent event = new ResetPositionsEvent(timer); + ResetPositionsEvent event = new ResetPositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -207,8 +205,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - Timer timer = time.timer(100); - ValidatePositionsEvent e = new ValidatePositionsEvent(timer); + ValidatePositionsEvent e = new ValidatePositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); @@ -233,8 +230,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - Timer timer = time.timer(Long.MAX_VALUE); - applicationEventsQueue.add(new TopicMetadataEvent("topic", timer)); + applicationEventsQueue.add(new TopicMetadataEvent("topic", Long.MAX_VALUE)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 57e6351b74682..b09f31864f7ef 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,6 +38,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -100,8 +100,7 @@ public void testPrepClosingCommitEvents() { @Test public void testPrepClosingLeaveGroupEvent() { - Timer timer = time.timer(100); - LeaveOnCloseEvent event = new LeaveOnCloseEvent(timer); + LeaveOnCloseEvent event = new LeaveOnCloseEvent(calculateDeadlineMs(time, 100)); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index c9a8529766af9..0ceb77de23f3c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -28,6 +28,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.LinkedBlockingQueue; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -44,7 +45,7 @@ public class CompletableEventReaperTest { public void testExpired() { // Add a new event to the reaper. Timer timer = time.timer(100); - UnsubscribeEvent event = new UnsubscribeEvent(timer); + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(timer)); reaper.add(event); // Without any time passing, we check the reaper and verify that the event is not done amd is still @@ -74,7 +75,7 @@ public void testExpired() { public void testCompleted() { // Add a new event to the reaper. Timer timer = time.timer(100); - UnsubscribeEvent event = new UnsubscribeEvent(timer); + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(timer)); reaper.add(event); // Without any time passing, we check the reaper and verify that the event is not done amd is still @@ -105,8 +106,8 @@ public void testCompleted() { public void testCompletedAndExpired() { // Add two events to the reaper. One event will be completed, the other we will let expire. Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(timer); - UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); reaper.add(event1); reaper.add(event2); @@ -143,8 +144,8 @@ public void testIncompleteQueue() { // Add two events to the queue. BlockingQueue> queue = new LinkedBlockingQueue<>(); Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(timer); - UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); queue.add(event1); queue.add(event2); @@ -181,8 +182,8 @@ public void testIncompleteTracked() { // Add two events for the reaper to track. Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(timer); - UnsubscribeEvent event2 = new UnsubscribeEvent(timer); + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); reaper.add(event1); reaper.add(event2); From 8bec7d7e230354901d9e00ddaf79852a6d946179 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Apr 2024 11:29:48 -0700 Subject: [PATCH 098/130] More updates to remove another usage of Timer to calculate deadline --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 2 +- .../kafka/clients/consumer/internals/events/CommitEvent.java | 5 ----- .../clients/consumer/internals/events/SyncCommitEvent.java | 5 ++--- .../consumer/internals/ConsumerNetworkThreadTest.java | 4 +--- 4 files changed, 4 insertions(+), 12 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 07143092d87df..cdf40f105f57f 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 @@ -1352,7 +1352,7 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, requestTimer); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(requestTimer)); CompletableFuture commitFuture = commit(syncCommitEvent); awaitPendingAsyncCommitsAndExecuteCommitCallbacks(requestTimer, true); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 1e265856a6019..0efc5713f332f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -30,10 +29,6 @@ public abstract class CommitEvent extends CompletableApplicationEvent { */ private final Map offsets; - protected CommitEvent(final Type type, final Map offsets, final Timer timer) { - this(type, offsets, CompletableEvent.calculateDeadlineMs(timer)); - } - public CommitEvent(final Type type, Map offsets, final long deadlineMs) { super(type, deadlineMs); this.offsets = validate(offsets); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java index 87945616ea71b..7dc7a023a8d01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Map; @@ -28,7 +27,7 @@ */ public class SyncCommitEvent extends CommitEvent { - public SyncCommitEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_SYNC, offsets, timer); + public SyncCommitEvent(final Map offsets, final long deadlineMs) { + super(Type.COMMIT_SYNC, offsets, deadlineMs); } } 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 7004b1b0c7d81..0a80a2832263c 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 @@ -41,7 +41,6 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -165,8 +164,7 @@ public void testAsyncCommitEvent() { @Test public void testSyncCommitEvent() { - Timer timer = time.timer(100); - ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), timer); + ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(SyncCommitEvent.class)); From 097522abd6b51bca2407ea0de7009ed6a2d970b4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 29 Apr 2024 11:31:42 -0700 Subject: [PATCH 099/130] Moved SyncCommitEvent to use the user-provided Duration directly --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cdf40f105f57f..e6cc309b5774b 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 @@ -1351,10 +1351,10 @@ public void commitSync(Map offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(requestTimer)); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(time, timeout)); CompletableFuture commitFuture = commit(syncCommitEvent); + Timer requestTimer = time.timer(timeout.toMillis()); awaitPendingAsyncCommitsAndExecuteCommitCallbacks(requestTimer, true); wakeupTrigger.setActiveTask(commitFuture); From 1803dac8e6cce29aebd9101e15ccb47824ee58b6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 10:24:12 -0700 Subject: [PATCH 100/130] Reworked class-level documentation for CompletableEventReaper --- .../consumer/internals/events/CompletableEventReaper.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 06b7e7a4eb97c..0269ef2a3249d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -31,9 +32,10 @@ import java.util.function.Consumer; /** - * The {@code CompletableEventReaper} is responsible for tracking any {@link CompletableEvent}s that were processed, - * making sure to reap them if they complete normally or pass their deadline. This is done so that we enforce an upper - * bound on the amount of time the event logic will execute. + * {@code CompletableEventReaper} is responsible for tracking {@link CompletableEvent time-bound events} and removing + * any that exceed their {@link CompletableEvent#deadlineMs() deadline} (unless they've already completed). This + * mechanism is used by the {@link AsyncKafkaConsumer} to enforce the timeout provided by the user in its API + * calls (e.g. {@link AsyncKafkaConsumer#commitSync(Duration)}). */ public class CompletableEventReaper> { From 71af5749fb62493e6077b0886570f19b50ffb013 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 10:27:08 -0700 Subject: [PATCH 101/130] More typos in CompletableEventReaper comments --- .../consumer/internals/events/CompletableEventReaper.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 0269ef2a3249d..66b9130941df3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -42,7 +42,7 @@ public class CompletableEventReaper> { private final Logger log; /** - * List of tracked events that we are candidates to expire or cancel when reviewed. + * List of tracked events that are candidates for expiration or cancellation. */ private final List tracked; @@ -61,8 +61,8 @@ public void add(T event) { } /** - * This method "completes" any {@link CompletableEvent}s that have either expired or completed normally. So this - * is a two-step process: + * This method performs a two-step process to "complete" {@link CompletableEvent events} that have either expired + * or completed normally: * *

      *
    1. From 63903a1e3085ef7a1aca1f75f78dfd6353960fff Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 10:28:56 -0700 Subject: [PATCH 102/130] Changed the ambiguous process method to processBackgroundEvents --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e6cc309b5774b..9585f6118e5e2 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 @@ -1698,7 +1698,7 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); offsetCommitCallbackInvoker.executeCallbacks(); maybeUpdateSubscriptionMetadata(); - process(backgroundEventProcessor); + processBackgroundEvents(backgroundEventProcessor); return updateFetchPositions(timer); } @@ -1830,7 +1830,7 @@ private void subscribeInternal(Collection topics, Optional processor) { + private boolean processBackgroundEvents(EventProcessor processor) { AtomicReference firstError = new AtomicReference<>(); LinkedList events = new LinkedList<>(); @@ -1903,7 +1903,7 @@ T processBackgroundEvents(EventProcessor eventProcessor, log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - boolean hadEvents = process(eventProcessor); + boolean hadEvents = processBackgroundEvents(eventProcessor); try { if (future.isDone()) { From 909f4dc24b8d908bfd89d75591149e2578b9812c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 10:35:45 -0700 Subject: [PATCH 103/130] Changes to processBackgroundEvents comments --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 9585f6118e5e2..71eba1929a7df 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 @@ -1880,13 +1880,14 @@ private boolean processBackgroundEvents(EventProcessor processo * As an example, take {@link #unsubscribe()}. To start unsubscribing, the application thread enqueues an * {@link UnsubscribeEvent} on the application event queue. That event will eventually trigger the * rebalancing logic in the background thread. Critically, as part of this rebalancing work, the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} callback needs to be invoked. However, + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} callback needs to be invoked for any + * partitions the consumer owns. However, * this callback must be executed on the application thread. To achieve this, the background thread enqueues a * {@link ConsumerRebalanceListenerCallbackNeededEvent} on its background event queue. That event queue is * periodically queried by the application thread to see if there's work to be done. When the application thread * sees {@link ConsumerRebalanceListenerCallbackNeededEvent}, it is processed, and then a * {@link ConsumerRebalanceListenerCallbackCompletedEvent} is then enqueued by the application thread on the - * background event queue. Moments later, the background thread will see that event, process it, and continue + * application event queue. Moments later, the background thread will see that event, process it, and continue * execution of the rebalancing logic. The rebalancing logic cannot complete until the * {@link ConsumerRebalanceListener} callback is performed. * From 1aa4c31ec42744dffc8c9f21353c5fb751f25ff7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 10:37:32 -0700 Subject: [PATCH 104/130] Removed logging from processBackgroundEvents to simplify logic --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 71eba1929a7df..d6da18add4331 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 @@ -1911,18 +1911,14 @@ T processBackgroundEvents(EventProcessor eventProcessor, // If the event is done (either successfully or otherwise), go ahead and attempt to return // without waiting. We use the ConsumerUtils.getResult() method here to handle the conversion // of the exception types. - T result = ConsumerUtils.getResult(future); - log.trace("Future {} completed successfully", future); - return result; + return ConsumerUtils.getResult(future); } else if (!hadEvents) { // If the above processing yielded no events, then let's sit tight for a bit to allow the // background thread to either a) finish the task, or b) populate the background event // queue with things to process in our next loop. Timer pollInterval = time.timer(100L); log.trace("Waiting {} ms for future {} to complete", pollInterval.remainingMs(), future); - T result = ConsumerUtils.getResult(future, pollInterval); - log.trace("Future {} completed successfully", future); - return result; + return ConsumerUtils.getResult(future, pollInterval); } } catch (TimeoutException e) { // Ignore this as we will retry the event until the timeout expires. From d8959db367bcec40510e5707a93ac349818c46b9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:19:50 -0700 Subject: [PATCH 105/130] Changed the signature of reapIncomplete to remove boilerplate for callers --- .../consumer/internals/AsyncKafkaConsumer.java | 14 ++------------ .../consumer/internals/ConsumerNetworkThread.java | 11 +---------- .../internals/events/CompletableEventReaper.java | 5 ++++- .../events/CompletableEventReaperTest.java | 4 ++-- 4 files changed, 9 insertions(+), 25 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 d6da18add4331..b077d47cf934c 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 @@ -1231,18 +1231,8 @@ private void close(Duration timeout, boolean swallowException) { // close() can be called from inside one of the constructors. In that case, it's possible that neither // the reaper nor the background event queue were constructed, so check them first to avoid NPE. - if (backgroundEventReaper != null && backgroundEventQueue != null) { - // Copy over the completable events to a separate list, then reap any incomplete - // events on that list. - LinkedList allEvents = new LinkedList<>(); - backgroundEventQueue.drainTo(allEvents); - List> completableEvents = allEvents - .stream() - .filter(e -> e instanceof CompletableBackgroundEvent) - .map(e -> (CompletableBackgroundEvent) e) - .collect(Collectors.toList()); - backgroundEventReaper.reapIncomplete(completableEvents); - } + if (backgroundEventReaper != null && backgroundEventQueue != null) + backgroundEventReaper.reapIncomplete(backgroundEventQueue); closeQuietly(interceptors, "consumer interceptors", firstException); closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); 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 fc840f63097bf..71d1f6c5c9362 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 @@ -39,7 +39,6 @@ import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; -import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; import static org.apache.kafka.common.utils.Utils.closeQuietly; @@ -310,15 +309,7 @@ void cleanup() { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { sendUnsentRequests(timer); - - LinkedList allEvents = new LinkedList<>(); - applicationEventQueue.drainTo(allEvents); - List> completableEvents = allEvents - .stream() - .filter(e -> e instanceof CompletableApplicationEvent) - .map(e -> (CompletableApplicationEvent) e) - .collect(Collectors.toList()); - applicationEventReaper.reapIncomplete(completableEvents); + applicationEventReaper.reapIncomplete(applicationEventQueue); closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 66b9130941df3..d18690cdb695b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -123,7 +123,7 @@ public void reapExpiredAndCompleted(long currentTimeMs) { * * @param events Events from a queue that have not yet been tracked that also need to be reviewed */ - public void reapIncomplete(Collection events) { + public void reapIncomplete(Collection events) { log.trace("Reaping incomplete events"); Objects.requireNonNull(events, "Event queue to reap must be non-null"); @@ -140,8 +140,11 @@ public void reapIncomplete(Collection events) { tracked.clear(); events.stream() + .filter(e -> e instanceof CompletableEvent) + .map(e -> (CompletableEvent) e) .filter(e -> !e.future().isDone()) .forEach(cancelEvent); + events.clear(); log.trace("Finished reaping incomplete events"); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index 0ceb77de23f3c..210b289ae839d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -169,10 +169,10 @@ public void testIncompleteQueue() { assertTrue(event2.future().isCancelled()); assertThrows(CancellationException.class, () -> ConsumerUtils.getResult(event2.future())); - // Because the events aren't tracked in the reaper *and* the queue isn't drained as part of the + // Because the events aren't tracked in the reaper *and* the queue is cleared as part of the // cancellation process, our data structures should both be the same as above. assertEquals(0, reaper.size()); - assertEquals(2, queue.size()); + assertEquals(0, queue.size()); } @Test From 0248c92f22f45ace0d151daeea93a1f6f31016d0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:21:40 -0700 Subject: [PATCH 106/130] Added back 'final' parameter qualifier --- .../kafka/clients/consumer/internals/events/CommitEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 0efc5713f332f..0b1cb4998e1f6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -29,7 +29,7 @@ public abstract class CommitEvent extends CompletableApplicationEvent { */ private final Map offsets; - public CommitEvent(final Type type, Map offsets, final long deadlineMs) { + public CommitEvent(final Type type, final Map offsets, final long deadlineMs) { super(type, deadlineMs); this.offsets = validate(offsets); } From 290b081d0fbc35fe47dd758aa09001915c3f8544 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:22:56 -0700 Subject: [PATCH 107/130] Made the abstract CommitEvent class' constructor protected (again) --- .../kafka/clients/consumer/internals/events/CommitEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 0b1cb4998e1f6..dc863b0ee659a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -29,7 +29,7 @@ public abstract class CommitEvent extends CompletableApplicationEvent { */ private final Map offsets; - public CommitEvent(final Type type, final Map offsets, final long deadlineMs) { + protected CommitEvent(final Type type, final Map offsets, final long deadlineMs) { super(type, deadlineMs); this.offsets = validate(offsets); } From 987dc4ab6e670d7a577cc46525b909e0bfbe3d45 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:26:02 -0700 Subject: [PATCH 108/130] Updated CompletableEvent.future() documentation to reflect using completeExceptionally(new CancellationException()) vs. cancel(true) --- .../clients/consumer/internals/events/CompletableEvent.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 017ab67755c4a..9af7bf23776b0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -56,9 +56,9 @@ public interface CompletableEvent { * {@link CompletableFuture#completeExceptionally(Throwable)}. *
    2. *
    3. - * Cancelled: when an event remains incomplete when the consumer closes, the future will be - * {@link CompletableFuture#cancel(boolean) cancelled}. Attempts to {@link Future#get() get the result} - * of the processing will throw a {@link CancellationException}. + * Cancelled: when an event remains incomplete when the consumer closes, an + * instance of {@link CancellationException} should be created and passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. *
    4. * * From 27ed6fcbea2db75aa2a148f6b5a4d9a2ddb5daf2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:37:12 -0700 Subject: [PATCH 109/130] Removed MockTime instances created with non-default arguments --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 1 - .../consumer/internals/events/CompletableEventReaperTest.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 0ec5119a3d4ba..8cce3fb847eab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -987,7 +987,6 @@ public void testResetUsingAutoResetPolicy(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) public void testOffsetIsValidAfterSeek(GroupProtocol groupProtocol) { - Time time = new MockTime(1); SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST); ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index 210b289ae839d..aabbd7742215b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -38,7 +38,7 @@ public class CompletableEventReaperTest { private final LogContext logContext = new LogContext(); - private final Time time = new MockTime(0, 0, 0); + private final Time time = new MockTime(); private final CompletableEventReaper> reaper = new CompletableEventReaper<>(logContext); @Test From b54c8172c064c3b71333a012d3117f4a57cd3a27 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:39:49 -0700 Subject: [PATCH 110/130] Minor formatting change per feedback --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b077d47cf934c..64f08e9664ca3 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 @@ -1127,8 +1127,7 @@ private Map beginningOrEndOffset(Collection offsetAndTimestampMap; - offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent); + offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); return offsetAndTimestampMap.entrySet() .stream() .collect(Collectors.toMap( From 26d64e849436c7c096b14f18d8ce408ccc545683 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 11:46:49 -0700 Subject: [PATCH 111/130] Restored modified comment to ApplicationEventHandler.addAndGet() --- .../consumer/internals/events/ApplicationEventHandler.java | 5 +++++ 1 file changed, 5 insertions(+) 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 f1931ff292ad8..057b61fcea0e3 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 @@ -30,6 +30,7 @@ import java.time.Duration; import java.util.Objects; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Future; import java.util.function.Supplier; /** @@ -96,6 +97,10 @@ public long maximumTimeToWait() { * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will * return the result value upon successful completion; otherwise throws an error. * + *

      + * + * See {@link ConsumerUtils#getResult(Future)} for more details. + * * @param event A {@link CompletableApplicationEvent} created by the polling thread * @return Value that is the result of the event * @param Type of return value of the event From 5d212c256adc223ad139fdad3d082aa8b99adc15 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 1 May 2024 13:32:52 -0700 Subject: [PATCH 112/130] Removed unnecessary import that broke the build --- .../clients/consumer/internals/events/CompletableEvent.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 9af7bf23776b0..e2714f243d3eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -22,7 +22,6 @@ import java.time.Duration; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; import java.util.concurrent.TimeoutException; import static java.util.Objects.requireNonNull; From a92e25e2a18fe5091ab742e4e5e3ff86f9d3ccf9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 13 May 2024 17:25:49 -0700 Subject: [PATCH 113/130] Using a Collection instead of BlockingQueue for simplicity --- .../internals/events/CompletableEventReaperTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index aabbd7742215b..7c7860afff232 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -24,9 +24,9 @@ import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.Test; -import java.util.concurrent.BlockingQueue; +import java.util.ArrayList; +import java.util.Collection; import java.util.concurrent.CancellationException; -import java.util.concurrent.LinkedBlockingQueue; import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -142,7 +142,7 @@ public void testCompletedAndExpired() { @Test public void testIncompleteQueue() { // Add two events to the queue. - BlockingQueue> queue = new LinkedBlockingQueue<>(); + Collection> queue = new ArrayList<>(); Timer timer = time.timer(100); UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); @@ -178,7 +178,7 @@ public void testIncompleteQueue() { @Test public void testIncompleteTracked() { // This queue is just here to test the case where the queue is empty. - BlockingQueue> queue = new LinkedBlockingQueue<>(); + Collection> queue = new ArrayList<>(); // Add two events for the reaper to track. Timer timer = time.timer(100); From 83687187db40cf0178b6a5215589de2798c0d93c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 13 May 2024 17:34:55 -0700 Subject: [PATCH 114/130] Removed use of Timer in CompletableEventReaperTest --- .../events/CompletableEventReaperTest.java | 41 ++++++++----------- 1 file changed, 17 insertions(+), 24 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index 7c7860afff232..d4621d64b913b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -44,8 +43,8 @@ public class CompletableEventReaperTest { @Test public void testExpired() { // Add a new event to the reaper. - Timer timer = time.timer(100); - UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(timer)); + long timeoutMs = 100; + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); reaper.add(event); // Without any time passing, we check the reaper and verify that the event is not done amd is still @@ -55,9 +54,7 @@ public void testExpired() { assertEquals(1, reaper.size()); // Sleep for at least 1 ms. *more* than the timeout so that the event is considered expired. - time.sleep(timer.timeoutMs() + 1); - timer.update(time.milliseconds()); - assertEquals(0, timer.remainingMs()); + time.sleep(timeoutMs + 1); // However, until we actually invoke the reaper, the event isn't complete and is still being tracked. assertFalse(event.future().isDone()); @@ -74,8 +71,8 @@ public void testExpired() { @Test public void testCompleted() { // Add a new event to the reaper. - Timer timer = time.timer(100); - UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(timer)); + long timeoutMs = 100; + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); reaper.add(event); // Without any time passing, we check the reaper and verify that the event is not done amd is still @@ -91,9 +88,7 @@ public void testCompleted() { assertEquals(1, reaper.size()); // To ensure we don't accidentally expire an event that completed normally, sleep past the timeout. - time.sleep(timer.timeoutMs() + 1); - timer.update(time.milliseconds()); - assertEquals(0, timer.remainingMs()); + time.sleep(timeoutMs + 1); // Call the reaper and validate that the event is not considered expired, but is still no longer tracked. reaper.reapExpiredAndCompleted(time.milliseconds()); @@ -105,9 +100,9 @@ public void testCompleted() { @Test public void testCompletedAndExpired() { // Add two events to the reaper. One event will be completed, the other we will let expire. - Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); - UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); reaper.add(event1); reaper.add(event2); @@ -122,9 +117,7 @@ public void testCompletedAndExpired() { event1.future().complete(null); assertTrue(event1.future().isDone()); - time.sleep(timer.timeoutMs() + 1); - timer.update(time.milliseconds()); - assertEquals(0, timer.remainingMs()); + time.sleep(timeoutMs + 1); // Though the first event is completed, it's still being tracked, along with the second expired event. assertEquals(2, reaper.size()); @@ -141,11 +134,11 @@ public void testCompletedAndExpired() { @Test public void testIncompleteQueue() { - // Add two events to the queue. + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + Collection> queue = new ArrayList<>(); - Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); - UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); queue.add(event1); queue.add(event2); @@ -181,9 +174,9 @@ public void testIncompleteTracked() { Collection> queue = new ArrayList<>(); // Add two events for the reaper to track. - Timer timer = time.timer(100); - UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(timer)); - UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(timer)); + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); reaper.add(event1); reaper.add(event2); From 40cc477c4bb63d0a0d0e9e41b95279b151fabcfb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 11:29:10 -0700 Subject: [PATCH 115/130] Minor refactoring to avoid generics --- .../consumer/internals/AsyncKafkaConsumer.java | 14 +++++++------- .../consumer/internals/ConsumerNetworkThread.java | 10 +++++----- .../internals/events/CompletableEventReaper.java | 6 +++--- .../events/CompletableEventReaperTest.java | 2 +- 4 files changed, 16 insertions(+), 16 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 da3738905acb1..2b8b9ea0d35f2 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 @@ -47,7 +47,7 @@ import org.apache.kafka.clients.consumer.internals.events.CommitEvent; import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; @@ -219,7 +219,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final String clientId; private final BlockingQueue backgroundEventQueue; private final BackgroundEventProcessor backgroundEventProcessor; - private final CompletableEventReaper> backgroundEventReaper; + private final CompletableEventReaper backgroundEventReaper; private final Deserializers deserializers; /** @@ -372,7 +372,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.backgroundEventProcessor = new BackgroundEventProcessor( rebalanceListenerInvoker ); - this.backgroundEventReaper = new CompletableEventReaper<>(logContext); + this.backgroundEventReaper = new CompletableEventReaper(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -436,7 +436,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.time = time; this.backgroundEventQueue = backgroundEventQueue; this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); - this.backgroundEventReaper = new CompletableEventReaper<>(logContext); + this.backgroundEventReaper = new CompletableEventReaper(logContext); this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; @@ -545,7 +545,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { networkClientDelegateSupplier, requestManagersSupplier); this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); - this.backgroundEventReaper = new CompletableEventReaper<>(logContext); + this.backgroundEventReaper = new CompletableEventReaper(logContext); } // auxiliary interface for testing @@ -1828,8 +1828,8 @@ private boolean processBackgroundEvents(EventProcessor processo for (BackgroundEvent event : events) { try { - if (event instanceof CompletableBackgroundEvent) - backgroundEventReaper.add((CompletableBackgroundEvent) event); + if (event instanceof CompletableEvent) + backgroundEventReaper.add((CompletableEvent) event); processor.process(event); } catch (Throwable t) { 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 71d1f6c5c9362..a1651157d7575 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,7 +20,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; @@ -55,7 +55,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private final Time time; private final Logger log; private final BlockingQueue applicationEventQueue; - private final CompletableEventReaper> applicationEventReaper; + private final CompletableEventReaper applicationEventReaper; private final Supplier applicationEventProcessorSupplier; private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; @@ -77,7 +77,7 @@ public ConsumerNetworkThread(LogContext logContext, this.time = time; this.log = logContext.logger(getClass()); this.applicationEventQueue = applicationEventQueue; - this.applicationEventReaper = new CompletableEventReaper<>(logContext); + this.applicationEventReaper = new CompletableEventReaper(logContext); this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; this.networkClientDelegateSupplier = networkClientDelegateSupplier; this.requestManagersSupplier = requestManagersSupplier; @@ -163,8 +163,8 @@ private void processApplicationEvents() { for (ApplicationEvent event : events) { try { - if (event instanceof CompletableApplicationEvent) - applicationEventReaper.add((CompletableApplicationEvent) event); + if (event instanceof CompletableEvent) + applicationEventReaper.add((CompletableEvent) event); applicationEventProcessor.process(event); } catch (Throwable t) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index d18690cdb695b..2a2a0205f54c4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -37,14 +37,14 @@ * mechanism is used by the {@link AsyncKafkaConsumer} to enforce the timeout provided by the user in its API * calls (e.g. {@link AsyncKafkaConsumer#commitSync(Duration)}). */ -public class CompletableEventReaper> { +public class CompletableEventReaper { private final Logger log; /** * List of tracked events that are candidates for expiration or cancellation. */ - private final List tracked; + private final List> tracked; public CompletableEventReaper(LogContext logContext) { this.log = logContext.logger(CompletableEventReaper.class); @@ -56,7 +56,7 @@ public CompletableEventReaper(LogContext logContext) { * * @param event Event to track */ - public void add(T event) { + public void add(CompletableEvent event) { tracked.add(Objects.requireNonNull(event, "Event to track must be non-null")); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index d4621d64b913b..bc1aee3412a61 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -38,7 +38,7 @@ public class CompletableEventReaperTest { private final LogContext logContext = new LogContext(); private final Time time = new MockTime(); - private final CompletableEventReaper> reaper = new CompletableEventReaper<>(logContext); + private final CompletableEventReaper reaper = new CompletableEventReaper(logContext); @Test public void testExpired() { From 1234c1e2659ac29d34140fd59e78768c03a35262 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 12:30:06 -0700 Subject: [PATCH 116/130] Added unit test to ConsumerNetworkThread and minor refactoring/renaming --- .../internals/AsyncKafkaConsumer.java | 4 +- .../internals/ConsumerNetworkThread.java | 8 ++- .../internals/events/CompletableEvent.java | 9 +--- .../events/CompletableEventReaper.java | 54 +++++++++++-------- .../internals/ConsumerNetworkThreadTest.java | 44 +++++++++++++++ .../events/CompletableEventReaperTest.java | 32 +++++------ 6 files changed, 100 insertions(+), 51 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 2b8b9ea0d35f2..8c5f7d8f9c652 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 @@ -1233,7 +1233,7 @@ private void close(Duration timeout, boolean swallowException) { // close() can be called from inside one of the constructors. In that case, it's possible that neither // the reaper nor the background event queue were constructed, so check them first to avoid NPE. if (backgroundEventReaper != null && backgroundEventQueue != null) - backgroundEventReaper.reapIncomplete(backgroundEventQueue); + backgroundEventReaper.reap(backgroundEventQueue); closeQuietly(interceptors, "consumer interceptors", firstException); closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); @@ -1840,7 +1840,7 @@ private boolean processBackgroundEvents(EventProcessor processo } } - backgroundEventReaper.reapExpiredAndCompleted(time.milliseconds()); + backgroundEventReaper.reap(time.milliseconds()); if (firstError.get() != null) throw firstError.get(); 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 a1651157d7575..48e287ff18749 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 @@ -179,7 +179,7 @@ private void processApplicationEvents() { * is given least one attempt to satisfy any network requests before checking if a timeout has expired. */ private void reapExpiredApplicationEvents(long currentTimeMs) { - applicationEventReaper.reapExpiredAndCompleted(currentTimeMs); + applicationEventReaper.reap(currentTimeMs); } /** @@ -309,11 +309,15 @@ void cleanup() { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { sendUnsentRequests(timer); - applicationEventReaper.reapIncomplete(applicationEventQueue); + applicationEventReaper.reap(applicationEventQueue); closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); log.debug("Closed the consumer network thread"); } } + + CompletableEventReaper completableEventReaper() { + return applicationEventReaper; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index e2714f243d3eb..20231b0f99a10 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.utils.Timer; import java.time.Duration; -import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeoutException; @@ -52,12 +51,8 @@ public interface CompletableEvent { *

    5. * Timeout: when the time spent executing the event logic exceeds the {@link #deadlineMs() deadline}, an * instance of {@link TimeoutException} should be created and passed to - * {@link CompletableFuture#completeExceptionally(Throwable)}. - *
    6. - *
    7. - * Cancelled: when an event remains incomplete when the consumer closes, an - * instance of {@link CancellationException} should be created and passed to - * {@link CompletableFuture#completeExceptionally(Throwable)}. + * {@link CompletableFuture#completeExceptionally(Throwable)}. This also occurs when an event remains + * incomplete when the consumer closes. *
    8. * * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 2a2a0205f54c4..e9cef49a9ba8f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -26,9 +26,7 @@ import java.util.Collection; import java.util.List; import java.util.Objects; -import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; import java.util.function.Consumer; /** @@ -42,7 +40,7 @@ public class CompletableEventReaper { private final Logger log; /** - * List of tracked events that are candidates for expiration or cancellation. + * List of tracked events that are candidates for expiration. */ private final List> tracked; @@ -83,22 +81,25 @@ public void add(CompletableEvent event) { * @param currentTimeMs Current time with which to compare against the * {@link CompletableEvent#deadlineMs() expiration time} */ - public void reapExpiredAndCompleted(long currentTimeMs) { + public void reap(long currentTimeMs) { log.trace("Reaping expired events"); - Consumer> timeoutEvent = e -> { - TimeoutException error = new TimeoutException(String.format("%s could not be completed within its timeout", e.getClass().getSimpleName())); - long pastDueMs = currentTimeMs - e.deadlineMs(); - log.debug("Completing event {} exceptionally since it expired {} ms ago", e, pastDueMs); - CompletableFuture f = e.future(); - f.completeExceptionally(error); + Consumer> expireEvent = event -> { + long pastDueMs = currentTimeMs - event.deadlineMs(); + TimeoutException error = new TimeoutException(String.format("%s was %s ms past its expiration of %s", event.getClass().getSimpleName(), pastDueMs, event.deadlineMs())); + + if (event.future().completeExceptionally(error)) { + log.debug("Event {} completed exceptionally since its expiration of {} passed {} ms ago", event, event.deadlineMs(), pastDueMs); + } else { + log.trace("Event {} not completed exceptionally since it was previously completed", event); + } }; // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. tracked.stream() .filter(e -> !e.future().isDone()) .filter(e -> currentTimeMs > e.deadlineMs()) - .forEach(timeoutEvent); + .forEach(expireEvent); // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. tracked.removeIf(e -> e.future().isDone()); @@ -108,13 +109,14 @@ public void reapExpiredAndCompleted(long currentTimeMs) { /** * It is possible for the {@link AsyncKafkaConsumer#close() consumer to close} before completing the processing of - * all the events in the queue. In this case, we need to {@link Future#cancel(boolean) cancel} any remaining events. + * all the events in the queue. In this case, we need to + * {@link CompletableFuture#completeExceptionally(Throwable) expire} any remaining events. * *

      * * Check each of the {@link #add(CompletableEvent) previously-added} {@link CompletableEvent completable events}, - * and for any that are incomplete, {@link Future#cancel(boolean) cancel} them. Also check the core event queue - * for any incomplete events and likewise cancel them. + * and for any that are incomplete, expire them. Also check the core event queue for any incomplete events and + * likewise expire them. * *

      * @@ -123,33 +125,41 @@ public void reapExpiredAndCompleted(long currentTimeMs) { * * @param events Events from a queue that have not yet been tracked that also need to be reviewed */ - public void reapIncomplete(Collection events) { + public void reap(Collection events) { log.trace("Reaping incomplete events"); Objects.requireNonNull(events, "Event queue to reap must be non-null"); - Consumer> cancelEvent = e -> { - log.debug("Canceling event {} since the consumer is closing", e); - CompletableFuture f = e.future(); - f.completeExceptionally(new CancellationException("Canceling event since the consumer is closing")); + Consumer> expireEvent = event -> { + TimeoutException error = new TimeoutException(String.format("%s could not be completed before the consumer closed", event.getClass().getSimpleName())); + + if (event.future().completeExceptionally(error)) { + log.debug("Event {} completed exceptionally since the consumer is closing", event); + } else { + log.trace("Event {} not completed exceptionally since it was completed prior to the consumer closing", event); + } }; tracked.stream() .filter(e -> !e.future().isDone()) - .forEach(cancelEvent); + .forEach(expireEvent); tracked.clear(); events.stream() .filter(e -> e instanceof CompletableEvent) .map(e -> (CompletableEvent) e) .filter(e -> !e.future().isDone()) - .forEach(cancelEvent); + .forEach(expireEvent); events.clear(); log.trace("Finished reaping incomplete events"); } - int size() { + public int size() { return tracked.size(); } + + public boolean contains(CompletableEvent event) { + return event != null && tracked.contains(event); + } } 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 0a80a2832263c..44bde0892f9e7 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 @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.PollEvent; @@ -32,6 +33,7 @@ import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorRequest; @@ -66,6 +68,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; @@ -301,6 +304,47 @@ void testEnsureEventsAreCompleted() { assertTrue(applicationEventsQueue.isEmpty()); } + @Test + void testReaperExpiresExpiredEvents() { + CompletableEventReaper completableEventReaper = consumerNetworkThread.completableEventReaper(); + long event1TimeoutMs = 100; + long event2TimeoutMs = 200; + SyncCommitEvent event1 = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, event1TimeoutMs)); + SyncCommitEvent event2 = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, event2TimeoutMs)); + applicationEventsQueue.add(event1); + applicationEventsQueue.add(event2); + consumerNetworkThread.runOnce(); + + // Make sure both events have been moved from the event queue to the reaper's "tracked" list. + assertFalse(applicationEventsQueue.contains(event1)); + assertFalse(applicationEventsQueue.contains(event2)); + assertTrue(completableEventReaper.contains(event1)); + assertTrue(completableEventReaper.contains(event2)); + assertEquals(2, completableEventReaper.size()); + + // Sleep long enough for the first event to have expired. + time.sleep(event1TimeoutMs + 1); + + consumerNetworkThread.runOnce(); + + // Validate that the first event was expired, but the second continues to be tracked + assertTrue(event1.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future())); + assertFalse(completableEventReaper.contains(event1)); + + assertTrue(completableEventReaper.contains(event2)); + assertFalse(event2.future().isDone()); + assertEquals(1, completableEventReaper.size()); + + // The cleanup will trigger the reaper's + consumerNetworkThread.cleanup(); + + assertTrue(event2.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); + assertFalse(consumerNetworkThread.completableEventReaper().contains(event2)); + assertEquals(0, consumerNetworkThread.completableEventReaper().size()); + } + private void prepareOffsetCommitRequest(final Map expectedOffsets, final Errors error, final boolean disconnected) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java index bc1aee3412a61..460b7368fb9e6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.concurrent.CancellationException; import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -49,7 +48,7 @@ public void testExpired() { // Without any time passing, we check the reaper and verify that the event is not done amd is still // being tracked. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertFalse(event.future().isDone()); assertEquals(1, reaper.size()); @@ -62,7 +61,7 @@ public void testExpired() { // Call the reaper and validate that the event is now "done" (expired), the correct exception type is // thrown, and the event is no longer tracked. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertTrue(event.future().isDone()); assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event.future())); assertEquals(0, reaper.size()); @@ -77,7 +76,7 @@ public void testCompleted() { // Without any time passing, we check the reaper and verify that the event is not done amd is still // being tracked. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertFalse(event.future().isDone()); assertEquals(1, reaper.size()); @@ -91,7 +90,7 @@ public void testCompleted() { time.sleep(timeoutMs + 1); // Call the reaper and validate that the event is not considered expired, but is still no longer tracked. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertTrue(event.future().isDone()); assertNull(ConsumerUtils.getResult(event.future())); assertEquals(0, reaper.size()); @@ -108,7 +107,7 @@ public void testCompletedAndExpired() { // Without any time passing, we check the reaper and verify that the event is not done amd is still // being tracked. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertFalse(event1.future().isDone()); assertFalse(event2.future().isDone()); assertEquals(2, reaper.size()); @@ -124,7 +123,7 @@ public void testCompletedAndExpired() { // Validate that the first (completed) event is not expired, but the second one is expired. In either case, // both should be completed and neither should be tracked anymore. - reaper.reapExpiredAndCompleted(time.milliseconds()); + reaper.reap(time.milliseconds()); assertTrue(event1.future().isDone()); assertTrue(event2.future().isDone()); assertNull(ConsumerUtils.getResult(event1.future())); @@ -150,17 +149,16 @@ public void testIncompleteQueue() { assertEquals(2, queue.size()); // Go ahead and reap the incomplete from the queue. - reaper.reapIncomplete(queue); + reaper.reap(queue); - // The first event was completed, so we didn't cancel it in the reaper. + // The first event was completed, so we didn't expire it in the reaper. assertTrue(event1.future().isDone()); assertFalse(event1.future().isCancelled()); assertNull(ConsumerUtils.getResult(event1.future())); - // The second event was incomplete, so it was canceled. - assertTrue(event2.future().isDone()); - assertTrue(event2.future().isCancelled()); - assertThrows(CancellationException.class, () -> ConsumerUtils.getResult(event2.future())); + // The second event was incomplete, so it was expired. + assertTrue(event2.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); // Because the events aren't tracked in the reaper *and* the queue is cleared as part of the // cancellation process, our data structures should both be the same as above. @@ -187,18 +185,16 @@ public void testIncompleteTracked() { assertEquals(2, reaper.size()); // Go ahead and reap the incomplete events. Both sets should be zero after that. - reaper.reapIncomplete(queue); + reaper.reap(queue); assertEquals(0, reaper.size()); assertEquals(0, queue.size()); // The first event was completed, so we didn't cancel it in the reaper. assertTrue(event1.future().isDone()); - assertFalse(event1.future().isCancelled()); assertNull(ConsumerUtils.getResult(event1.future())); // The second event was incomplete, so it was canceled. - assertTrue(event2.future().isDone()); - assertTrue(event2.future().isCancelled()); - assertThrows(CancellationException.class, () -> ConsumerUtils.getResult(event2.future())); + assertTrue(event2.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); } } From b3b93a9dde1c0521bedbd09287e75eb66467a429 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 15:40:42 -0700 Subject: [PATCH 117/130] Added test to AsyncKafkaConsumerTest --- .../internals/AsyncKafkaConsumer.java | 16 +++++++-- .../internals/ConsumerNetworkThread.java | 7 ++-- .../events/ApplicationEventHandler.java | 2 ++ .../internals/AsyncKafkaConsumerTest.java | 35 +++++++++++++++++-- .../internals/ConsumerNetworkThreadTest.java | 19 +++++----- .../internals/ConsumerTestBuilder.java | 4 +++ 6 files changed, 65 insertions(+), 18 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 8c5f7d8f9c652..ed6f6940632c3 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 @@ -266,6 +266,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { valueDeserializer, Time.SYSTEM, ApplicationEventHandler::new, + CompletableEventReaper::new, FetchCollector::new, ConsumerMetadata::new, new LinkedBlockingQueue<>() @@ -278,6 +279,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final Deserializer valueDeserializer, final Time time, final ApplicationEventHandlerFactory applicationEventHandlerFactory, + final CompletableEventReaperFactory backgroundEventReaperFactory, final FetchCollectorFactory fetchCollectorFactory, final ConsumerMetadataFactory metadataFactory, final LinkedBlockingQueue backgroundEventQueue) { @@ -359,6 +361,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { logContext, time, applicationEventQueue, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); @@ -372,7 +375,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.backgroundEventProcessor = new BackgroundEventProcessor( rebalanceListenerInvoker ); - this.backgroundEventReaper = new CompletableEventReaper(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -417,6 +419,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { Time time, ApplicationEventHandler applicationEventHandler, BlockingQueue backgroundEventQueue, + CompletableEventReaper backgroundEventReaper, ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, Metrics metrics, SubscriptionState subscriptions, @@ -436,7 +439,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.time = time; this.backgroundEventQueue = backgroundEventQueue; this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); - this.backgroundEventReaper = new CompletableEventReaper(logContext); + this.backgroundEventReaper = backgroundEventReaper; this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; @@ -541,6 +544,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); @@ -555,6 +559,7 @@ ApplicationEventHandler build( final LogContext logContext, final Time time, final BlockingQueue applicationEventQueue, + final CompletableEventReaper applicationEventReaper, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier @@ -562,6 +567,13 @@ ApplicationEventHandler build( } + // auxiliary interface for testing + interface CompletableEventReaperFactory { + + CompletableEventReaper build(final LogContext logContext); + + } + // auxiliary interface for testing interface FetchCollectorFactory { 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 48e287ff18749..adee6594603bb 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 @@ -70,6 +70,7 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { public ConsumerNetworkThread(LogContext logContext, Time time, BlockingQueue applicationEventQueue, + CompletableEventReaper applicationEventReaper, Supplier applicationEventProcessorSupplier, Supplier networkClientDelegateSupplier, Supplier requestManagersSupplier) { @@ -77,7 +78,7 @@ public ConsumerNetworkThread(LogContext logContext, this.time = time; this.log = logContext.logger(getClass()); this.applicationEventQueue = applicationEventQueue; - this.applicationEventReaper = new CompletableEventReaper(logContext); + this.applicationEventReaper = applicationEventReaper; this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; this.networkClientDelegateSupplier = networkClientDelegateSupplier; this.requestManagersSupplier = requestManagersSupplier; @@ -316,8 +317,4 @@ void cleanup() { log.debug("Closed the consumer network thread"); } } - - CompletableEventReaper completableEventReaper() { - return applicationEventReaper; - } } 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 057b61fcea0e3..ffda33cabdb9d 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 @@ -47,6 +47,7 @@ public class ApplicationEventHandler implements Closeable { public ApplicationEventHandler(final LogContext logContext, final Time time, final BlockingQueue applicationEventQueue, + final CompletableEventReaper applicationEventReaper, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier) { @@ -55,6 +56,7 @@ public ApplicationEventHandler(final LogContext logContext, this.networkThread = new ConsumerNetworkThread(logContext, time, applicationEventQueue, + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); 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 e7c98ee62677c..2cfca42e5552d 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 @@ -39,6 +39,7 @@ import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; @@ -155,6 +156,7 @@ public class AsyncKafkaConsumerTest { private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + private final CompletableEventReaper backgroundEventReaper = new CompletableEventReaper(new LogContext()); @AfterEach public void resetAll() { @@ -195,7 +197,8 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (a, b, c, d, e, f) -> applicationEventHandler, + (a, b, c, d, e, f, g) -> applicationEventHandler, + (a) -> backgroundEventReaper, (a, b, c, d, e, f, g) -> fetchCollector, (a, b, c, d) -> metadata, backgroundEventQueue @@ -220,6 +223,7 @@ private AsyncKafkaConsumer newConsumer( time, applicationEventHandler, backgroundEventQueue, + backgroundEventReaper, rebalanceListenerInvoker, new Metrics(), subscriptions, @@ -1848,7 +1852,34 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { } assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } - + + @Test + void testReaperExpiresExpiredEvents() { + consumer = newConsumer(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final SortedSet partitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + partitions.add(tp); + + consumer.subscribe(Collections.singletonList(topicName), new CounterConsumerRebalanceListener()); + + final ConsumerRebalanceListenerCallbackNeededEvent event1 = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, partitions); + final ConsumerRebalanceListenerCallbackNeededEvent event2 = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_ASSIGNED, partitions); + backgroundEventReaper.add(event1); + backgroundEventQueue.add(event2); + + assertEquals(1, backgroundEventReaper.size()); + assertEquals(1, backgroundEventQueue.size()); + + consumer.close(); + + assertEquals(0, backgroundEventReaper.size()); + assertEquals(0, backgroundEventQueue.size()); + assertTrue(event1.future().isCompletedExceptionally()); + assertTrue(event2.future().isCompletedExceptionally()); + } + private Map mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); 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 44bde0892f9e7..42d6228693061 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 @@ -94,6 +94,7 @@ public class ConsumerNetworkThreadTest { private HeartbeatRequestManager heartbeatRequestManager; private MembershipManager memberhipsManager; private ConsumerNetworkThread consumerNetworkThread; + private CompletableEventReaper applicationEventReaper; private MockClient client; private SubscriptionState subscriptions; @@ -112,6 +113,7 @@ public void setup() { heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new); memberhipsManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); consumerNetworkThread = testBuilder.consumerNetworkThread; + applicationEventReaper = testBuilder.applicationEventReaper; subscriptions = testBuilder.subscriptions; consumerNetworkThread.initializeResources(); } @@ -306,7 +308,6 @@ void testEnsureEventsAreCompleted() { @Test void testReaperExpiresExpiredEvents() { - CompletableEventReaper completableEventReaper = consumerNetworkThread.completableEventReaper(); long event1TimeoutMs = 100; long event2TimeoutMs = 200; SyncCommitEvent event1 = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, event1TimeoutMs)); @@ -318,9 +319,9 @@ void testReaperExpiresExpiredEvents() { // Make sure both events have been moved from the event queue to the reaper's "tracked" list. assertFalse(applicationEventsQueue.contains(event1)); assertFalse(applicationEventsQueue.contains(event2)); - assertTrue(completableEventReaper.contains(event1)); - assertTrue(completableEventReaper.contains(event2)); - assertEquals(2, completableEventReaper.size()); + assertTrue(applicationEventReaper.contains(event1)); + assertTrue(applicationEventReaper.contains(event2)); + assertEquals(2, applicationEventReaper.size()); // Sleep long enough for the first event to have expired. time.sleep(event1TimeoutMs + 1); @@ -330,19 +331,19 @@ void testReaperExpiresExpiredEvents() { // Validate that the first event was expired, but the second continues to be tracked assertTrue(event1.future().isCompletedExceptionally()); assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future())); - assertFalse(completableEventReaper.contains(event1)); + assertFalse(applicationEventReaper.contains(event1)); - assertTrue(completableEventReaper.contains(event2)); + assertTrue(applicationEventReaper.contains(event2)); assertFalse(event2.future().isDone()); - assertEquals(1, completableEventReaper.size()); + assertEquals(1, applicationEventReaper.size()); // The cleanup will trigger the reaper's consumerNetworkThread.cleanup(); assertTrue(event2.future().isCompletedExceptionally()); assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); - assertFalse(consumerNetworkThread.completableEventReaper().contains(event2)); - assertEquals(0, consumerNetworkThread.completableEventReaper().size()); + assertFalse(applicationEventReaper.contains(event2)); + assertEquals(0, applicationEventReaper.size()); } private void prepareOffsetCommitRequest(final Map expectedOffsets, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 4ac402972c926..84e7532ecfe9c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.internals.ClusterResourceListeners; @@ -72,6 +73,7 @@ public class ConsumerTestBuilder implements Closeable { final LogContext logContext = new LogContext(); final Time time; public final BlockingQueue applicationEventQueue; + final CompletableEventReaper applicationEventReaper; public final BlockingQueue backgroundEventQueue; final ConsumerConfig config; final long retryBackoffMs; @@ -108,6 +110,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.groupInfo = groupInfo; this.time = enableAutoTick ? new MockTime(1) : new MockTime(); this.applicationEventQueue = new LinkedBlockingQueue<>(); + this.applicationEventReaper = new CompletableEventReaper(logContext); this.backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); @@ -302,6 +305,7 @@ public ConsumerNetworkThreadTestBuilder(Optional groupInfo) { logContext, time, applicationEventQueue, + applicationEventReaper, () -> applicationEventProcessor, () -> networkClientDelegate, () -> requestManagers From 16aa7eaf9551322294603667bd84875c2650a037 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 15:40:49 -0700 Subject: [PATCH 118/130] Update AsyncKafkaConsumer.java --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index ed6f6940632c3..77483b2b9c1db 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 @@ -375,6 +375,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.backgroundEventProcessor = new BackgroundEventProcessor( rebalanceListenerInvoker ); + this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) From 40a6e76348b3f20669e31b2b211b904ed4451922 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 16:36:47 -0700 Subject: [PATCH 119/130] Removing some noisy trace logging --- .../consumer/internals/events/CompletableEventReaper.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index e9cef49a9ba8f..9843bcf1ecfa9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -82,8 +82,6 @@ public void add(CompletableEvent event) { * {@link CompletableEvent#deadlineMs() expiration time} */ public void reap(long currentTimeMs) { - log.trace("Reaping expired events"); - Consumer> expireEvent = event -> { long pastDueMs = currentTimeMs - event.deadlineMs(); TimeoutException error = new TimeoutException(String.format("%s was %s ms past its expiration of %s", event.getClass().getSimpleName(), pastDueMs, event.deadlineMs())); @@ -103,8 +101,6 @@ public void reap(long currentTimeMs) { // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. tracked.removeIf(e -> e.future().isDone()); - - log.trace("Finished reaping expired events"); } /** @@ -126,8 +122,6 @@ public void reap(long currentTimeMs) { * @param events Events from a queue that have not yet been tracked that also need to be reviewed */ public void reap(Collection events) { - log.trace("Reaping incomplete events"); - Objects.requireNonNull(events, "Event queue to reap must be non-null"); Consumer> expireEvent = event -> { @@ -151,8 +145,6 @@ public void reap(Collection events) { .filter(e -> !e.future().isDone()) .forEach(expireEvent); events.clear(); - - log.trace("Finished reaping incomplete events"); } public int size() { From 7067b9577819c0ef1ed56365be0827f2aeafe6a6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 14 May 2024 16:43:59 -0700 Subject: [PATCH 120/130] Fixed lint error --- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index cd17b85e93663..2a2574c31b7f7 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 @@ -193,7 +193,7 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { new StringDeserializer(), time, (a, b, c, d, e, f, g) -> applicationEventHandler, - (a) -> backgroundEventReaper, + a -> backgroundEventReaper, (a, b, c, d, e, f, g) -> fetchCollector, (a, b, c, d) -> metadata, backgroundEventQueue From 284cf94592243fcbe2fe33bcddb50f0e6913a1d6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 15 May 2024 16:34:02 -0700 Subject: [PATCH 121/130] Update to ConsumerNetworkThread test logic Moved logic out of ConsumerTestBuilder and started using mocks. --- .../internals/ConsumerNetworkThreadTest.java | 134 +++--------------- .../internals/ConsumerTestBuilder.java | 31 ---- 2 files changed, 21 insertions(+), 144 deletions(-) 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 0158c47684a30..3ff7e2ce51ad4 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 @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -31,16 +30,10 @@ import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; -import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.FindCoordinatorRequestData; -import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorRequest; -import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.requests.OffsetCommitRequest; -import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.test.TestCondition; @@ -59,29 +52,26 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.createDefaultGroupInformation; import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; public class ConsumerNetworkThreadTest { - private ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder testBuilder; + private ConsumerTestBuilder testBuilder; private Time time; private ConsumerMetadata metadata; private NetworkClientDelegate networkClient; @@ -89,14 +79,13 @@ public class ConsumerNetworkThreadTest { private ApplicationEventProcessor applicationEventProcessor; private OffsetsRequestManager offsetsRequestManager; private CommitRequestManager commitRequestManager; - private CoordinatorRequestManager coordinatorRequestManager; private ConsumerNetworkThread consumerNetworkThread; - private CompletableEventReaper applicationEventReaper; + private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private MockClient client; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder(); + testBuilder = new ConsumerTestBuilder(createDefaultGroupInformation()); time = testBuilder.time; metadata = testBuilder.metadata; networkClient = testBuilder.networkClientDelegate; @@ -105,16 +94,24 @@ public void setup() { applicationEventProcessor = testBuilder.applicationEventProcessor; commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); offsetsRequestManager = testBuilder.offsetsRequestManager; - coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); - consumerNetworkThread = testBuilder.consumerNetworkThread; - applicationEventReaper = testBuilder.applicationEventReaper; + consumerNetworkThread = new ConsumerNetworkThread( + testBuilder.logContext, + time, + testBuilder.applicationEventQueue, + applicationEventReaper, + () -> applicationEventProcessor, + () -> testBuilder.networkClientDelegate, + () -> testBuilder.requestManagers + ); consumerNetworkThread.initializeResources(); } @AfterEach public void tearDown() { - if (testBuilder != null) + if (testBuilder != null) { testBuilder.close(); + consumerNetworkThread.close(Duration.ZERO); + } } @Test @@ -280,104 +277,15 @@ void testEnsureMetadataUpdateOnPoll() { } @Test - void testEnsureEventsAreCompleted() { - Node node = metadata.fetch().nodes().get(0); - coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); - prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); - ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); - CompletableFuture future = new CompletableFuture<>(); - when(event1.future()).thenReturn(future); - applicationEventsQueue.add(event1); - applicationEventsQueue.add(event2); - assertFalse(future.isDone()); - assertFalse(applicationEventsQueue.isEmpty()); - + void testCleanupInvokesReaper() { consumerNetworkThread.cleanup(); - assertTrue(future.isCompletedExceptionally()); - assertTrue(applicationEventsQueue.isEmpty()); + verify(applicationEventReaper).reap(any(Collection.class)); } @Test - void testReaperExpiresExpiredEvents() { - long event1TimeoutMs = 100; - long event2TimeoutMs = 200; - SyncCommitEvent event1 = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, event1TimeoutMs)); - SyncCommitEvent event2 = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, event2TimeoutMs)); - applicationEventsQueue.add(event1); - applicationEventsQueue.add(event2); + void testRunOnceInvokesReaper() { consumerNetworkThread.runOnce(); - - // Make sure both events have been moved from the event queue to the reaper's "tracked" list. - assertFalse(applicationEventsQueue.contains(event1)); - assertFalse(applicationEventsQueue.contains(event2)); - assertTrue(applicationEventReaper.contains(event1)); - assertTrue(applicationEventReaper.contains(event2)); - assertEquals(2, applicationEventReaper.size()); - - // Sleep long enough for the first event to have expired. - time.sleep(event1TimeoutMs + 1); - - consumerNetworkThread.runOnce(); - - // Validate that the first event was expired, but the second continues to be tracked - assertTrue(event1.future().isCompletedExceptionally()); - assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future())); - assertFalse(applicationEventReaper.contains(event1)); - - assertTrue(applicationEventReaper.contains(event2)); - assertFalse(event2.future().isDone()); - assertEquals(1, applicationEventReaper.size()); - - // The cleanup will trigger the reaper's - consumerNetworkThread.cleanup(); - - assertTrue(event2.future().isCompletedExceptionally()); - assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); - assertFalse(applicationEventReaper.contains(event2)); - assertEquals(0, applicationEventReaper.size()); - } - - private void prepareOffsetCommitRequest(final Map expectedOffsets, - final Errors error, - final boolean disconnected) { - Map errors = partitionErrors(expectedOffsets.keySet(), error); - client.prepareResponse(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors), disconnected); - } - - private Map partitionErrors(final Collection partitions, - final Errors error) { - final Map errors = new HashMap<>(); - for (TopicPartition partition : partitions) { - errors.put(partition, error); - } - return errors; - } - - private OffsetCommitResponse offsetCommitResponse(final Map responseData) { - return new OffsetCommitResponse(responseData); - } - - private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map expectedOffsets) { - return body -> { - OffsetCommitRequest req = (OffsetCommitRequest) body; - Map offsets = req.offsets(); - if (offsets.size() != expectedOffsets.size()) - return false; - - for (Map.Entry expectedOffset : expectedOffsets.entrySet()) { - if (!offsets.containsKey(expectedOffset.getKey())) { - return false; - } else { - Long actualOffset = offsets.get(expectedOffset.getKey()); - if (!actualOffset.equals(expectedOffset.getValue())) { - return false; - } - } - } - return true; - }; + verify(applicationEventReaper).reap(any(Long.class)); } private HashMap mockTopicPartitionOffset() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 84e7532ecfe9c..d7975eae12feb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -25,7 +25,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.internals.ClusterResourceListeners; @@ -39,7 +38,6 @@ import org.apache.kafka.common.utils.Timer; import java.io.Closeable; -import java.time.Duration; import java.util.HashMap; import java.util.Optional; import java.util.Properties; @@ -73,7 +71,6 @@ public class ConsumerTestBuilder implements Closeable { final LogContext logContext = new LogContext(); final Time time; public final BlockingQueue applicationEventQueue; - final CompletableEventReaper applicationEventReaper; public final BlockingQueue backgroundEventQueue; final ConsumerConfig config; final long retryBackoffMs; @@ -110,7 +107,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.groupInfo = groupInfo; this.time = enableAutoTick ? new MockTime(1) : new MockTime(); this.applicationEventQueue = new LinkedBlockingQueue<>(); - this.applicationEventReaper = new CompletableEventReaper(logContext); this.backgroundEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); @@ -291,33 +287,6 @@ public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); } - public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { - - final ConsumerNetworkThread consumerNetworkThread; - - public ConsumerNetworkThreadTestBuilder() { - this(createDefaultGroupInformation()); - } - - public ConsumerNetworkThreadTestBuilder(Optional groupInfo) { - super(groupInfo); - this.consumerNetworkThread = new ConsumerNetworkThread( - logContext, - time, - applicationEventQueue, - applicationEventReaper, - () -> applicationEventProcessor, - () -> networkClientDelegate, - () -> requestManagers - ); - } - - @Override - public void close() { - consumerNetworkThread.close(Duration.ZERO); - } - } - public static class GroupInformation { final String groupId; final Optional groupInstanceId; From 6a3ced693c0c9907cb8fb71a68e7609c80ebfade Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 15 May 2024 16:54:45 -0700 Subject: [PATCH 122/130] More work on improving and simplifying unit tests --- .../internals/AsyncKafkaConsumerTest.java | 24 ++----------------- .../internals/ConsumerNetworkThreadTest.java | 3 +-- 2 files changed, 3 insertions(+), 24 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 2a2574c31b7f7..8f4a3fefc91e5 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 @@ -151,7 +151,7 @@ public class AsyncKafkaConsumerTest { private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - private final CompletableEventReaper backgroundEventReaper = new CompletableEventReaper(new LogContext()); + private final CompletableEventReaper backgroundEventReaper = mock(CompletableEventReaper.class); @AfterEach public void resetAll() { @@ -1853,28 +1853,8 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { @Test void testReaperExpiresExpiredEvents() { consumer = newConsumer(); - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final SortedSet partitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - partitions.add(tp); - - consumer.subscribe(Collections.singletonList(topicName), new CounterConsumerRebalanceListener()); - - final ConsumerRebalanceListenerCallbackNeededEvent event1 = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_REVOKED, partitions); - final ConsumerRebalanceListenerCallbackNeededEvent event2 = new ConsumerRebalanceListenerCallbackNeededEvent(ON_PARTITIONS_ASSIGNED, partitions); - backgroundEventReaper.add(event1); - backgroundEventQueue.add(event2); - - assertEquals(1, backgroundEventReaper.size()); - assertEquals(1, backgroundEventQueue.size()); - consumer.close(); - - assertEquals(0, backgroundEventReaper.size()); - assertEquals(0, backgroundEventQueue.size()); - assertTrue(event1.future().isCompletedExceptionally()); - assertTrue(event2.future().isCompletedExceptionally()); + verify(backgroundEventReaper).reap(backgroundEventQueue); } private Map mockTopicPartitionOffset() { 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 3ff7e2ce51ad4..3ccdfb985f232 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 @@ -46,7 +46,6 @@ import java.time.Duration; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -279,7 +278,7 @@ void testEnsureMetadataUpdateOnPoll() { @Test void testCleanupInvokesReaper() { consumerNetworkThread.cleanup(); - verify(applicationEventReaper).reap(any(Collection.class)); + verify(applicationEventReaper).reap(applicationEventsQueue); } @Test From ec0edd423184389ce3eb48b7ac22de008a61b400 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 16 May 2024 13:30:20 -0700 Subject: [PATCH 123/130] Added more unit tests to ensure the reaper is being properly invoked --- .../internals/AsyncKafkaConsumerTest.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 8f4a3fefc91e5..c1f700302de6c 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 @@ -1851,12 +1851,29 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { } @Test - void testReaperExpiresExpiredEvents() { + void testReaperInvokedInClose() { consumer = newConsumer(); consumer.close(); verify(backgroundEventReaper).reap(backgroundEventQueue); } + @Test + void testReaperInvokedInUnsubscribe() { + consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); + consumer.unsubscribe(); + verify(backgroundEventReaper).reap(any(Long.class)); + } + + @Test + void testReaperInvokedInPoll() { + consumer = newConsumer(); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.subscribe(Collections.singletonList("topic")); + consumer.poll(Duration.ZERO); + verify(backgroundEventReaper).reap(any(Long.class)); + } + private Map mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); From ae6a6cd5f16043ecdf7bceb11eaf4de46c4782df Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 08:29:46 -0700 Subject: [PATCH 124/130] Testing expected value in timestamp-related reap tests --- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index c1f700302de6c..eaba32b0b36a3 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 @@ -1862,7 +1862,7 @@ void testReaperInvokedInUnsubscribe() { consumer = newConsumer(); completeUnsubscribeApplicationEventSuccessfully(); consumer.unsubscribe(); - verify(backgroundEventReaper).reap(any(Long.class)); + verify(backgroundEventReaper).reap(time.milliseconds()); } @Test @@ -1871,7 +1871,7 @@ void testReaperInvokedInPoll() { doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); consumer.subscribe(Collections.singletonList("topic")); consumer.poll(Duration.ZERO); - verify(backgroundEventReaper).reap(any(Long.class)); + verify(backgroundEventReaper).reap(time.milliseconds()); } private Map mockTopicPartitionOffset() { From 808eb4b5c7c7433137061d389e126e623e947d23 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 09:00:27 -0700 Subject: [PATCH 125/130] Added testEnsureEventsAreCompleted() back along with all its trapping --- .../internals/ConsumerNetworkThreadTest.java | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) 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 3ccdfb985f232..8c3f97dd64379 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 @@ -22,6 +22,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -30,10 +32,16 @@ import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.test.TestCondition; @@ -46,11 +54,14 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Optional; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; @@ -59,14 +70,18 @@ import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class ConsumerNetworkThreadTest { @@ -78,6 +93,7 @@ public class ConsumerNetworkThreadTest { private ApplicationEventProcessor applicationEventProcessor; private OffsetsRequestManager offsetsRequestManager; private CommitRequestManager commitRequestManager; + private CoordinatorRequestManager coordinatorRequestManager; private ConsumerNetworkThread consumerNetworkThread; private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private MockClient client; @@ -93,6 +109,7 @@ public void setup() { applicationEventProcessor = testBuilder.applicationEventProcessor; commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); offsetsRequestManager = testBuilder.offsetsRequestManager; + coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); consumerNetworkThread = new ConsumerNetworkThread( testBuilder.logContext, time, @@ -275,6 +292,41 @@ void testEnsureMetadataUpdateOnPoll() { verify(metadata, times(1)).updateWithCurrentRequestVersion(eq(metadataResponse), eq(false), anyLong()); } + @Test + void testEnsureEventsAreCompleted() { + // Mimic the logic of CompletableEventReaper.reap(Collection): + doAnswer(__ -> { + Iterator i = applicationEventsQueue.iterator(); + + while (i.hasNext()) { + ApplicationEvent event = i.next(); + + if (event instanceof CompletableEvent) + ((CompletableEvent) event).future().completeExceptionally(new TimeoutException()); + + i.remove(); + } + + return null; + }).when(applicationEventReaper).reap(any(Collection.class)); + + Node node = metadata.fetch().nodes().get(0); + coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); + prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); + CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); + ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); + CompletableFuture future = new CompletableFuture<>(); + when(event1.future()).thenReturn(future); + applicationEventsQueue.add(event1); + applicationEventsQueue.add(event2); + assertFalse(future.isDone()); + assertFalse(applicationEventsQueue.isEmpty()); + consumerNetworkThread.cleanup(); + assertTrue(future.isCompletedExceptionally()); + assertTrue(applicationEventsQueue.isEmpty()); + } + @Test void testCleanupInvokesReaper() { consumerNetworkThread.cleanup(); @@ -287,6 +339,47 @@ void testRunOnceInvokesReaper() { verify(applicationEventReaper).reap(any(Long.class)); } + private void prepareOffsetCommitRequest(final Map expectedOffsets, + final Errors error, + final boolean disconnected) { + Map errors = partitionErrors(expectedOffsets.keySet(), error); + client.prepareResponse(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors), disconnected); + } + + private Map partitionErrors(final Collection partitions, + final Errors error) { + final Map errors = new HashMap<>(); + for (TopicPartition partition : partitions) { + errors.put(partition, error); + } + return errors; + } + + private OffsetCommitResponse offsetCommitResponse(final Map responseData) { + return new OffsetCommitResponse(responseData); + } + + private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map expectedOffsets) { + return body -> { + OffsetCommitRequest req = (OffsetCommitRequest) body; + Map offsets = req.offsets(); + if (offsets.size() != expectedOffsets.size()) + return false; + + for (Map.Entry expectedOffset : expectedOffsets.entrySet()) { + if (!offsets.containsKey(expectedOffset.getKey())) { + return false; + } else { + Long actualOffset = offsets.get(expectedOffset.getKey()); + if (!actualOffset.equals(expectedOffset.getValue())) { + return false; + } + } + } + return true; + }; + } + private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); From e9ea1e687f9108b9f0fe8c2e9ae8afb9563f3796 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 09:02:22 -0700 Subject: [PATCH 126/130] Update testCommitAsyncWithFencedException Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com> --- .../consumer/internals/AsyncKafkaConsumerTest.java | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index eaba32b0b36a3..04630b2079d8d 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 @@ -335,16 +335,9 @@ public void testCommitAsyncWithFencedException() { assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); - // Close the consumer here as we know it will cause a FencedInstanceIdException to be thrown. - // If we get an error other than the FencedInstanceIdException, we'll raise a ruckus. - try { - consumer.close(); - } catch (KafkaException e) { - assertNotNull(e.getCause()); - assertInstanceOf(FencedInstanceIdException.class, e.getCause()); - } finally { - consumer = null; - } + Throwable e = assertThrows(KafkaException.class, () -> consumer.close()); + assertInstanceOf(FencedInstanceIdException.class, e.getCause()); + consumer = null; } @Test From 62bec7eba96834b1e82bdd734064307d6ec99aea Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 09:07:54 -0700 Subject: [PATCH 127/130] Changing expiration logic in CompletableEventReaper.reap() to match Timer.isExpired() --- .../consumer/internals/events/CompletableEventReaper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java index 9843bcf1ecfa9..545a03df8b30d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -96,7 +96,7 @@ public void reap(long currentTimeMs) { // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. tracked.stream() .filter(e -> !e.future().isDone()) - .filter(e -> currentTimeMs > e.deadlineMs()) + .filter(e -> currentTimeMs >= e.deadlineMs()) .forEach(expireEvent); // Second, remove any events that are already complete, just to make sure we don't hold references. This will // include any events that finished successfully as well as any events we just completed exceptionally above. From 0f168e202e452c6a908ec42d037a1c752408deb3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 09:11:03 -0700 Subject: [PATCH 128/130] Removed outdated logic in testCommitAsyncWithFencedException --- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 04630b2079d8d..40d37d9dfe813 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 @@ -334,10 +334,6 @@ public void testCommitAsyncWithFencedException() { commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); - - Throwable e = assertThrows(KafkaException.class, () -> consumer.close()); - assertInstanceOf(FencedInstanceIdException.class, e.getCause()); - consumer = null; } @Test From c423560a175c76686f0346bc2e3536d5740d3ad4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 09:13:44 -0700 Subject: [PATCH 129/130] maybeAutoCommitSync > autoCommitSync is there's no longer a conditional --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 ++-- .../clients/consumer/internals/AsyncKafkaConsumerTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 3a119eb21f988..dfbb1eb4cd441 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 @@ -1270,7 +1270,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx return; if (autoCommitEnabled) - maybeAutoCommitSync(timer); + autoCommitSync(timer); applicationEventHandler.add(new CommitOnCloseEvent()); completeQuietly( @@ -1282,7 +1282,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx } // Visible for testing - void maybeAutoCommitSync(final Timer timer) { + void autoCommitSync(final Timer timer) { Map allConsumed = subscriptions.allConsumed(); log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed); try { 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 40d37d9dfe813..7089b5a6d8b68 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 @@ -850,7 +850,7 @@ public void testAutoCommitSyncEnabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(time.timer(100)); + consumer.autoCommitSync(time.timer(100)); verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } From 91af164415b8b950c70d3a61bb0837c34ae4ed69 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 May 2024 11:02:53 -0700 Subject: [PATCH 130/130] Refactored processBackgroundEvents to eliminate passing in the EventProcessor --- .../consumer/internals/AsyncKafkaConsumer.java | 15 ++++++--------- .../internals/AsyncKafkaConsumerTest.java | 15 ++++++--------- 2 files changed, 12 insertions(+), 18 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 dfbb1eb4cd441..2a7db5b24663e 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 @@ -1501,7 +1501,7 @@ public void unsubscribe() { log.info("Unsubscribing all topics or patterns and assigned partitions"); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeEvent.future(), timer); + processBackgroundEvents(unsubscribeEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1695,7 +1695,7 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); offsetCommitCallbackInvoker.executeCallbacks(); maybeUpdateSubscriptionMetadata(); - processBackgroundEvents(backgroundEventProcessor); + processBackgroundEvents(); return updateFetchPositions(timer); } @@ -1827,7 +1827,7 @@ private void subscribeInternal(Collection topics, Optional processor) { + private boolean processBackgroundEvents() { AtomicReference firstError = new AtomicReference<>(); LinkedList events = new LinkedList<>(); @@ -1838,7 +1838,7 @@ private boolean processBackgroundEvents(EventProcessor processo if (event instanceof CompletableEvent) backgroundEventReaper.add((CompletableEvent) event); - processor.process(event); + backgroundEventProcessor.process(event); } catch (Throwable t) { KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); @@ -1888,18 +1888,15 @@ private boolean processBackgroundEvents(EventProcessor processo * execution of the rebalancing logic. The rebalancing logic cannot complete until the * {@link ConsumerRebalanceListener} callback is performed. * - * @param eventProcessor Event processor that contains the queue of events to process * @param future Event that contains a {@link CompletableFuture}; it is on this future that the * application thread will wait for completion * @param timer Overall timer that bounds how long to wait for the event to complete * @return {@code true} if the event completed within the timeout, {@code false} otherwise */ // Visible for testing - T processBackgroundEvents(EventProcessor eventProcessor, - Future future, - Timer timer) { + T processBackgroundEvents(Future future, Timer timer) { do { - boolean hadEvents = processBackgroundEvents(eventProcessor); + boolean hadEvents = processBackgroundEvents(); try { if (future.isDone()) { 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 7089b5a6d8b68..eab9f3e5ac12d 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 @@ -1736,7 +1736,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1762,15 +1762,14 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - EventProcessor processor = mock(EventProcessor.class); - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. assertEquals(800, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1781,8 +1780,7 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - EventProcessor processor = mock(EventProcessor.class); - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); // Because we didn't need to perform a timed get, we should still have every last millisecond // of our initial timeout. @@ -1790,7 +1788,7 @@ public void testProcessBackgroundEventsWithoutDelay() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1805,8 +1803,7 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - EventProcessor processor = mock(EventProcessor.class); - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); // Because we forced our mocked future to continuously time out, we should have no time remaining. assertEquals(0, timer.remainingMs());