From 81ce457fb57c85a3f52e94d8608aa637d4d8a5b8 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 15:01:15 -0700 Subject: [PATCH 01/68] KAFKA-15276: Implement partition assignment reconciliation Added logic for reconciling partition assignment between the target assignment provided by the group coordinator and the current assignment as specified in the SubscriptionState. This refactors the ConsumerRebalanceListener code from ConsumerCoordinator for reuse in both places. --- checkstyle/suppressions.xml | 3 + .../org/apache/kafka/clients/Metadata.java | 7 + .../apache/kafka/clients/MetadataCache.java | 10 +- .../internals/AbstractCoordinator.java | 2 +- .../internals/AssignmentReconciler.java | 245 +++++++++++++++++ .../internals/ConsumerCoordinator.java | 143 ++-------- .../internals/ConsumerCoordinatorMetrics.java | 78 ++++++ .../ConsumerRebalanceListenerInvoker.java | 162 +++++++++++ .../internals/HeartbeatRequestManager.java | 11 +- .../consumer/internals/MembershipManager.java | 58 ++++ .../internals/MembershipManagerImpl.java | 101 ++++++- .../internals/PrototypeAsyncConsumer.java | 56 +++- .../consumer/internals/RequestManagers.java | 14 +- .../internals/events/ApplicationEvent.java | 3 +- .../events/ApplicationEventHandler.java | 47 +--- .../events/ApplicationEventProcessor.java | 28 +- .../internals/events/BackgroundEvent.java | 4 +- .../events/BackgroundEventProcessor.java | 68 ++++- .../events/PartitionLostCompleteEvent.java | 82 ++++++ .../events/PartitionLostStartedEvent.java | 76 ++++++ .../events/RebalanceCompleteEvent.java | 97 +++++++ .../events/RebalanceStartedEvent.java | 88 ++++++ .../internals/AssignmentReconcilerTest.java | 251 ++++++++++++++++++ .../internals/ConsumerTestBuilder.java | 103 ++++--- .../internals/MembershipManagerImplTest.java | 189 ++++++------- 25 files changed, 1577 insertions(+), 349 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index e1e746c755afc..39f76e70d81dc 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -127,6 +127,9 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 052f2081d1854..0b33cf2b0b717 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -279,6 +279,13 @@ public synchronized Map topicIds() { return cache.topicIds(); } + /** + * @return a mapping from topic IDs to topic names for all topics with valid IDs in the cache + */ + public synchronized Map topicNames() { + return cache.topicNames(); + } + public synchronized LeaderAndEpoch currentLeader(TopicPartition topicPartition) { Optional maybeMetadata = partitionMetadataIfCurrent(topicPartition); if (!maybeMetadata.isPresent()) diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java index d7b6bfd344e61..faa33d2b79388 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java @@ -51,6 +51,7 @@ public class MetadataCache { private final Node controller; private final Map metadataByPartition; private final Map topicIds; + private final Map topicNames; private Cluster clusterInstance; @@ -80,7 +81,10 @@ private MetadataCache(String clusterId, this.invalidTopics = invalidTopics; this.internalTopics = internalTopics; this.controller = controller; - this.topicIds = topicIds; + this.topicIds = Collections.unmodifiableMap(topicIds); + this.topicNames = Collections.unmodifiableMap( + topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) + ); this.metadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -102,6 +106,10 @@ Map topicIds() { return topicIds; } + Map topicNames() { + return topicNames; + } + Optional nodeById(int id) { return Optional.ofNullable(nodes.get(id)); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 24d05e7d63bfb..c6b655a21f360 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -1294,7 +1294,7 @@ boolean generationUnchanged() { } } - protected final Meter createMeter(Metrics metrics, String groupName, String baseName, String descriptiveName) { + static Meter createMeter(Metrics metrics, String groupName, String baseName, String descriptiveName) { return new Meter(new WindowedCount(), metrics.metricName(baseName + "-rate", groupName, String.format("The number of %s per second", descriptiveName)), diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java new file mode 100644 index 0000000000000..c39e33f09d53d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.PartitionLostStartedEvent; +import org.apache.kafka.clients.consumer.internals.events.RebalanceCompleteEvent; +import org.apache.kafka.clients.consumer.internals.events.RebalanceStartedEvent; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.BlockingQueue; + +/** + * {@code AssignmentReconciler} performs the work of reconciling this consumer's partition assignment as directed + * by the consumer group coordinator. When the coordinator determines that a change to the partition ownership of + * the group is required, it will communicate with each consumer to relay its respective target + * assignment, that is, the set of partitions for which that consumer should now assume ownership. It is then the + * responsibility of the consumer to work toward that target by performing the necessary internal modifications to + * satisfy the assignment from the coordinator. In practical terms, this means that it must first determine the set + * difference between the {@link SubscriptionState#assignedPartitions() current assignment} and the + * target assignment. + * + *

+ * + * Internally, reconciliation requires the following steps: + * + *

    + *
  1. + * On the background thread, upon receipt of a new assignment from the group coordinator, the + * {@link MembershipManager} should call {@link #startReconcile(Set)} to start reconciliation. + *
  2. + *
  3. + * Internally, the partitions to revoke are determined via {@link #getPartitionsToRevoke(Set)}; + * these are the partitions in the current assignment that are not in the target assignment. + * Next, we calculate the partitions to assign using {@link #getPartitionsToAssign(Set)}; + * these are the partitions in the target assignment that are not in the current assignment. + *
  4. + *
  5. + * Send a {@link RebalanceStartedEvent} so that the application thread will execute the + * {@link ConsumerRebalanceListener} callback methods. + *
  6. + *
  7. + * On the application thread, when the {@link RebalanceStartedEvent} is received, execute the + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and + * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callback methods + *
  8. + *
  9. + * Enqueue a corresponding {@link RebalanceCompleteEvent} so that the background thread + * will know the listener was invoked and the result of the invocation + *
  10. + *
  11. + * On the background thread, process the {@link RebalanceCompleteEvent}, which should call the + * {@link MembershipManager#completeReconcile(Set, Set, Optional)} method. This method will call + * {@link #completeReconcile(Set, Set)} to remove the revoked partitions and add the assigned partitions in the + * {@link SubscriptionState#assignFromSubscribed(Collection) current assignment} and then make a note to + * send a {@link ConsumerGroupHeartbeatRequest} to the group coordinator on its next pass of + * {@link HeartbeatRequestManager#poll(long)} + *
  12. + *
+ */ +public class AssignmentReconciler { + + private final Logger log; + private final SubscriptionState subscriptions; + private final BlockingQueue backgroundEventQueue; + + AssignmentReconciler(LogContext logContext, + SubscriptionState subscriptions, + BlockingQueue backgroundEventQueue) { + this.log = logContext.logger(getClass()); + this.subscriptions = subscriptions; + this.backgroundEventQueue = backgroundEventQueue; + } + + /** + * Performs the step of dropping any assigned {@link TopicPartition partitions} as this consumer is no longer + * to be considered a valid member of the group. + */ + void startLost() { + SortedSet partitionsToLose = getPartitionsToLose(); + + if (partitionsToLose.isEmpty()) { + log.debug("Skipping invocation of {} callbacks as no partitions changed in the new assignment", + ConsumerRebalanceListener.class.getSimpleName()); + return; + } + + markPartitionsPendingRevocation(); + + // TODO: determine if there's a "valid" ConsumerRebalanceListener in use. If not, we can return false + // immediately as there's no need to perform the rebalance callback invocation. + + log.debug("Enqueuing event to invoke {} callbacks", ConsumerRebalanceListener.class.getSimpleName()); + backgroundEventQueue.add(new PartitionLostStartedEvent(partitionsToLose)); + } + + /** + * This method should be invoked to signal the completion of the "{@link TopicPartition lost partition}" + * process. Specifically, it is to be executed on background thread after the + * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback was executed on the application + * thread. It should clear the set of {@link SubscriptionState#assignedPartitions() assigned partitions}, + * regardless of the set of "lost partitions." + * + * @param lostPartitions Set of {@link TopicPartition partitions} that were lost + * @see AssignmentReconciler + */ + void completeLost(Set lostPartitions) { + log.debug("{} callbacks were successfully invoked", ConsumerRebalanceListener.class.getSimpleName()); + subscriptions.assignFromSubscribed(Collections.emptySet()); + } + + /** + * Performs the assignment phase of the reconciliation process as described in the top-level class documentation. + * + * @param target Set of {@link TopicPartition} which represents the target set of topics + */ + void startReconcile(Set target) { + SortedSet partitionsToRevoke = getPartitionsToRevoke(target); + SortedSet partitionsToAssign = getPartitionsToAssign(target); + + if (partitionsToRevoke.isEmpty() && partitionsToAssign.isEmpty()) { + log.debug("Skipping invocation of {} callbacks as no partitions changed in the new assignment", + ConsumerRebalanceListener.class.getSimpleName()); + return; + } + + markPartitionsPendingRevocation(); + + // TODO: determine if there's a "valid" ConsumerRebalanceListener in use. If not, we can return false + // immediately as there's no need to perform the rebalance callback invocation. + + log.debug("Enqueuing event to invoke {} callbacks", ConsumerRebalanceListener.class.getSimpleName()); + RebalanceStartedEvent event = new RebalanceStartedEvent( + partitionsToRevoke, + partitionsToAssign + ); + backgroundEventQueue.add(event); + } + + /** + * This method should be invoked to signal the completion of a successful {@link TopicPartition partition} + * assignment reconciliation. Specifically, it is to be executed on background thread after the + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and + * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks have completed execution on + * the application thread. It should update the set of + * {@link SubscriptionState#assignedPartitions() assigned partitions} based on the + * given partitions. + * + * @param revokedPartitions Set of {@link TopicPartition partitions} that were revoked + * @param assignedPartitions Set of {@link TopicPartition partitions} that were assigned + * @see MembershipManager#completeReconcile(Set, Set, Optional) + */ + void completeReconcile(Set revokedPartitions, Set assignedPartitions) { + log.debug("{} callbacks were successfully invoked", ConsumerRebalanceListener.class.getSimpleName()); + + Set newAssignment = new HashSet<>(subscriptions.assignedPartitions()); + newAssignment.addAll(assignedPartitions); + newAssignment.removeAll(revokedPartitions); + subscriptions.assignFromSubscribed(newAssignment); + } + + /** + * Determine which partitions are newly revoked. This is done by comparing the target set from the assignment + * against the {@link SubscriptionState#assignedPartitions() current set}. The returned set of + * {@link TopicPartition partitions} are composed of any partitions that are in the current set but + * are no longer in the target set. + * + * @param target Set of {@link TopicPartition} which represents the target set of topics + * @return Set of partitions to revoke + */ + SortedSet getPartitionsToRevoke(Set target) { + SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); + partitions.addAll(subscriptions.assignedPartitions()); + partitions.removeAll(target); + return partitions; + } + + /** + * Determine which partitions are newly assigned. This is done by comparing the target set from the assignment + * against the {@link SubscriptionState#assignedPartitions() current set}. Any {@link TopicPartition partitions} + * from the target set that are not already in the current set are included in the returned set. + * + * @param target Set of {@link TopicPartition} which represents the target set of topics + * @return Set of partitions to assign + */ + SortedSet getPartitionsToAssign(Set target) { + SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); + partitions.addAll(target); + partitions.removeAll(subscriptions.assignedPartitions()); + return partitions; + } + + /** + * Determine which partitions should be "lost". This is simply the + * {@link SubscriptionState#assignedPartitions() current set} of {@link TopicPartition partitions}. + * + * @return Set of partitions to "lose" + */ + SortedSet getPartitionsToLose() { + SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); + partitions.addAll(subscriptions.assignedPartitions()); + return partitions; + } + + /** + * When asynchronously committing offsets prior to the revocation of a set of partitions, there will be a + * window of time between when the offset commit is sent and when it returns and revocation completes. It is + * possible for pending fetches for these partitions to return during this time, which means the application's + * position may get ahead of the committed position prior to revocation. This can cause duplicate consumption. + * To prevent this, we mark the partitions as "pending revocation," which stops the Fetcher from sending new + * fetches or returning data from previous fetches to the user. + */ + private void markPartitionsPendingRevocation() { + Set partitions = subscriptions.assignedPartitions(); + log.debug("Marking assigned partitions pending for revocation: {}", partitions); + subscriptions.markPendingRevocation(partitions); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index bdcbfc39dfc27..d04a886468c7b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -25,7 +25,6 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; @@ -48,11 +47,7 @@ import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; -import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; @@ -148,6 +143,7 @@ private boolean sameRequest(final Set currentRequest, final Gene } private final RebalanceProtocol protocol; + private final ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker; // pending commit offset request in onJoinPrepare private RequestFuture autoCommitOffsetRequestFuture = null; // a timer for join prepare to know when to stop. @@ -189,7 +185,7 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig, this.autoCommitIntervalMs = autoCommitIntervalMs; this.assignors = assignors; this.completedOffsetCommits = new ConcurrentLinkedQueue<>(); - this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix); + this.sensors = new ConsumerCoordinatorMetrics(subscriptions, metrics, metricGrpPrefix); this.interceptors = interceptors; this.inFlightAsyncCommits = new AtomicInteger(); this.pendingAsyncCommits = new AtomicInteger(); @@ -227,6 +223,12 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig, protocol = null; } + this.consumerRebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( + logContext, + subscriptions, + time, + sensors + ); this.metadata.requestUpdate(true); } @@ -321,71 +323,6 @@ private Exception invokeOnAssignment(final ConsumerPartitionAssignor assignor, f return null; } - private Exception invokePartitionsAssigned(final SortedSet assignedPartitions) { - log.info("Adding newly assigned partitions: {}", Utils.join(assignedPartitions, ", ")); - - ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); - try { - final long startMs = time.milliseconds(); - listener.onPartitionsAssigned(assignedPartitions); - sensors.assignCallbackSensor.record(time.milliseconds() - startMs); - } catch (WakeupException | InterruptException e) { - throw e; - } catch (Exception e) { - log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}", - listener.getClass().getName(), assignedPartitions, e); - return e; - } - - return null; - } - - private Exception invokePartitionsRevoked(final SortedSet revokedPartitions) { - log.info("Revoke previously assigned partitions {}", Utils.join(revokedPartitions, ", ")); - Set revokePausedPartitions = subscriptions.pausedPartitions(); - revokePausedPartitions.retainAll(revokedPartitions); - if (!revokePausedPartitions.isEmpty()) - log.info("The pause flag in partitions [{}] will be removed due to revocation.", Utils.join(revokePausedPartitions, ", ")); - - ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); - try { - final long startMs = time.milliseconds(); - listener.onPartitionsRevoked(revokedPartitions); - sensors.revokeCallbackSensor.record(time.milliseconds() - startMs); - } catch (WakeupException | InterruptException e) { - throw e; - } catch (Exception e) { - log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}", - listener.getClass().getName(), revokedPartitions, e); - return e; - } - - return null; - } - - private Exception invokePartitionsLost(final SortedSet lostPartitions) { - log.info("Lost previously assigned partitions {}", Utils.join(lostPartitions, ", ")); - Set lostPausedPartitions = subscriptions.pausedPartitions(); - lostPausedPartitions.retainAll(lostPartitions); - if (!lostPausedPartitions.isEmpty()) - log.info("The pause flag in partitions [{}] will be removed due to partition lost.", Utils.join(lostPausedPartitions, ", ")); - - ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); - try { - final long startMs = time.milliseconds(); - listener.onPartitionsLost(lostPartitions); - sensors.loseCallbackSensor.record(time.milliseconds() - startMs); - } catch (WakeupException | InterruptException e) { - throw e; - } catch (Exception e) { - log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}", - listener.getClass().getName(), lostPartitions, e); - return e; - } - - return null; - } - @Override protected void onJoinComplete(int generation, String memberId, @@ -453,7 +390,7 @@ protected void onJoinComplete(int generation, // Revoke partitions that were previously owned but no longer assigned; // note that we should only change the assignment (or update the assignor's state) // AFTER we've triggered the revoke callback - firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); + firstException.compareAndSet(null, consumerRebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions)); // If revoked any partitions, need to re-join the group afterwards final String fullReason = String.format("need to revoke partitions %s as indicated " + @@ -476,7 +413,7 @@ protected void onJoinComplete(int generation, subscriptions.assignFromSubscribed(assignedPartitions); // Add partitions that were not previously owned but are now assigned - firstException.compareAndSet(null, invokePartitionsAssigned(addedPartitions)); + firstException.compareAndSet(null, consumerRebalanceListenerInvoker.invokePartitionsAssigned(addedPartitions)); if (firstException.get() != null) { if (firstException.get() instanceof KafkaException) { @@ -831,7 +768,7 @@ protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { if (!revokedPartitions.isEmpty()) { log.info("Giving away all assigned partitions as lost since generation/memberID has been reset," + "indicating that consumer is in old state or no longer part of the group"); - exception = invokePartitionsLost(revokedPartitions); + exception = consumerRebalanceListenerInvoker.invokePartitionsLost(revokedPartitions); subscriptions.assignFromSubscribed(Collections.emptySet()); } @@ -840,7 +777,7 @@ protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { case EAGER: // revoke all partitions revokedPartitions.addAll(subscriptions.assignedPartitions()); - exception = invokePartitionsRevoked(revokedPartitions); + exception = consumerRebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions); subscriptions.assignFromSubscribed(Collections.emptySet()); @@ -854,7 +791,7 @@ protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { .collect(Collectors.toSet())); if (!revokedPartitions.isEmpty()) { - exception = invokePartitionsRevoked(revokedPartitions); + exception = consumerRebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions); ownedPartitions.removeAll(revokedPartitions); subscriptions.assignFromSubscribed(ownedPartitions); @@ -908,9 +845,9 @@ public void onLeavePrepare() { if ((currentGeneration.generationId == Generation.NO_GENERATION.generationId || currentGeneration.memberId.equals(Generation.NO_GENERATION.memberId)) || rebalanceInProgress()) { - e = invokePartitionsLost(droppedPartitions); + e = consumerRebalanceListenerInvoker.invokePartitionsLost(droppedPartitions); } else { - e = invokePartitionsRevoked(droppedPartitions); + e = consumerRebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions); } subscriptions.assignFromSubscribed(Collections.emptySet()); @@ -1579,56 +1516,6 @@ public void handle(OffsetFetchResponse response, RequestFuture subscriptions.numAssignedPartitions(); - metrics.addMetric(metrics.metricName("assigned-partitions", - this.metricGrpName, - "The number of partitions currently assigned to this consumer"), numParts); - } - } - private static class MetadataSnapshot { private final int version; private final Map> partitionsPerTopic; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java new file mode 100644 index 0000000000000..9e13d77d8e851 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; + +import static org.apache.kafka.clients.consumer.internals.AbstractCoordinator.createMeter; + +class ConsumerCoordinatorMetrics { + + final String metricGrpName; + final Sensor commitSensor; + final Sensor revokeCallbackSensor; + final Sensor assignCallbackSensor; + final Sensor loseCallbackSensor; + + ConsumerCoordinatorMetrics(SubscriptionState subscriptions, + Metrics metrics, + String metricGrpPrefix) { + this.metricGrpName = metricGrpPrefix + "-coordinator-metrics"; + + this.commitSensor = metrics.sensor("commit-latency"); + this.commitSensor.add(metrics.metricName("commit-latency-avg", + this.metricGrpName, + "The average time taken for a commit request"), new Avg()); + this.commitSensor.add(metrics.metricName("commit-latency-max", + this.metricGrpName, + "The max time taken for a commit request"), new Max()); + this.commitSensor.add(createMeter(metrics, metricGrpName, "commit", "commit calls")); + + this.revokeCallbackSensor = metrics.sensor("partition-revoked-latency"); + this.revokeCallbackSensor.add(metrics.metricName("partition-revoked-latency-avg", + this.metricGrpName, + "The average time taken for a partition-revoked rebalance listener callback"), new Avg()); + this.revokeCallbackSensor.add(metrics.metricName("partition-revoked-latency-max", + this.metricGrpName, + "The max time taken for a partition-revoked rebalance listener callback"), new Max()); + + this.assignCallbackSensor = metrics.sensor("partition-assigned-latency"); + this.assignCallbackSensor.add(metrics.metricName("partition-assigned-latency-avg", + this.metricGrpName, + "The average time taken for a partition-assigned rebalance listener callback"), new Avg()); + this.assignCallbackSensor.add(metrics.metricName("partition-assigned-latency-max", + this.metricGrpName, + "The max time taken for a partition-assigned rebalance listener callback"), new Max()); + + this.loseCallbackSensor = metrics.sensor("partition-lost-latency"); + this.loseCallbackSensor.add(metrics.metricName("partition-lost-latency-avg", + this.metricGrpName, + "The average time taken for a partition-lost rebalance listener callback"), new Avg()); + this.loseCallbackSensor.add(metrics.metricName("partition-lost-latency-max", + this.metricGrpName, + "The max time taken for a partition-lost rebalance listener callback"), new Max()); + + Measurable numParts = (config, now) -> subscriptions.numAssignedPartitions(); + metrics.addMetric(metrics.metricName("assigned-partitions", + this.metricGrpName, + "The number of partitions currently assigned to this consumer"), numParts); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java new file mode 100644 index 0000000000000..f671811d79672 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.metrics.Sensor; +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; + +import java.util.Set; +import java.util.SortedSet; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class encapsulates the invocation of the callback methods defined in the {@link ConsumerRebalanceListener} + * interface. When consumer group partition assignment changes, these methods are invoked. This class wraps those + * callback calls with some logging, optional {@link Sensor} updates, etc. + */ +public class ConsumerRebalanceListenerInvoker { + + private final Logger log; + private final SubscriptionState subscriptions; + private final Time time; + private final ConsumerCoordinatorMetrics sensors; + + ConsumerRebalanceListenerInvoker(LogContext logContext, + SubscriptionState subscriptions, + Time time, + ConsumerCoordinatorMetrics sensors) { + this.log = logContext.logger(getClass()); + this.subscriptions = subscriptions; + this.time = time; + this.sensors = sensors; + } + + Exception invokePartitionsAssigned(final SortedSet assignedPartitions) { + log.info("Adding newly assigned partitions: {}", Utils.join(assignedPartitions, ", ")); + + ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); + try { + final long startMs = time.milliseconds(); + listener.onPartitionsAssigned(assignedPartitions); + sensors.assignCallbackSensor.record(time.milliseconds() - startMs); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}", + listener.getClass().getName(), assignedPartitions, e); + return e; + } + + return null; + } + + Exception invokePartitionsRevoked(final SortedSet revokedPartitions) { + log.info("Revoke previously assigned partitions {}", Utils.join(revokedPartitions, ", ")); + Set revokePausedPartitions = subscriptions.pausedPartitions(); + revokePausedPartitions.retainAll(revokedPartitions); + if (!revokePausedPartitions.isEmpty()) + log.info("The pause flag in partitions [{}] will be removed due to revocation.", org.apache.kafka.common.utils.Utils.join(revokePausedPartitions, ", ")); + + ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); + try { + final long startMs = time.milliseconds(); + listener.onPartitionsRevoked(revokedPartitions); + sensors.revokeCallbackSensor.record(time.milliseconds() - startMs); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}", + listener.getClass().getName(), revokedPartitions, e); + return e; + } + + return null; + } + + Exception invokePartitionsLost(final SortedSet lostPartitions) { + log.info("Lost previously assigned partitions {}", Utils.join(lostPartitions, ", ")); + Set lostPausedPartitions = subscriptions.pausedPartitions(); + lostPausedPartitions.retainAll(lostPartitions); + if (!lostPausedPartitions.isEmpty()) + log.info("The pause flag in partitions [{}] will be removed due to partition lost.", Utils.join(lostPausedPartitions, ", ")); + + ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); + try { + final long startMs = time.milliseconds(); + listener.onPartitionsLost(lostPartitions); + sensors.loseCallbackSensor.record(time.milliseconds() - startMs); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}", + listener.getClass().getName(), lostPartitions, e); + return e; + } + + return null; + } + + public void rebalance(final SortedSet revokedPartitions, + final SortedSet assignedPartitions) { + final AtomicReference firstException = new AtomicReference<>(null); + + if (!revokedPartitions.isEmpty()) { + // Revoke partitions that were previously owned but no longer assigned; + // note that we should only change the assignment (or update the assignor's state) + // AFTER we've triggered the revoke callback + firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); + } + + if (!assignedPartitions.isEmpty()) { + // Add partitions that were not previously owned but are now assigned + firstException.compareAndSet(null, invokePartitionsAssigned(assignedPartitions)); + } + + if (firstException.get() == null) + return; + + if (firstException.get() instanceof KafkaException) + throw (KafkaException) firstException.get(); + else + throw new KafkaException("User rebalance callback throws an error", firstException.get()); + } + + public void lose(final SortedSet lostPartitions) { + if (lostPartitions.isEmpty()) + return; + + log.info("Giving away all assigned partitions as lost since generation/memberID has been reset, " + + "indicating that consumer is in old state or no longer part of the group"); + Exception e = invokePartitionsLost(lostPartitions); + + if (e == null) + return; + + if (e instanceof KafkaException) + throw (KafkaException) e; + else + throw new KafkaException("User rebalance callback throws an error", e); + } +} 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 2172313054d10..f45ffe9229972 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 @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.UnknownTopicIdException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; @@ -212,7 +213,15 @@ private void onResponse(final ConsumerGroupHeartbeatResponse response, long curr this.heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); this.heartbeatRequestState.resetTimer(); - this.membershipManager.updateState(response.data()); + + try { + this.membershipManager.updateState(response.data()); + } catch (UnknownTopicIdException e) { + // This can occur when the assignment has a topic ID that we don't know about yet. + // + // TODO: I have no idea what to do here + // metadata.requestUpdate(); + } return; } onErrorResponse(response, currentTimeMs); 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 8a95a80c65991..1e76086a873db 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 @@ -16,9 +16,14 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import java.util.Collection; import java.util.Optional; +import java.util.Set; /** * A stateful object tracking the state of a single member in relationship to a consumer group: @@ -99,4 +104,57 @@ public interface MembershipManager { * @return True if the member should send heartbeat to the coordinator. */ boolean shouldSendHeartbeat(); + + /** + * This method should be invoked to signal the completion of a successful {@link TopicPartition partition} + * assignment reconciliation. Specifically, it is to be executed on background thread after the + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and + * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks have completed execution on + * the application thread. It should perform two tasks: + * + *
    + *
  1. + * Update the set of {@link SubscriptionState#assignedPartitions() assigned partitions} based on the + * given partitions + *
  2. + *
  3. + * Update the necessary internal state to signal to the {@link HeartbeatRequestManager} that it + * should send an acknowledgement heartbeat request to the group coordinator + *
  4. + *
+ * + * Note: the partition assignment reconciliation process is started based on the receipt of a new + * {@link ConsumerGroupHeartbeatResponseData.Assignment target assignment}. + * + * @param revokedPartitions Set of {@link TopicPartition partitions} that were revoked + * @param assignedPartitions Set of {@link TopicPartition partitions} that were assigned + * @param callbackError Optional {@link KafkaException error} if an exception was thrown during callbacks + * @see AssignmentReconciler + */ + void completeReconcile(Set revokedPartitions, + Set assignedPartitions, + Optional callbackError); + + /** + * This method should be invoked to signal the completion of the "{@link TopicPartition lost partition}" + * process. Specifically, it is to be executed on background thread after the + * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback was executed on the application + * thread. It should perform two tasks: + * + *
    + *
  1. + * Clear the set of {@link SubscriptionState#assignedPartitions() assigned partitions}, regardless of + * the set of "lost partitions" + *
  2. + *
  3. + * Update the necessary internal state to signal to the {@link HeartbeatRequestManager} that it + * should send an acknowledgement heartbeat request to the group coordinator + *
  4. + *
+ * + * @param lostPartitions Set of {@link TopicPartition partitions} that were lost + * @param callbackError Optional {@link KafkaException error} if an exception was thrown during callback + * @see AssignmentReconciler + */ + void completeLost(Set lostPartitions, Optional callbackError); } 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 2a9a5d2992daa..6bb7c543bc3bf 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 @@ -17,6 +17,11 @@ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.UnknownTopicIdException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; @@ -24,6 +29,9 @@ import org.slf4j.Logger; import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; /** * Membership manager that maintains group membership for a single member, following the new @@ -37,6 +45,21 @@ */ public class MembershipManagerImpl implements MembershipManager { + /** + * Logger. + */ + private final Logger log; + + /** + * For reconciling assignments. + */ + private final AssignmentReconciler assignmentReconciler; + + /** + * Metadata that allows us to create the partitions needed for {@link ConsumerRebalanceListener}. + */ + private final ConsumerMetadata metadata; + /** * Group ID of the consumer group the member will be part of, provided when creating the current * membership manager. @@ -86,24 +109,26 @@ public class MembershipManagerImpl implements MembershipManager { private Optional targetAssignment; /** - * Logger. + * Latest assignment that the member received from the server while a {@link #targetAssignment} + * was in process. */ - private final Logger log; - - public MembershipManagerImpl(String groupId, LogContext logContext) { - this(groupId, null, null, logContext); - } - - public MembershipManagerImpl(String groupId, - String groupInstanceId, - String serverAssignor, - LogContext logContext) { + private Optional nextTargetAssignment; + + public MembershipManagerImpl(LogContext logContext, + AssignmentReconciler assignmentReconciler, + ConsumerMetadata metadata, + String groupId, + Optional groupInstanceId, + Optional serverAssignor) { + this.log = logContext.logger(MembershipManagerImpl.class); + this.assignmentReconciler = assignmentReconciler; + this.metadata = metadata; this.groupId = groupId; this.state = MemberState.UNJOINED; - this.serverAssignor = Optional.ofNullable(serverAssignor); - this.groupInstanceId = Optional.ofNullable(groupInstanceId); + this.serverAssignor = serverAssignor; + this.groupInstanceId = groupInstanceId; this.targetAssignment = Optional.empty(); - this.log = logContext.logger(MembershipManagerImpl.class); + this.nextTargetAssignment = Optional.empty(); } /** @@ -181,6 +206,7 @@ public void updateState(ConsumerGroupHeartbeatResponseData response) { public void transitionToFenced() { resetEpoch(); transitionTo(MemberState.FENCED); + assignmentReconciler.startLost(); } /** @@ -206,10 +232,33 @@ private boolean maybeTransitionToStable() { transitionTo(MemberState.STABLE); } else { transitionTo(MemberState.RECONCILING); + startReconciliation(); } return state.equals(MemberState.STABLE); } + private void startReconciliation() { + if (!targetAssignment.isPresent()) + return; + + SortedSet targetPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); + + for (ConsumerGroupHeartbeatResponseData.TopicPartitions topicPartitions : targetAssignment.get().topicPartitions()) { + Uuid topicId = topicPartitions.topicId(); + String topicName = metadata.topicNames().get(topicId); + + // TODO... I don't think this is right... + if (topicName == null) + throw new UnknownTopicIdException("A topic name for the topic ID " + topicId + " was not found in the local metadata cache"); + + for (Integer partition : topicPartitions.partitions()) { + targetPartitions.add(new TopicPartition(topicName, partition)); + } + } + + assignmentReconciler.startReconcile(targetPartitions); + } + /** * Take new target assignment received from the server and set it as targetAssignment to be * processed. Following the consumer group protocol, the server won't send a new target @@ -298,4 +347,28 @@ public void onTargetAssignmentProcessComplete(ConsumerGroupHeartbeatResponseData targetAssignment = Optional.empty(); transitionTo(MemberState.STABLE); } + + @Override + public void completeReconcile(Set revokedPartitions, + Set assignedPartitions, + Optional callbackError) { + if (callbackError.isPresent()) { + // TODO: how to react to callback errors? + } + + assignmentReconciler.completeReconcile(revokedPartitions, assignedPartitions); + transitionTo(MemberState.STABLE); + // TODO: update state to signal the HeartbeatRequestManager to send an ACK heartbeat + } + + @Override + public void completeLost(Set lostPartitions, Optional callbackError) { + if (callbackError.isPresent()) { + // TODO: how to react to callback errors? + } + + assignmentReconciler.completeLost(lostPartitions); + transitionTo(MemberState.UNJOINED); + // TODO: update state to signal the HeartbeatRequestManager to send an ACK heartbeat + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 949616daa8551..59713bf8cd2cb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -34,11 +34,11 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -117,6 +117,7 @@ public class PrototypeAsyncConsumer implements Consumer { private final ApplicationEventHandler applicationEventHandler; + private final ConsumerNetworkThread consumerNetworkThread; private final Time time; private final Optional groupId; private final KafkaConsumerMetrics kafkaConsumerMetrics; @@ -143,10 +144,10 @@ public class PrototypeAsyncConsumer implements Consumer { private final long defaultApiTimeoutMs; private volatile boolean closed = false; private final List assignors; + private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates private boolean cachedSubscriptionHasAllFetchPositions; - private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); public PrototypeAsyncConsumer(final Properties properties, final Deserializer keyDeserializer, @@ -232,16 +233,40 @@ public PrototypeAsyncConsumer(final Time time, fetchMetricsManager, networkClientDelegateSupplier); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, + requestManagersSupplier, metadata, applicationEventQueue, - requestManagersSupplier); - this.applicationEventHandler = new ApplicationEventHandler(logContext, + Optional.empty()); + this.consumerNetworkThread = new ConsumerNetworkThread(logContext, time, - applicationEventQueue, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); - this.backgroundEventProcessor = new BackgroundEventProcessor(logContext, backgroundEventQueue); + ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( + subscriptions, + metrics, + CONSUMER_METRIC_GROUP_PREFIX + ); + ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( + logContext, + subscriptions, + time, + sensors + ); + this.applicationEventHandler = new ApplicationEventHandler(logContext, applicationEventQueue) { + @Override + public void add(ApplicationEvent event) { + super.add(event); + consumerNetworkThread.wakeup(); + } + }; + + this.backgroundEventProcessor = new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -263,6 +288,7 @@ public PrototypeAsyncConsumer(final Time time, time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); + this.consumerNetworkThread.start(); config.logUnused(); AppInfoParser.registerAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics, time.milliseconds()); @@ -286,7 +312,9 @@ public PrototypeAsyncConsumer(LogContext logContext, ConsumerInterceptors interceptors, Time time, ApplicationEventHandler applicationEventHandler, + ConsumerNetworkThread consumerNetworkThread, BlockingQueue backgroundEventQueue, + BackgroundEventProcessor backgroundEventProcessor, Metrics metrics, SubscriptionState subscriptions, ConsumerMetadata metadata, @@ -302,7 +330,7 @@ public PrototypeAsyncConsumer(LogContext logContext, this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; this.interceptors = Objects.requireNonNull(interceptors); this.time = time; - this.backgroundEventProcessor = new BackgroundEventProcessor(logContext, backgroundEventQueue); + this.backgroundEventProcessor = backgroundEventProcessor; this.metrics = metrics; this.groupId = Optional.ofNullable(groupId); this.metadata = metadata; @@ -310,8 +338,11 @@ public PrototypeAsyncConsumer(LogContext logContext, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.deserializers = deserializers; this.applicationEventHandler = applicationEventHandler; + this.consumerNetworkThread = consumerNetworkThread; this.assignors = assignors; this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); + + this.consumerNetworkThread.start(); } /** @@ -336,6 +367,7 @@ public ConsumerRecords poll(final Duration timeout) { } do { + backgroundEventProcessor.process(); updateAssignmentMetadataIfNeeded(timer); final Fetch fetch = pollForFetches(timer); @@ -709,8 +741,8 @@ private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference firstException = new AtomicReference<>(); - if (applicationEventHandler != null) - closeQuietly(() -> applicationEventHandler.close(timeout), "Failed to close application event handler with a timeout(ms)=" + timeout, firstException); + if (consumerNetworkThread != null) + closeQuietly(() -> consumerNetworkThread.close(timeout), "Failed to close consumer network thread with a timeout(ms)=" + timeout, firstException); closeQuietly(fetchBuffer, "Failed to close the fetch buffer", firstException); closeQuietly(interceptors, "consumer interceptors", firstException); @@ -956,14 +988,14 @@ private Fetch pollForFetches(Timer timer) { * *

* - * This method will {@link ApplicationEventHandler#wakeupNetworkThread() wake up} the {@link ConsumerNetworkThread} before - * retuning. This is done as an optimization so that the next round of data can be pre-fetched. + * This method will {@link ConsumerNetworkThread#wakeup() wake up the network thread} before returning. This is + * done as an optimization so that the next round of data can be pre-fetched. */ private Fetch collectFetch() { final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); // Notify the network thread to wake up and start the next round of fetching. - applicationEventHandler.wakeupNetworkThread(); + consumerNetworkThread.wakeup(); return fetch; } 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 b600a528164fe..8606b49729066 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 @@ -160,7 +160,19 @@ protected RequestManagers create() { backgroundEventHandler, groupState.groupId); commit = new CommitRequestManager(time, logContext, subscriptions, config, coordinator, groupState); - MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext); + AssignmentReconciler assignmentReconciler = new AssignmentReconciler( + logContext, + subscriptions, + backgroundEventQueue + ); + MembershipManager membershipManager = new MembershipManagerImpl( + logContext, + assignmentReconciler, + metadata, + groupState.groupId, + Optional.empty(), + Optional.empty() + ); heartbeatRequestManager = new HeartbeatRequestManager( logContext, time, 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 133836da3b753..7c2197318e5a7 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 @@ -25,7 +25,8 @@ public abstract class ApplicationEvent { public enum Type { COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, - LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA + LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, + PARTITION_RECONCILIATION_COMPLETE, PARTITION_LOST_COMPLETE } private final Type type; 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 2917d507d7f02..2fba532771b48 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,53 +17,32 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -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; -import java.io.Closeable; -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; /** * An event handler that receives {@link ApplicationEvent application events} from the application thread which * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. */ -public class ApplicationEventHandler implements Closeable { +public class ApplicationEventHandler { private final Logger log; private final BlockingQueue applicationEventQueue; - private final ConsumerNetworkThread networkThread; - private final IdempotentCloser closer = new IdempotentCloser(); public ApplicationEventHandler(final LogContext logContext, - final Time time, - final BlockingQueue applicationEventQueue, - final Supplier applicationEventProcessorSupplier, - final Supplier networkClientDelegateSupplier, - final Supplier requestManagersSupplier) { + final BlockingQueue applicationEventQueue) { this.log = logContext.logger(ApplicationEventHandler.class); this.applicationEventQueue = applicationEventQueue; - this.networkThread = new ConsumerNetworkThread(logContext, - time, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); - this.networkThread.start(); } /** - * Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread} - * to alert the network I/O thread that it has something to process. + * Add an {@link ApplicationEvent} to the handler. * * @param event An {@link ApplicationEvent} created by the application thread */ @@ -71,14 +50,6 @@ public void add(final ApplicationEvent event) { Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null"); log.trace("Enqueued event: {}", event); applicationEventQueue.add(event); - wakeupNetworkThread(); - } - - /** - * Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the next event(s) from the queue. - */ - public void wakeupNetworkThread() { - networkThread.wakeup(); } /** @@ -100,16 +71,4 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t add(event); return event.get(timer); } - - @Override - public void close() { - close(Duration.ZERO); - } - - public void close(final Duration timeout) { - closer.close( - () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), - () -> log.warn("The application event handler was already closed") - ); - } } 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 ccbdd21b9dc24..57626050a31e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -20,6 +20,7 @@ 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.MembershipManager; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; @@ -44,15 +45,18 @@ public class ApplicationEventProcessor extends EventProcessor private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; + private final Optional membershipManager; public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, - final ConsumerMetadata metadata) { + final ConsumerMetadata metadata, + final Optional membershipManager) { super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; + this.membershipManager = membershipManager; } /** @@ -104,6 +108,14 @@ public void process(ApplicationEvent event) { processValidatePositionsEvent(); return; + case PARTITION_RECONCILIATION_COMPLETE: + process((RebalanceCompleteEvent) event); + return; + + case PARTITION_LOST_COMPLETE: + process((PartitionLostCompleteEvent) event); + return; + default: log.warn("Application event type " + event.type() + " was not expected"); } @@ -185,9 +197,10 @@ private void process(final TopicMetadataApplicationEvent event) { * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, + final Supplier requestManagersSupplier, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, - final Supplier requestManagersSupplier) { + final Optional membershipManager) { return new CachedSupplier() { @Override protected ApplicationEventProcessor create() { @@ -196,9 +209,18 @@ protected ApplicationEventProcessor create() { logContext, applicationEventQueue, requestManagers, - metadata + metadata, + membershipManager ); } }; } + + private void process(final RebalanceCompleteEvent event) { + membershipManager.ifPresent(mm -> mm.completeReconcile(event.revokedPartitions(), event.assignedPartitions(), event.error())); + } + + private void process(final PartitionLostCompleteEvent event) { + membershipManager.ifPresent(mm -> mm.completeLost(event.lostPartitions(), event.error())); + } } 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 a7dc3e454a776..13f2cc4db5bd2 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 @@ -26,10 +26,10 @@ public abstract class BackgroundEvent { public enum Type { - ERROR, + ERROR, PARTITION_RECONCILIATION_STARTED, PARTITION_LOST_STARTED } - protected final Type type; + private final Type type; public BackgroundEvent(Type type) { this.type = Objects.requireNonNull(type); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index cafd4fba492ec..ba2bcfbba3b18 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -18,12 +18,17 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerInvoker; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; +import java.util.Optional; +import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicReference; + /** * An {@link EventProcessor} that is created and executes in the application thread for the purpose of processing * {@link BackgroundEvent background events} generated by the {@link ConsumerNetworkThread network thread}. @@ -36,9 +41,16 @@ */ public class BackgroundEventProcessor extends EventProcessor { + private final ApplicationEventHandler applicationEventHandler; + private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; + public BackgroundEventProcessor(final LogContext logContext, - final BlockingQueue backgroundEventQueue) { + final BlockingQueue backgroundEventQueue, + final ApplicationEventHandler applicationEventHandler, + final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { super(logContext, backgroundEventQueue); + this.applicationEventHandler = applicationEventHandler; + this.rebalanceListenerInvoker = rebalanceListenerInvoker; } /** @@ -58,10 +70,22 @@ public void process() { @Override public void process(final BackgroundEvent event) { - if (event.type() == BackgroundEvent.Type.ERROR) - process((ErrorBackgroundEvent) event); - else - throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); + switch (event.type()) { + case ERROR: + process((ErrorBackgroundEvent) event); + return; + + case PARTITION_RECONCILIATION_STARTED: + process((RebalanceStartedEvent) event); + return; + + case PARTITION_LOST_STARTED: + process((PartitionLostStartedEvent) event); + return; + + default: + throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); + } } @Override @@ -72,4 +96,38 @@ protected Class getEventClass() { private void process(final ErrorBackgroundEvent event) { throw event.error(); } + + private void process(final RebalanceStartedEvent event) { + final SortedSet revokedPartitions = event.revokedPartitions(); + final SortedSet assignedPartitions = event.assignedPartitions(); + Optional error = Optional.empty(); + + try { + rebalanceListenerInvoker.rebalance(revokedPartitions, assignedPartitions); + } catch (KafkaException e) { + error = Optional.of(e); + throw e; + } finally { + ApplicationEvent invokedEvent = new RebalanceCompleteEvent( + revokedPartitions, + assignedPartitions, + error); + applicationEventHandler.add(invokedEvent); + } + } + + private void process(final PartitionLostStartedEvent event) { + final SortedSet lostPartitions = event.lostPartitions(); + Optional error = Optional.empty(); + + try { + rebalanceListenerInvoker.lose(lostPartitions); + } catch (KafkaException e) { + error = Optional.of(e); + throw e; + } finally { + ApplicationEvent invokedEvent = new PartitionLostCompleteEvent(lostPartitions, error); + applicationEventHandler.add(invokedEvent); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java new file mode 100644 index 0000000000000..a087cce8151f0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java @@ -0,0 +1,82 @@ +/* + * 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.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.KafkaException; + +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.SortedSet; + +/** + * Event that signifies that the application thread has executed the + * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback. If the callback execution threw an error, + * it is included in the event should any event listener want to know. + */ +public class PartitionLostCompleteEvent extends ApplicationEvent { + + private final SortedSet lostPartitions; + private final Optional error; + + public PartitionLostCompleteEvent(SortedSet lostPartitions, Optional error) { + super(Type.PARTITION_LOST_COMPLETE); + this.lostPartitions = Collections.unmodifiableSortedSet(lostPartitions); + this.error = error; + } + + public SortedSet lostPartitions() { + return lostPartitions; + } + + 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; + + PartitionLostCompleteEvent that = (PartitionLostCompleteEvent) o; + + return lostPartitions.equals(that.lostPartitions) && error.equals(that.error); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + lostPartitions.hashCode(); + result = 31 * result + error.hashCode(); + return result; + } + + @Override + protected String toStringBase() { + return super.toStringBase() + ", lostPartitions=" + lostPartitions + ", error=" + error; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java new file mode 100644 index 0000000000000..ecad0fa7aaa57 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java @@ -0,0 +1,76 @@ +/* + * 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.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; + +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.SortedSet; + +/** + * Event that signifies that the background thread has determined that the member should abandon its partition + * assignment. This event will be processed by the application thread when the next {@link Consumer#poll(Duration)} + * call is performed by the user. When processed, the application thread should invoke the + * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback with the given partitions. + */ +public class PartitionLostStartedEvent extends BackgroundEvent { + + private final SortedSet lostPartitions; + + public PartitionLostStartedEvent(SortedSet lostPartitions) { + super(Type.PARTITION_LOST_STARTED); + this.lostPartitions = Collections.unmodifiableSortedSet(lostPartitions); + } + + public SortedSet lostPartitions() { + return lostPartitions; + } + + @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; + + PartitionLostStartedEvent that = (PartitionLostStartedEvent) o; + + return lostPartitions.equals(that.lostPartitions); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + lostPartitions.hashCode(); + return result; + } + + @Override + protected String toStringBase() { + return super.toStringBase() + ", lostPartitions=" + lostPartitions; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java new file mode 100644 index 0000000000000..65f6af7a02faf --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java @@ -0,0 +1,97 @@ +/* + * 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.ConsumerRebalanceListener; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.SortedSet; + +/** + * Event that signifies that the application thread has executed the + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and + * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks. If either callback execution threw + * an error, the first error encountered is included in the event should any event listener want to know. + */ +public class RebalanceCompleteEvent extends ApplicationEvent { + + private final SortedSet revokedPartitions; + private final SortedSet assignedPartitions; + private final Optional error; + + public RebalanceCompleteEvent(SortedSet revokedPartitions, + SortedSet assignedPartitions, + Optional error) { + super(Type.PARTITION_RECONCILIATION_COMPLETE); + this.revokedPartitions = Collections.unmodifiableSortedSet(revokedPartitions); + this.assignedPartitions = Collections.unmodifiableSortedSet(assignedPartitions); + this.error = error; + } + + public SortedSet revokedPartitions() { + return revokedPartitions; + } + + public SortedSet assignedPartitions() { + return assignedPartitions; + } + + 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; + + RebalanceCompleteEvent that = (RebalanceCompleteEvent) o; + + return revokedPartitions.equals(that.revokedPartitions) && + assignedPartitions.equals(that.assignedPartitions) && + error.equals(that.error); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + revokedPartitions.hashCode(); + result = 31 * result + assignedPartitions.hashCode(); + result = 31 * result + error.hashCode(); + return result; + } + + @Override + protected String toStringBase() { + return super.toStringBase() + + ", revokedPartitions=" + revokedPartitions + + ", assignedPartitions=" + assignedPartitions + + ", error=" + error; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java new file mode 100644 index 0000000000000..bf0ae25b9c506 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; + +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.SortedSet; + +/** + * Event that signifies that the background thread has started the partition assignment process. This event will + * be processed by the application thread when the next {@link Consumer#poll(Duration)} call is performed by the + * user. When processed, the application thread should invoke both the + * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and + * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks with the given partitions. + */ +public class RebalanceStartedEvent extends BackgroundEvent { + + private final SortedSet revokedPartitions; + private final SortedSet assignedPartitions; + + public RebalanceStartedEvent(SortedSet revokedPartitions, + SortedSet assignedPartitions) { + super(Type.PARTITION_RECONCILIATION_STARTED); + this.revokedPartitions = Collections.unmodifiableSortedSet(revokedPartitions); + this.assignedPartitions = Collections.unmodifiableSortedSet(assignedPartitions); + } + + public SortedSet revokedPartitions() { + return revokedPartitions; + } + + public SortedSet assignedPartitions() { + return assignedPartitions; + } + + @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; + + RebalanceStartedEvent that = (RebalanceStartedEvent) o; + + return revokedPartitions.equals(that.revokedPartitions) && + assignedPartitions.equals(that.assignedPartitions); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + revokedPartitions.hashCode(); + result = 31 * result + assignedPartitions.hashCode(); + return result; + } + + @Override + protected String toStringBase() { + return super.toStringBase() + + ", revokedPartitions=" + revokedPartitions + + ", assignedPartitions=" + assignedPartitions; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java new file mode 100644 index 0000000000000..b4c5ab5da6a87 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.PartitionLostCompleteEvent; +import org.apache.kafka.clients.consumer.internals.events.PartitionLostStartedEvent; +import org.apache.kafka.clients.consumer.internals.events.RebalanceCompleteEvent; +import org.apache.kafka.clients.consumer.internals.events.RebalanceStartedEvent; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.BlockingQueue; + +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_TOPIC_NAME; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class AssignmentReconcilerTest { + + private ConsumerTestBuilder testBuilder; + private SubscriptionState subscriptions; + private BlockingQueue applicationEventQueue; + private BlockingQueue backgroundEventQueue; + private BackgroundEventProcessor backgroundEventProcessor; + private AssignmentReconciler reconciler; + + @BeforeEach + public void setup() { + setup(new NoOpConsumerRebalanceListener()); + } + + @AfterEach + public void tearDown() { + if (testBuilder != null) { + testBuilder.close(); + } + } + + @Test + public void testAssignment() { + // Create our initial assignment + Set assignment = newTopicPartitions(0, 1, 2, 3); + + // Start the reconciliation process. At this point, since there are no partitions assigned to our + // subscriptions, we don't need to revoke anything. Validate that after our initial step that we haven't + // prematurely assigned anything to the subscriptions. + reconciler.startReconcile(assignment); + assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); + + // Grab the background event. Because we didn't remove any partitions, but only added them, jump + // directly to the assign partitions. Let's verify that there's an appropriate event on the + // background event queue, and it has the correct partitions. + RebalanceStartedEvent event = pollBackgroundEvent(); + assertEquals(event.assignedPartitions(), assignment); + + // Complete the future to signal to the reconciler that the ConsumerRebalanceListener callback + // has completed. This will trigger the "commit" of the partition assignment to the subscriptions. + assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); + backgroundEventProcessor.process(event); + RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); + assertEquals(invokedEvent.assignedPartitions(), assignment); + assertEquals(Optional.empty(), invokedEvent.error()); + reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + assertEquals(assignment, subscriptions.assignedPartitions()); + } + + @Test + public void testAssignmentAndRevocation() { + Set originalAssignment = newTopicPartitions(0, 1, 2, 3); + + // Create our initial assignment that adds four partitions + { + // Start the reconciliation process. At this point, since there are no partitions assigned to our + // subscriptions, we don't need to revoke anything. Validate that after our initial step that we haven't + // prematurely assigned anything to the subscriptions. + reconciler.startReconcile(originalAssignment); + assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); + + // Grab the background event. Because we didn't remove any partitions, but only added them, jump + // directly to the assign partitions. Let's verify that there's an appropriate event on the + // background event queue, and it has the correct partitions. + RebalanceStartedEvent event = pollBackgroundEvent(); + assertEquals(event.assignedPartitions(), originalAssignment); + + // Now process the callback. + backgroundEventProcessor.process(event); + RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); + assertEquals(invokedEvent.assignedPartitions(), originalAssignment); + assertEquals(Optional.empty(), invokedEvent.error()); + reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + assertEquals(originalAssignment, subscriptions.assignedPartitions()); + } + + // Create our follow-up assignment that removes two partitions. + { + Set newAssignment = newTopicPartitions(0, 2); + Set expectedRevoked = newTopicPartitions(1, 3); + + // We get another assignment. Since we have partitions assigned, we will need to revoke some + // old partitions that are no longer part of the new target assignment. + reconciler.startReconcile(newAssignment); + assertEquals(originalAssignment, subscriptions.assignedPartitions()); + + // Grab the background event. We are removing some partitions, so verify that we have the correct event + // type on the background event queue, and it has the correct partitions to remove. + RebalanceStartedEvent event = pollBackgroundEvent(); + assertEquals(event.revokedPartitions(), expectedRevoked); + + // Now process the callback. + backgroundEventProcessor.process(event); + RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); + assertEquals(invokedEvent.revokedPartitions(), expectedRevoked); + reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + assertEquals(newAssignment, subscriptions.assignedPartitions()); + } + } + + @Test + public void testLose() { + // This mimics having set up an assignment already. + SortedSet partitions = newTopicPartitions(0, 1, 2, 3); + subscriptions.assignFromSubscribed(partitions); + + assertEquals(partitions, subscriptions.assignedPartitions()); + reconciler.startLost(); + assertEquals(partitions, subscriptions.assignedPartitions()); + + // Grab the background event. Because we are "losing" the partitions, verify that there's an + // appropriate event on the background event queue, and it still has the partitions. + PartitionLostStartedEvent event = pollBackgroundEvent(); + assertEquals(partitions, event.lostPartitions()); + + // Now process the callback. Afterward we should have an empty set of partitions + backgroundEventProcessor.process(event); + PartitionLostCompleteEvent invokedEvent = pollApplicationEvent(); + assertEquals(partitions, invokedEvent.lostPartitions()); + reconciler.completeLost(invokedEvent.lostPartitions()); + assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); + } + + @Test + public void testRevocationFailure() { + ConsumerRebalanceListener failingListener = new ConsumerRebalanceListener() { + @Override + public void onPartitionsRevoked(Collection partitions) { + throw new KafkaException("Simulating callback failure"); + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + + } + }; + + // Perform a tear down and set up our new listener. + tearDown(); + setup(failingListener); + + // This mimics having set up an assignment of four partitions. + SortedSet partitions = newTopicPartitions(0, 1, 2, 3); + subscriptions.assignFromSubscribed(partitions); + + // When revoking partitions, we get an error. This should not stop the reconciliation process, though. + { + Set assignment = newTopicPartitions(0, 2); + + // Start the reconciliation process. + reconciler.startReconcile(assignment); + + RebalanceStartedEvent event = pollBackgroundEvent(); + assertEquals(event.revokedPartitions(), newTopicPartitions(1, 3)); + + // Now process the callback. It should throw an exception, but it should still finish and allow + // the reconciler to alter the assigned partition set. + assertThrows(KafkaException.class, () -> backgroundEventProcessor.process(event)); + RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); + assertEquals(invokedEvent.revokedPartitions(), newTopicPartitions(1, 3)); + reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + assertEquals(newTopicPartitions(0, 2), subscriptions.assignedPartitions()); + } + } + + private SortedSet newTopicPartitions(Integer... partitions) { + SortedSet topicPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); + + if (partitions != null) { + for (int partition : partitions) + topicPartitions.add(new TopicPartition(DEFAULT_TOPIC_NAME, partition)); + } + + return topicPartitions; + } + + private void setup(ConsumerRebalanceListener listener) { + testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); + + // Create our subscriptions and subscribe to the topics. + subscriptions = testBuilder.subscriptions; + subscriptions.subscribe(Collections.singleton(DEFAULT_TOPIC_NAME), listener); + + // We need the background event queue to check for the events from the network thread to the application thread + // to signal the ConsumerRebalanceListener callbacks. + applicationEventQueue = testBuilder.applicationEventQueue; + backgroundEventQueue = testBuilder.backgroundEventQueue; + + backgroundEventProcessor = testBuilder.backgroundEventProcessor; + reconciler = testBuilder.assignmentReconciler.orElseThrow(() -> new IllegalStateException("Should be in a group")); + } + + @SuppressWarnings("unchecked") + private T pollApplicationEvent() { + ApplicationEvent event = applicationEventQueue.poll(); + assertNotNull(event); + return (T) event; + } + + @SuppressWarnings("unchecked") + private T pollBackgroundEvent() { + BackgroundEvent event = backgroundEventQueue.poll(); + assertNotNull(event); + return (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 894f8305d02bd..39e5ba8411dbe 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 @@ -28,6 +28,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.BackgroundEventProcessor; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.MetadataResponse; @@ -39,7 +40,6 @@ import java.io.Closeable; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -50,10 +50,12 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_INSTANCE_ID_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; 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.common.utils.Utils.closeQuietly; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.spy; @SuppressWarnings("ClassDataAbstractionCoupling") @@ -63,8 +65,11 @@ public class ConsumerTestBuilder implements Closeable { static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; static final int DEFAULT_REQUEST_TIMEOUT_MS = 500; static final int DEFAULT_MAX_POLL_INTERVAL_MS = 10000; + static final String DEFAULT_TOPIC_NAME = "sample-topic-name"; + static final Uuid DEFAULT_TOPIC_ID = Uuid.randomUuid(); static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id"; static final String DEFAULT_GROUP_ID = "group-id"; + static final String DEFAULT_MEMBER_ID = "test-member-1"; static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; @@ -85,14 +90,17 @@ public class ConsumerTestBuilder implements Closeable { final Optional coordinatorRequestManager; final Optional commitRequestManager; final Optional heartbeatRequestManager; - final Optional membershipManager; + final Optional membershipManager; final Optional heartbeatRequestState; + final Optional assignmentReconciler; final TopicMetadataRequestManager topicMetadataRequestManager; final FetchRequestManager fetchRequestManager; final RequestManagers requestManagers; public final ApplicationEventProcessor applicationEventProcessor; public final BackgroundEventProcessor backgroundEventProcessor; + public final ApplicationEventHandler applicationEventHandler; public final BackgroundEventHandler backgroundEventHandler; + public final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; final MockClient client; final Optional groupInfo; @@ -104,6 +112,7 @@ public ConsumerTestBuilder(Optional groupInfo) { this.groupInfo = groupInfo; this.applicationEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventQueue = new LinkedBlockingQueue<>(); + this.applicationEventHandler = spy(new ApplicationEventHandler(logContext, applicationEventQueue)); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( 100, @@ -141,16 +150,18 @@ public ConsumerTestBuilder(Optional groupInfo) { this.metricsManager = createFetchMetricsManager(metrics); this.client = new MockClient(time, metadata); - MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap() { - { - String topic1 = "test1"; - put(topic1, 1); - String topic2 = "test2"; - put(topic2, 1); - } - }); + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWithIds( + 1, + Collections.singletonMap(DEFAULT_TOPIC_NAME, 1), + Collections.singletonMap(DEFAULT_TOPIC_NAME, DEFAULT_TOPIC_ID) + ); this.client.updateMetadata(metadataResponse); + assertEquals(1, metadata.topicNames().size()); + assertEquals(DEFAULT_TOPIC_NAME, metadata.topicNames().get(DEFAULT_TOPIC_ID)); + assertEquals(1, metadata.topicIds().size()); + assertEquals(DEFAULT_TOPIC_ID, metadata.topicIds().get(DEFAULT_TOPIC_NAME)); + this.networkClientDelegate = spy(new NetworkClientDelegate(time, config, logContext, @@ -182,12 +193,19 @@ public ConsumerTestBuilder(Optional groupInfo) { config, coordinator, groupState)); - MembershipManager mm = spy( + AssignmentReconciler assignmentReconciler = new AssignmentReconciler( + logContext, + subscriptions, + backgroundEventQueue + ); + MembershipManagerImpl mm = spy( new MembershipManagerImpl( + logContext, + assignmentReconciler, + metadata, gi.groupState.groupId, - gi.groupState.groupInstanceId.orElse(null), - null, - logContext + gi.groupState.groupInstanceId, + Optional.empty() ) ); HeartbeatRequestManager.HeartbeatRequestState state = spy(new HeartbeatRequestManager.HeartbeatRequestState(logContext, @@ -211,12 +229,14 @@ public ConsumerTestBuilder(Optional groupInfo) { this.heartbeatRequestManager = Optional.of(heartbeat); this.heartbeatRequestState = Optional.of(state); this.membershipManager = Optional.of(mm); + this.assignmentReconciler = Optional.of(assignmentReconciler); } else { this.coordinatorRequestManager = Optional.empty(); this.commitRequestManager = Optional.empty(); this.heartbeatRequestManager = Optional.empty(); this.heartbeatRequestState = Optional.empty(); this.membershipManager = Optional.empty(); + this.assignmentReconciler = Optional.empty(); } this.fetchBuffer = new FetchBuffer(logContext); @@ -228,8 +248,7 @@ public ConsumerTestBuilder(Optional groupInfo) { fetchBuffer, metricsManager, networkClientDelegate)); - this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, - config)); + this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config)); this.requestManagers = new RequestManagers(logContext, offsetsRequestManager, topicMetadataRequestManager, @@ -241,9 +260,28 @@ public ConsumerTestBuilder(Optional groupInfo) { logContext, applicationEventQueue, requestManagers, - metadata) + metadata, + Optional.ofNullable(membershipManager.orElse(null))) + ); + ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( + subscriptions, + metrics, + CONSUMER_METRIC_GROUP_PREFIX + ); + this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( + logContext, + subscriptions, + time, + sensors + ); + this.backgroundEventProcessor = spy( + new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ) ); - this.backgroundEventProcessor = spy(new BackgroundEventProcessor(logContext, backgroundEventQueue)); } @Override @@ -278,32 +316,7 @@ public void close() { } } - public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { - - public final ApplicationEventHandler applicationEventHandler; - - public ApplicationEventHandlerTestBuilder() { - this(createDefaultGroupInformation()); - } - - public ApplicationEventHandlerTestBuilder(Optional groupInfo) { - super(groupInfo); - this.applicationEventHandler = spy(new ApplicationEventHandler( - logContext, - time, - applicationEventQueue, - () -> applicationEventProcessor, - () -> networkClientDelegate, - () -> requestManagers)); - } - - @Override - public void close() { - closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); - } - } - - public static class PrototypeAsyncConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { + public static class PrototypeAsyncConsumerTestBuilder extends ConsumerNetworkThreadTestBuilder { final PrototypeAsyncConsumer consumer; @@ -331,7 +344,9 @@ public PrototypeAsyncConsumerTestBuilder(Optional groupInfo) { new ConsumerInterceptors<>(Collections.emptyList()), time, applicationEventHandler, + consumerNetworkThread, backgroundEventQueue, + backgroundEventProcessor, metrics, subscriptions, metadata, 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 d78bbf2ab63ee..7874af6fe24e8 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 @@ -17,17 +17,20 @@ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; -import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; import java.util.Optional; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_MEMBER_ID; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_TOPIC_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -35,50 +38,57 @@ public class MembershipManagerImplTest { - private static final String GROUP_ID = "test-group"; - private static final String MEMBER_ID = "test-member-1"; private static final int MEMBER_EPOCH = 1; - private final LogContext logContext = new LogContext(); + private ConsumerTestBuilder testBuilder; + private MembershipManagerImpl membershipManager; + + @BeforeEach + public void setup() { + testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); + membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); + } + + @AfterEach + public void tearDown() { + if (testBuilder != null) { + testBuilder.close(); + } + } @Test public void testMembershipManagerServerAssignor() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(Optional.empty(), membershipManager.serverAssignor()); - membershipManager = new MembershipManagerImpl(GROUP_ID, "instance1", "Uniform", logContext); + membershipManager = new MembershipManagerImpl( + testBuilder.logContext, + testBuilder.assignmentReconciler.orElseThrow(IllegalStateException::new), + testBuilder.metadata, + DEFAULT_GROUP_ID, + Optional.of("instance1"), + Optional.of("Uniform") + ); assertEquals(Optional.of("Uniform"), membershipManager.serverAssignor()); } - @Test - public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { - new MembershipManagerImpl(GROUP_ID, logContext); - new MembershipManagerImpl(GROUP_ID, null, null, logContext); - } - @Test public void testTransitionToReconcilingOnlyIfAssignmentReceived() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); - ConsumerGroupHeartbeatResponse responseWithoutAssignment = - createConsumerGroupHeartbeatResponse(null); + ConsumerGroupHeartbeatResponse responseWithoutAssignment = createHeartbeatResponse(); membershipManager.updateState(responseWithoutAssignment.data()); assertNotEquals(MemberState.RECONCILING, membershipManager.state()); - ConsumerGroupHeartbeatResponse responseWithAssignment = - createConsumerGroupHeartbeatResponse(createAssignment()); + ConsumerGroupHeartbeatResponse responseWithAssignment = createHeartbeatResponse(createAssignment()); membershipManager.updateState(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @Test public void testMemberIdAndEpochResetOnFencedMembers() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(null); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(MEMBER_ID, membershipManager.memberId()); + assertEquals(DEFAULT_MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); membershipManager.transitionToFenced(); @@ -88,12 +98,10 @@ public void testMemberIdAndEpochResetOnFencedMembers() { @Test public void testTransitionToFailure() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(null); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(MEMBER_ID, membershipManager.memberId()); + assertEquals(DEFAULT_MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); membershipManager.transitionToFailed(); @@ -102,45 +110,40 @@ public void testTransitionToFailure() { @Test public void testFencingWhenStateIsStable() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - testStateUpdateOnFenceError(membershipManager); + testStateUpdateOnFenceError(); } @Test public void testFencingWhenStateIsReconciling() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(createAssignment()); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(createAssignment()); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - testStateUpdateOnFenceError(membershipManager); + testStateUpdateOnFenceError(); } @Test public void testFatalFailureWhenStateIsUnjoined() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); - testStateUpdateOnFatalFailure(membershipManager); + testStateUpdateOnFatalFailure(); } @Test public void testFatalFailureWhenStateIsStable() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - testStateUpdateOnFatalFailure(membershipManager); + testStateUpdateOnFatalFailure(); } @Test public void testFencingShouldNotHappenWhenStateIsUnjoined() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); // Getting fenced when the member is not part of the group is not expected and should @@ -150,80 +153,62 @@ public void testFencingShouldNotHappenWhenStateIsUnjoined() { @Test public void testUpdateStateFailsOnResponsesWithErrors() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); // Updating state with a heartbeat response containing errors cannot be performed and // should fail. - ConsumerGroupHeartbeatResponse unknownMemberResponse = - createConsumerGroupHeartbeatResponseWithError(Errors.UNKNOWN_MEMBER_ID); - assertThrows(IllegalArgumentException.class, - () -> membershipManager.updateState(unknownMemberResponse.data())); + ConsumerGroupHeartbeatResponse unknownMemberResponse = createHeartbeatResponse(Errors.UNKNOWN_MEMBER_ID); + assertThrows(IllegalArgumentException.class, () -> membershipManager.updateState(unknownMemberResponse.data())); } @Test public void testAssignmentUpdatedAsReceivedAndProcessed() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); ConsumerGroupHeartbeatResponseData.Assignment newAssignment = createAssignment(); - ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(newAssignment); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(newAssignment); membershipManager.updateState(heartbeatResponse.data()); // Target assignment should be in the process of being reconciled - checkAssignments(membershipManager, null, newAssignment); + checkAssignments(null, newAssignment); // Mark assignment processing completed membershipManager.onTargetAssignmentProcessComplete(newAssignment); // Target assignment should now be the current assignment - checkAssignments(membershipManager, newAssignment, null); + checkAssignments(newAssignment, null); } @Test public void testMemberFailsIfAssignmentReceivedWhileAnotherOnBeingReconciled() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); ConsumerGroupHeartbeatResponseData.Assignment newAssignment1 = createAssignment(); - membershipManager.updateState(createConsumerGroupHeartbeatResponse(newAssignment1).data()); + ConsumerGroupHeartbeatResponseData response1 = createHeartbeatResponse(newAssignment1).data(); + membershipManager.updateState(response1); // First target assignment received should be in the process of being reconciled - checkAssignments(membershipManager, null, newAssignment1); + checkAssignments(null, newAssignment1); // Second target assignment received while there is another one being reconciled ConsumerGroupHeartbeatResponseData.Assignment newAssignment2 = createAssignment(); - assertThrows(IllegalStateException.class, - () -> membershipManager.updateState(createConsumerGroupHeartbeatResponse(newAssignment2).data())); + ConsumerGroupHeartbeatResponseData response2 = createHeartbeatResponse(newAssignment2).data(); + assertThrows(IllegalStateException.class, () -> membershipManager.updateState(response2)); assertEquals(MemberState.FAILED, membershipManager.state()); } @Test public void testAssignmentUpdatedFailsIfAssignmentReconciledDoesNotMatchTargetAssignment() { - MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); - ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Arrays.asList(0, 1, 2)))); - ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(targetAssignment); + ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = createAssignment(0, 1, 2); + ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(targetAssignment); membershipManager.updateState(heartbeatResponse.data()); // Target assignment should be in the process of being reconciled - checkAssignments(membershipManager, null, targetAssignment); + checkAssignments(null, targetAssignment); // Mark assignment processing completed - ConsumerGroupHeartbeatResponseData.Assignment reconciled = - new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Collections.singletonList(0)))); + ConsumerGroupHeartbeatResponseData.Assignment reconciled = createAssignment(0); assertThrows(IllegalStateException.class, () -> membershipManager.onTargetAssignmentProcessComplete(reconciled)); } - private void checkAssignments( - MembershipManagerImpl membershipManager, - ConsumerGroupHeartbeatResponseData.Assignment expectedCurrentAssignment, - ConsumerGroupHeartbeatResponseData.Assignment expectedTargetAssignment) { + private void checkAssignments(ConsumerGroupHeartbeatResponseData.Assignment expectedCurrentAssignment, + ConsumerGroupHeartbeatResponseData.Assignment expectedTargetAssignment) { assertEquals(expectedCurrentAssignment, membershipManager.currentAssignment()); assertEquals(expectedTargetAssignment, membershipManager.targetAssignment().orElse(null)); } - private void testStateUpdateOnFenceError(MembershipManager membershipManager) { + private void testStateUpdateOnFenceError() { membershipManager.transitionToFenced(); assertEquals(MemberState.FENCED, membershipManager.state()); // Should reset member epoch and keep member id @@ -231,7 +216,7 @@ private void testStateUpdateOnFenceError(MembershipManager membershipManager) { assertEquals(0, membershipManager.memberEpoch()); } - private void testStateUpdateOnFatalFailure(MembershipManager membershipManager) { + private void testStateUpdateOnFatalFailure() { String initialMemberId = membershipManager.memberId(); int initialMemberEpoch = membershipManager.memberEpoch(); membershipManager.transitionToFailed(); @@ -241,30 +226,50 @@ private void testStateUpdateOnFatalFailure(MembershipManager membershipManager) assertEquals(initialMemberEpoch, membershipManager.memberEpoch()); } - private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse(ConsumerGroupHeartbeatResponseData.Assignment assignment) { - return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH) - .setAssignment(assignment)); + private ConsumerGroupHeartbeatResponse createHeartbeatResponse() { + return new ConsumerGroupHeartbeatResponse(createHeartbeatResponseData()); + } + + private ConsumerGroupHeartbeatResponse createHeartbeatResponse(ConsumerGroupHeartbeatResponseData.Assignment assignment) { + ConsumerGroupHeartbeatResponseData data = createHeartbeatResponseData() + .setAssignment(assignment); + return new ConsumerGroupHeartbeatResponse(data); } - private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError(Errors error) { - return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + private ConsumerGroupHeartbeatResponse createHeartbeatResponse(Errors error) { + ConsumerGroupHeartbeatResponseData data = createHeartbeatResponseData() .setErrorCode(error.code()) - .setMemberId(MEMBER_ID) - .setMemberEpoch(5)); + .setMemberEpoch(5); + return new ConsumerGroupHeartbeatResponse(data); + } + + private ConsumerGroupHeartbeatResponseData createHeartbeatResponseData() { + return new ConsumerGroupHeartbeatResponseData() + .setErrorCode(Errors.NONE.code()) + .setMemberId(DEFAULT_MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH); } private ConsumerGroupHeartbeatResponseData.Assignment createAssignment() { + return createAssignment( + createTopicPartitions(0, 1, 2), + createTopicPartitions(3, 4, 5) + ); + } + + private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(ConsumerGroupHeartbeatResponseData.TopicPartitions... partitions) { return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Arrays.asList(0, 1, 2)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Arrays.asList(3, 4, 5)) - )); + .setTopicPartitions(Arrays.asList(partitions)); + } + + private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(Integer... partitions) { + return new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList(createTopicPartitions(partitions))); + } + + private ConsumerGroupHeartbeatResponseData.TopicPartitions createTopicPartitions(Integer... partitions) { + return new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(DEFAULT_TOPIC_ID) + .setPartitions(Arrays.asList(partitions)); } } From 8560860e253a7093e1098b9a9f06a4c276eed179 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 15:50:55 -0700 Subject: [PATCH 02/68] Reverted previous change to decouple the network thread from the application event handler --- .../internals/PrototypeAsyncConsumer.java | 33 +++---- .../events/ApplicationEventHandler.java | 47 ++++++++- .../internals/AssignmentReconcilerTest.java | 95 ++++++------------- .../internals/ConsumerTestBuilder.java | 51 +++++++--- .../events/BackgroundEventHandlerTest.java | 4 +- 5 files changed, 123 insertions(+), 107 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 59713bf8cd2cb..fc23a66efb4a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -117,7 +117,6 @@ public class PrototypeAsyncConsumer implements Consumer { private final ApplicationEventHandler applicationEventHandler; - private final ConsumerNetworkThread consumerNetworkThread; private final Time time; private final Optional groupId; private final KafkaConsumerMetrics kafkaConsumerMetrics; @@ -237,11 +236,6 @@ public PrototypeAsyncConsumer(final Time time, metadata, applicationEventQueue, Optional.empty()); - this.consumerNetworkThread = new ConsumerNetworkThread(logContext, - time, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( subscriptions, metrics, @@ -253,13 +247,14 @@ public PrototypeAsyncConsumer(final Time time, time, sensors ); - this.applicationEventHandler = new ApplicationEventHandler(logContext, applicationEventQueue) { - @Override - public void add(ApplicationEvent event) { - super.add(event); - consumerNetworkThread.wakeup(); - } - }; + this.applicationEventHandler = new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier + ); this.backgroundEventProcessor = new BackgroundEventProcessor( logContext, @@ -288,7 +283,6 @@ public void add(ApplicationEvent event) { time); this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); - this.consumerNetworkThread.start(); config.logUnused(); AppInfoParser.registerAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics, time.milliseconds()); @@ -312,8 +306,6 @@ public PrototypeAsyncConsumer(LogContext logContext, ConsumerInterceptors interceptors, Time time, ApplicationEventHandler applicationEventHandler, - ConsumerNetworkThread consumerNetworkThread, - BlockingQueue backgroundEventQueue, BackgroundEventProcessor backgroundEventProcessor, Metrics metrics, SubscriptionState subscriptions, @@ -338,11 +330,8 @@ public PrototypeAsyncConsumer(LogContext logContext, this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.deserializers = deserializers; this.applicationEventHandler = applicationEventHandler; - this.consumerNetworkThread = consumerNetworkThread; this.assignors = assignors; this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); - - this.consumerNetworkThread.start(); } /** @@ -741,8 +730,8 @@ private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference firstException = new AtomicReference<>(); - if (consumerNetworkThread != null) - closeQuietly(() -> consumerNetworkThread.close(timeout), "Failed to close consumer network thread with a timeout(ms)=" + timeout, firstException); + if (applicationEventHandler != null) + closeQuietly(() -> applicationEventHandler.close(timeout), "Failed to close consumer network thread with a timeout(ms)=" + timeout, firstException); closeQuietly(fetchBuffer, "Failed to close the fetch buffer", firstException); closeQuietly(interceptors, "consumer interceptors", firstException); @@ -995,7 +984,7 @@ private Fetch collectFetch() { final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); // Notify the network thread to wake up and start the next round of fetching. - consumerNetworkThread.wakeup(); + applicationEventHandler.wakeupNetworkThread(); return fetch; } 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 2fba532771b48..2917d507d7f02 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,32 +17,53 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +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; +import java.io.Closeable; +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; /** * An event handler that receives {@link ApplicationEvent application events} from the application thread which * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. */ -public class ApplicationEventHandler { +public class ApplicationEventHandler implements Closeable { private final Logger log; private final BlockingQueue applicationEventQueue; + private final ConsumerNetworkThread networkThread; + private final IdempotentCloser closer = new IdempotentCloser(); public ApplicationEventHandler(final LogContext logContext, - final BlockingQueue applicationEventQueue) { + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier) { this.log = logContext.logger(ApplicationEventHandler.class); this.applicationEventQueue = applicationEventQueue; + this.networkThread = new ConsumerNetworkThread(logContext, + time, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); + this.networkThread.start(); } /** - * Add an {@link ApplicationEvent} to the handler. + * Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread} + * to alert the network I/O thread that it has something to process. * * @param event An {@link ApplicationEvent} created by the application thread */ @@ -50,6 +71,14 @@ public void add(final ApplicationEvent event) { Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null"); log.trace("Enqueued event: {}", event); applicationEventQueue.add(event); + wakeupNetworkThread(); + } + + /** + * Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the next event(s) from the queue. + */ + public void wakeupNetworkThread() { + networkThread.wakeup(); } /** @@ -71,4 +100,16 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t add(event); return event.get(timer); } + + @Override + public void close() { + close(Duration.ZERO); + } + + public void close(final Duration timeout) { + closer.close( + () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), + () -> log.warn("The application event handler was already closed") + ); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java index b4c5ab5da6a87..5610a93f0fc7f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 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.BackgroundEventProcessor; import org.apache.kafka.clients.consumer.internals.events.PartitionLostCompleteEvent; @@ -45,10 +46,11 @@ public class AssignmentReconcilerTest { - private ConsumerTestBuilder testBuilder; + private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; private SubscriptionState subscriptions; private BlockingQueue applicationEventQueue; private BlockingQueue backgroundEventQueue; + private ApplicationEventProcessor applicationEventProcessor; private BackgroundEventProcessor backgroundEventProcessor; private AssignmentReconciler reconciler; @@ -75,20 +77,11 @@ public void testAssignment() { reconciler.startReconcile(assignment); assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - // Grab the background event. Because we didn't remove any partitions, but only added them, jump - // directly to the assign partitions. Let's verify that there's an appropriate event on the - // background event queue, and it has the correct partitions. - RebalanceStartedEvent event = pollBackgroundEvent(); - assertEquals(event.assignedPartitions(), assignment); - // Complete the future to signal to the reconciler that the ConsumerRebalanceListener callback // has completed. This will trigger the "commit" of the partition assignment to the subscriptions. assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - backgroundEventProcessor.process(event); - RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); - assertEquals(invokedEvent.assignedPartitions(), assignment); - assertEquals(Optional.empty(), invokedEvent.error()); - reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + backgroundEventProcessor.process(); + applicationEventProcessor.process(); assertEquals(assignment, subscriptions.assignedPartitions()); } @@ -104,18 +97,9 @@ public void testAssignmentAndRevocation() { reconciler.startReconcile(originalAssignment); assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - // Grab the background event. Because we didn't remove any partitions, but only added them, jump - // directly to the assign partitions. Let's verify that there's an appropriate event on the - // background event queue, and it has the correct partitions. - RebalanceStartedEvent event = pollBackgroundEvent(); - assertEquals(event.assignedPartitions(), originalAssignment); - // Now process the callback. - backgroundEventProcessor.process(event); - RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); - assertEquals(invokedEvent.assignedPartitions(), originalAssignment); - assertEquals(Optional.empty(), invokedEvent.error()); - reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + backgroundEventProcessor.process(); + applicationEventProcessor.process(); assertEquals(originalAssignment, subscriptions.assignedPartitions()); } @@ -129,16 +113,9 @@ public void testAssignmentAndRevocation() { reconciler.startReconcile(newAssignment); assertEquals(originalAssignment, subscriptions.assignedPartitions()); - // Grab the background event. We are removing some partitions, so verify that we have the correct event - // type on the background event queue, and it has the correct partitions to remove. - RebalanceStartedEvent event = pollBackgroundEvent(); - assertEquals(event.revokedPartitions(), expectedRevoked); - // Now process the callback. - backgroundEventProcessor.process(event); - RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); - assertEquals(invokedEvent.revokedPartitions(), expectedRevoked); - reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + backgroundEventProcessor.process(); + applicationEventProcessor.process(); assertEquals(newAssignment, subscriptions.assignedPartitions()); } } @@ -153,16 +130,8 @@ public void testLose() { reconciler.startLost(); assertEquals(partitions, subscriptions.assignedPartitions()); - // Grab the background event. Because we are "losing" the partitions, verify that there's an - // appropriate event on the background event queue, and it still has the partitions. - PartitionLostStartedEvent event = pollBackgroundEvent(); - assertEquals(partitions, event.lostPartitions()); - - // Now process the callback. Afterward we should have an empty set of partitions - backgroundEventProcessor.process(event); - PartitionLostCompleteEvent invokedEvent = pollApplicationEvent(); - assertEquals(partitions, invokedEvent.lostPartitions()); - reconciler.completeLost(invokedEvent.lostPartitions()); + backgroundEventProcessor.process(); + applicationEventProcessor.process(); assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); } @@ -195,15 +164,10 @@ public void onPartitionsAssigned(Collection partitions) { // Start the reconciliation process. reconciler.startReconcile(assignment); - RebalanceStartedEvent event = pollBackgroundEvent(); - assertEquals(event.revokedPartitions(), newTopicPartitions(1, 3)); - // Now process the callback. It should throw an exception, but it should still finish and allow // the reconciler to alter the assigned partition set. - assertThrows(KafkaException.class, () -> backgroundEventProcessor.process(event)); - RebalanceCompleteEvent invokedEvent = pollApplicationEvent(); - assertEquals(invokedEvent.revokedPartitions(), newTopicPartitions(1, 3)); - reconciler.completeReconcile(invokedEvent.revokedPartitions(), invokedEvent.assignedPartitions()); + assertThrows(KafkaException.class, () -> backgroundEventProcessor.process()); + applicationEventProcessor.process(); assertEquals(newTopicPartitions(0, 2), subscriptions.assignedPartitions()); } } @@ -220,7 +184,7 @@ private SortedSet newTopicPartitions(Integer... partitions) { } private void setup(ConsumerRebalanceListener listener) { - testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); + testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); // Create our subscriptions and subscribe to the topics. subscriptions = testBuilder.subscriptions; @@ -228,24 +192,25 @@ private void setup(ConsumerRebalanceListener listener) { // We need the background event queue to check for the events from the network thread to the application thread // to signal the ConsumerRebalanceListener callbacks. - applicationEventQueue = testBuilder.applicationEventQueue; - backgroundEventQueue = testBuilder.backgroundEventQueue; +// applicationEventQueue = testBuilder.applicationEventQueue; +// backgroundEventQueue = testBuilder.backgroundEventQueue; + applicationEventProcessor = testBuilder.applicationEventProcessor; backgroundEventProcessor = testBuilder.backgroundEventProcessor; reconciler = testBuilder.assignmentReconciler.orElseThrow(() -> new IllegalStateException("Should be in a group")); } - @SuppressWarnings("unchecked") - private T pollApplicationEvent() { - ApplicationEvent event = applicationEventQueue.poll(); - assertNotNull(event); - return (T) event; - } - - @SuppressWarnings("unchecked") - private T pollBackgroundEvent() { - BackgroundEvent event = backgroundEventQueue.poll(); - assertNotNull(event); - return (T) event; - } +// @SuppressWarnings("unchecked") +// private T pollApplicationEvent() { +// ApplicationEvent event = applicationEventQueue.poll(); +// assertNotNull(event); +// return (T) event; +// } +// +// @SuppressWarnings("unchecked") +// private T pollBackgroundEvent() { +// BackgroundEvent event = backgroundEventQueue.poll(); +// assertNotNull(event); +// return (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 39e5ba8411dbe..a7e9bc4f27a40 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 @@ -97,8 +97,6 @@ public class ConsumerTestBuilder implements Closeable { final FetchRequestManager fetchRequestManager; final RequestManagers requestManagers; public final ApplicationEventProcessor applicationEventProcessor; - public final BackgroundEventProcessor backgroundEventProcessor; - public final ApplicationEventHandler applicationEventHandler; public final BackgroundEventHandler backgroundEventHandler; public final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; final MockClient client; @@ -112,7 +110,6 @@ public ConsumerTestBuilder(Optional groupInfo) { this.groupInfo = groupInfo; this.applicationEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventQueue = new LinkedBlockingQueue<>(); - this.applicationEventHandler = spy(new ApplicationEventHandler(logContext, applicationEventQueue)); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( 100, @@ -274,21 +271,12 @@ public ConsumerTestBuilder(Optional groupInfo) { time, sensors ); - this.backgroundEventProcessor = spy( - new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ) - ); } @Override public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); - closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); } public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { @@ -316,7 +304,42 @@ public void close() { } } - public static class PrototypeAsyncConsumerTestBuilder extends ConsumerNetworkThreadTestBuilder { + public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { + + public final ApplicationEventHandler applicationEventHandler; + public final BackgroundEventProcessor backgroundEventProcessor; + + public ApplicationEventHandlerTestBuilder() { + this(createDefaultGroupInformation()); + } + + public ApplicationEventHandlerTestBuilder(Optional groupInfo) { + super(groupInfo); + this.applicationEventHandler = spy(new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + () -> applicationEventProcessor, + () -> networkClientDelegate, + () -> requestManagers)); + this.backgroundEventProcessor = spy( + new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ) + ); + } + + @Override + public void close() { + closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); + closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); + } + } + + public static class PrototypeAsyncConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { final PrototypeAsyncConsumer consumer; @@ -344,8 +367,6 @@ public PrototypeAsyncConsumerTestBuilder(Optional groupInfo) { new ConsumerInterceptors<>(Collections.emptyList()), time, applicationEventHandler, - consumerNetworkThread, - backgroundEventQueue, backgroundEventProcessor, metrics, subscriptions, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java index 0670b8bdb7c3f..6ef2c1f64a436 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java @@ -32,14 +32,14 @@ public class BackgroundEventHandlerTest { - private ConsumerTestBuilder testBuilder; + private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private BackgroundEventProcessor backgroundEventProcessor; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder(); + testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(); backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; backgroundEventProcessor = testBuilder.backgroundEventProcessor; From 5d3efaa1e44c79dbfeb0745a594a678664e75f2c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 15:57:29 -0700 Subject: [PATCH 03/68] Updates to clean up code a bit --- .../kafka/clients/consumer/internals/MembershipManager.java | 5 +++++ .../clients/consumer/internals/MembershipManagerImpl.java | 3 ++- .../clients/consumer/internals/PrototypeAsyncConsumer.java | 2 +- .../consumer/internals/events/ApplicationEventProcessor.java | 2 +- .../clients/consumer/internals/ConsumerTestBuilder.java | 2 +- .../consumer/internals/MembershipManagerImplTest.java | 2 +- 6 files changed, 11 insertions(+), 5 deletions(-) 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 1e76086a873db..780656d3c77cf 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 @@ -80,6 +80,11 @@ public interface MembershipManager { */ ConsumerGroupHeartbeatResponseData.Assignment currentAssignment(); + /** + * @return Target assignment for the member. + */ + Optional targetAssignment(); + /** * Update the assignment for the member, indicating that the provided assignment is the new * current assignment. 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 6bb7c543bc3bf..665022a5c580a 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 @@ -318,7 +318,8 @@ public ConsumerGroupHeartbeatResponseData.Assignment currentAssignment() { * @return Assignment that the member received from the server but hasn't completely processed * yet. Visible for testing. */ - Optional targetAssignment() { + @Override + public Optional targetAssignment() { return targetAssignment; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index fc23a66efb4a7..70b72824d64e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -232,9 +232,9 @@ public PrototypeAsyncConsumer(final Time time, fetchMetricsManager, networkClientDelegateSupplier); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, - requestManagersSupplier, metadata, applicationEventQueue, + requestManagersSupplier, Optional.empty()); ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( subscriptions, 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 57626050a31e5..7732ac79f9331 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 @@ -197,9 +197,9 @@ private void process(final TopicMetadataApplicationEvent event) { * {@link ConsumerNetworkThread}. */ public static Supplier supplier(final LogContext logContext, - final Supplier requestManagersSupplier, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, + final Supplier requestManagersSupplier, final Optional membershipManager) { return new CachedSupplier() { @Override 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 a7e9bc4f27a40..0d25e53d02c87 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 @@ -90,7 +90,7 @@ public class ConsumerTestBuilder implements Closeable { final Optional coordinatorRequestManager; final Optional commitRequestManager; final Optional heartbeatRequestManager; - final Optional membershipManager; + final Optional membershipManager; final Optional heartbeatRequestState; final Optional assignmentReconciler; final TopicMetadataRequestManager topicMetadataRequestManager; 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 7874af6fe24e8..da9f2b5983a68 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 @@ -40,7 +40,7 @@ public class MembershipManagerImplTest { private static final int MEMBER_EPOCH = 1; private ConsumerTestBuilder testBuilder; - private MembershipManagerImpl membershipManager; + private MembershipManager membershipManager; @BeforeEach public void setup() { From 5f17c9d12d23f06a75b4ffa5bc67c4afafd2573e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 15:59:48 -0700 Subject: [PATCH 04/68] Restoring AbstractCoordinator.createMeter() to protected final --- .../consumer/internals/AbstractCoordinator.java | 2 +- .../consumer/internals/ConsumerCoordinatorMetrics.java | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index c6b655a21f360..24d05e7d63bfb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -1294,7 +1294,7 @@ boolean generationUnchanged() { } } - static Meter createMeter(Metrics metrics, String groupName, String baseName, String descriptiveName) { + protected final Meter createMeter(Metrics metrics, String groupName, String baseName, String descriptiveName) { return new Meter(new WindowedCount(), metrics.metricName(baseName + "-rate", groupName, String.format("The number of %s per second", descriptiveName)), diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java index 9e13d77d8e851..197bc2cf006f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java @@ -21,8 +21,8 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; - -import static org.apache.kafka.clients.consumer.internals.AbstractCoordinator.createMeter; +import org.apache.kafka.common.metrics.stats.Meter; +import org.apache.kafka.common.metrics.stats.WindowedCount; class ConsumerCoordinatorMetrics { @@ -44,7 +44,11 @@ class ConsumerCoordinatorMetrics { this.commitSensor.add(metrics.metricName("commit-latency-max", this.metricGrpName, "The max time taken for a commit request"), new Max()); - this.commitSensor.add(createMeter(metrics, metricGrpName, "commit", "commit calls")); + this.commitSensor.add(new Meter(new WindowedCount(), + metrics.metricName("commit-rate", metricGrpName, + "The number of commit calls per second"), + metrics.metricName("commit-total", metricGrpName, + "The total number of commit calls"))); this.revokeCallbackSensor = metrics.sensor("partition-revoked-latency"); this.revokeCallbackSensor.add(metrics.metricName("partition-revoked-latency-avg", From d08e18063f3383f7615e1fa24ebd07865fcf2ae9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 16:19:14 -0700 Subject: [PATCH 05/68] Update ApplicationEventProcessor.java --- .../events/ApplicationEventProcessor.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 7732ac79f9331..41666e46427bc 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,14 @@ private void process(final TopicMetadataApplicationEvent event) { event.chain(future); } + private void process(final RebalanceCompleteEvent event) { + membershipManager.ifPresent(mm -> mm.completeReconcile(event.revokedPartitions(), event.assignedPartitions(), event.error())); + } + + private void process(final PartitionLostCompleteEvent event) { + membershipManager.ifPresent(mm -> mm.completeLost(event.lostPartitions(), event.error())); + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. @@ -215,12 +223,4 @@ protected ApplicationEventProcessor create() { } }; } - - private void process(final RebalanceCompleteEvent event) { - membershipManager.ifPresent(mm -> mm.completeReconcile(event.revokedPartitions(), event.assignedPartitions(), event.error())); - } - - private void process(final PartitionLostCompleteEvent event) { - membershipManager.ifPresent(mm -> mm.completeLost(event.lostPartitions(), event.error())); - } } From 0bfd39506256d3e6ad1f5760c2df5a4727ce8e30 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 25 Oct 2023 17:49:29 -0700 Subject: [PATCH 06/68] Updates to fix threading issues with tests --- .../InternalApplicationEventHandler.java | 56 +++++++++++++ .../internals/PrototypeAsyncConsumer.java | 12 ++- .../consumer/internals/RequestManagers.java | 5 +- .../events/ApplicationEventHandler.java | 69 +++------------- .../events/BackgroundEventHandler.java | 27 ++----- .../internals/events/EventHandler.java | 74 ++++++++++++++++++ .../internals/AssignmentReconcilerTest.java | 78 +++++++------------ .../internals/ConsumerTestBuilder.java | 66 +++++++--------- .../events/BackgroundEventHandlerTest.java | 4 +- 9 files changed, 215 insertions(+), 176 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java new file mode 100644 index 0000000000000..97be3e806555c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.common.internals.IdempotentCloser; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; + +import java.time.Duration; +import java.util.concurrent.BlockingQueue; + +/** + * An event handler that receives {@link ApplicationEvent application events} from the application thread which + * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. + */ +class InternalApplicationEventHandler extends ApplicationEventHandler { + + private final Logger log; + private final ConsumerNetworkThread networkThread; + private final IdempotentCloser closer = new IdempotentCloser(); + + InternalApplicationEventHandler(final LogContext logContext, + final BlockingQueue applicationEventQueue, + final ConsumerNetworkThread networkThread) { + super(logContext, applicationEventQueue, networkThread::wakeup); + this.log = logContext.logger(InternalApplicationEventHandler.class); + this.networkThread = networkThread; + this.networkThread.start(); + } + + @Override + public void close(final Duration timeout) { + closer.close( + () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), + () -> log.warn("The application event handler was already closed") + ); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 70b72824d64e1..87187c9fa5c46 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -247,15 +247,18 @@ public PrototypeAsyncConsumer(final Time time, time, sensors ); - this.applicationEventHandler = new ApplicationEventHandler( + ConsumerNetworkThread networkThread = new ConsumerNetworkThread( logContext, time, - applicationEventQueue, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier ); - + this.applicationEventHandler = new InternalApplicationEventHandler( + logContext, + applicationEventQueue, + networkThread + ); this.backgroundEventProcessor = new BackgroundEventProcessor( logContext, backgroundEventQueue, @@ -298,6 +301,7 @@ public PrototypeAsyncConsumer(final Time time, } } + // visible for testing public PrototypeAsyncConsumer(LogContext logContext, String clientId, Deserializers deserializers, @@ -984,7 +988,7 @@ private Fetch collectFetch() { final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); // Notify the network thread to wake up and start the next round of fetching. - applicationEventHandler.wakeupNetworkThread(); + applicationEventHandler.notifyWatcher(); return fetch; } 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 8606b49729066..7d2041f6c6ea4 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 @@ -121,7 +121,10 @@ public static Supplier supplier(final Time time, @Override protected RequestManagers create() { final NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); + final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( + logContext, + backgroundEventQueue + ); 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); 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 2917d507d7f02..85d74b71a900b 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,73 +17,34 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -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; -import java.io.Closeable; -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; /** * An event handler that receives {@link ApplicationEvent application events} from the application thread which * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. */ -public class ApplicationEventHandler implements Closeable { - - private final Logger log; - private final BlockingQueue applicationEventQueue; - private final ConsumerNetworkThread networkThread; - private final IdempotentCloser closer = new IdempotentCloser(); +public class ApplicationEventHandler extends EventHandler { public ApplicationEventHandler(final LogContext logContext, - final Time time, - final BlockingQueue applicationEventQueue, - final Supplier applicationEventProcessorSupplier, - final Supplier networkClientDelegateSupplier, - final Supplier requestManagersSupplier) { - this.log = logContext.logger(ApplicationEventHandler.class); - this.applicationEventQueue = applicationEventQueue; - this.networkThread = new ConsumerNetworkThread(logContext, - time, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier); - this.networkThread.start(); + final BlockingQueue queue) { + this(logContext, queue, () -> { }); } - /** - * Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread} - * to alert the network I/O thread that it has something to process. - * - * @param event An {@link ApplicationEvent} created by the application thread - */ - public void add(final ApplicationEvent event) { - Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null"); - log.trace("Enqueued event: {}", event); - applicationEventQueue.add(event); - wakeupNetworkThread(); - } - - /** - * Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the next event(s) from the queue. - */ - public void wakeupNetworkThread() { - networkThread.wakeup(); + public ApplicationEventHandler(final LogContext logContext, + final BlockingQueue queue, + final Watcher watcher) { + super(logContext, queue, watcher); } /** - * 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. + * Add a {@link CompletableApplicationEvent} to the underlying queue. The method blocks waiting for the result, + * and will return the result value upon successful completion; otherwise throws an error. * *

* @@ -100,16 +61,4 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t add(event); return event.get(timer); } - - @Override - public void close() { - close(Duration.ZERO); - } - - public void close(final Duration timeout) { - closer.close( - () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), - () -> log.warn("The application event handler was already closed") - ); - } } 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 cafa426d033aa..083accaedeba7 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 @@ -18,10 +18,8 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.common.utils.LogContext; -import org.slf4j.Logger; -import java.util.Objects; -import java.util.Queue; +import java.util.concurrent.BlockingQueue; /** * An event handler that receives {@link BackgroundEvent background events} from the @@ -29,24 +27,15 @@ * via the {@link BackgroundEventProcessor}. */ -public class BackgroundEventHandler { +public class BackgroundEventHandler extends EventHandler { - private final Logger log; - private final Queue backgroundEventQueue; - - public BackgroundEventHandler(final LogContext logContext, final Queue backgroundEventQueue) { - this.log = logContext.logger(BackgroundEventHandler.class); - this.backgroundEventQueue = backgroundEventQueue; + public BackgroundEventHandler(final LogContext logContext, final BlockingQueue queue) { + this(logContext, queue, () -> {}); } - /** - * Add a {@link BackgroundEvent} to the handler. - * - * @param event A {@link BackgroundEvent} created by the {@link ConsumerNetworkThread network thread} - */ - public void add(BackgroundEvent event) { - Objects.requireNonNull(event, "BackgroundEvent provided to add must be non-null"); - log.trace("Enqueued event: {}", event); - backgroundEventQueue.add(event); + public BackgroundEventHandler(final LogContext logContext, + final BlockingQueue queue, + final Watcher watcher) { + super(logContext, queue, watcher); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java new file mode 100644 index 0000000000000..ad087957603f8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.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.common.utils.LogContext; +import org.slf4j.Logger; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.BlockingQueue; + +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; + +/** + * An event handler is used to publish events from one thread which are then consumed from another thread. + */ +public class EventHandler implements Closeable { + + private final Logger log; + private final BlockingQueue queue; + private final Watcher watcher; + + public EventHandler(final LogContext logContext, final BlockingQueue queue, Watcher watcher) { + this.log = logContext.logger(EventHandler.class); + this.queue = queue; + this.watcher = watcher; + } + + /** + * Add an {@link T} to the underlying queue and internally invoke {@link #notifyWatcher} + * to alert the watcher that it has something to process. + * + * @param event An event to enqueue for later processing + */ + public void add(final T event) { + Objects.requireNonNull(event, "Event must be non-null"); + log.trace("Enqueued event: {}", event); + queue.add(event); + watcher.updated(); + } + + public void notifyWatcher() { + watcher.updated(); + } + + @Override + public void close() { + close(Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS)); + } + + public void close(Duration timeout) { + // Do nothing. Available for subclasses. + } + + public interface Watcher { + + void updated(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java index 5610a93f0fc7f..ef96a9d3e324e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java @@ -17,48 +17,31 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -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.BackgroundEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.PartitionLostCompleteEvent; -import org.apache.kafka.clients.consumer.internals.events.PartitionLostStartedEvent; -import org.apache.kafka.clients.consumer.internals.events.RebalanceCompleteEvent; -import org.apache.kafka.clients.consumer.internals.events.RebalanceStartedEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Collection; import java.util.Collections; -import java.util.Optional; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; -import java.util.concurrent.BlockingQueue; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_TOPIC_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; public class AssignmentReconcilerTest { - private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; + private ConsumerTestBuilder testBuilder; private SubscriptionState subscriptions; - private BlockingQueue applicationEventQueue; - private BlockingQueue backgroundEventQueue; private ApplicationEventProcessor applicationEventProcessor; private BackgroundEventProcessor backgroundEventProcessor; private AssignmentReconciler reconciler; - @BeforeEach - public void setup() { - setup(new NoOpConsumerRebalanceListener()); - } - @AfterEach public void tearDown() { if (testBuilder != null) { @@ -66,8 +49,29 @@ public void tearDown() { } } + private void setup() { + setup(new NoOpConsumerRebalanceListener()); + } + + private void setup(ConsumerRebalanceListener listener) { + testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); + + // Create our subscriptions and subscribe to the topics. + subscriptions = testBuilder.subscriptions; + subscriptions.subscribe(Collections.singleton(DEFAULT_TOPIC_NAME), listener); + + // We need the background event processor to process events from the background thread (to execute + // the ConsumerRebalanceListener callbacks) and the application event processor to forward the result + // of the callback execution back to the membership manager. + applicationEventProcessor = testBuilder.applicationEventProcessor; + backgroundEventProcessor = testBuilder.backgroundEventProcessor; + reconciler = testBuilder.assignmentReconciler.orElseThrow(() -> new IllegalStateException("Should be in a group")); + } + @Test public void testAssignment() { + setup(); + // Create our initial assignment Set assignment = newTopicPartitions(0, 1, 2, 3); @@ -87,6 +91,8 @@ public void testAssignment() { @Test public void testAssignmentAndRevocation() { + setup(); + Set originalAssignment = newTopicPartitions(0, 1, 2, 3); // Create our initial assignment that adds four partitions @@ -106,7 +112,6 @@ public void testAssignmentAndRevocation() { // Create our follow-up assignment that removes two partitions. { Set newAssignment = newTopicPartitions(0, 2); - Set expectedRevoked = newTopicPartitions(1, 3); // We get another assignment. Since we have partitions assigned, we will need to revoke some // old partitions that are no longer part of the new target assignment. @@ -122,6 +127,8 @@ public void testAssignmentAndRevocation() { @Test public void testLose() { + setup(); + // This mimics having set up an assignment already. SortedSet partitions = newTopicPartitions(0, 1, 2, 3); subscriptions.assignFromSubscribed(partitions); @@ -149,8 +156,6 @@ public void onPartitionsAssigned(Collection partitions) { } }; - // Perform a tear down and set up our new listener. - tearDown(); setup(failingListener); // This mimics having set up an assignment of four partitions. @@ -182,35 +187,4 @@ private SortedSet newTopicPartitions(Integer... partitions) { return topicPartitions; } - - private void setup(ConsumerRebalanceListener listener) { - testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); - - // Create our subscriptions and subscribe to the topics. - subscriptions = testBuilder.subscriptions; - subscriptions.subscribe(Collections.singleton(DEFAULT_TOPIC_NAME), listener); - - // We need the background event queue to check for the events from the network thread to the application thread - // to signal the ConsumerRebalanceListener callbacks. -// applicationEventQueue = testBuilder.applicationEventQueue; -// backgroundEventQueue = testBuilder.backgroundEventQueue; - - applicationEventProcessor = testBuilder.applicationEventProcessor; - backgroundEventProcessor = testBuilder.backgroundEventProcessor; - reconciler = testBuilder.assignmentReconciler.orElseThrow(() -> new IllegalStateException("Should be in a group")); - } - -// @SuppressWarnings("unchecked") -// private T pollApplicationEvent() { -// ApplicationEvent event = applicationEventQueue.poll(); -// assertNotNull(event); -// return (T) event; -// } -// -// @SuppressWarnings("unchecked") -// private T pollBackgroundEvent() { -// BackgroundEvent event = backgroundEventQueue.poll(); -// assertNotNull(event); -// return (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 0d25e53d02c87..d780ea6497996 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 @@ -97,7 +97,10 @@ public class ConsumerTestBuilder implements Closeable { final FetchRequestManager fetchRequestManager; final RequestManagers requestManagers; public final ApplicationEventProcessor applicationEventProcessor; + public final BackgroundEventProcessor backgroundEventProcessor; + public ApplicationEventHandler applicationEventHandler; public final BackgroundEventHandler backgroundEventHandler; + public final ConsumerCoordinatorMetrics consumerCoordinatorMetrics; public final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; final MockClient client; final Optional groupInfo; @@ -110,6 +113,10 @@ public ConsumerTestBuilder(Optional groupInfo) { this.groupInfo = groupInfo; this.applicationEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventQueue = new LinkedBlockingQueue<>(); + this.applicationEventHandler = spy(new ApplicationEventHandler( + logContext, + applicationEventQueue + )); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( 100, @@ -260,7 +267,7 @@ public ConsumerTestBuilder(Optional groupInfo) { metadata, Optional.ofNullable(membershipManager.orElse(null))) ); - ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( + this.consumerCoordinatorMetrics = new ConsumerCoordinatorMetrics( subscriptions, metrics, CONSUMER_METRIC_GROUP_PREFIX @@ -269,7 +276,15 @@ public ConsumerTestBuilder(Optional groupInfo) { logContext, subscriptions, time, - sensors + consumerCoordinatorMetrics + ); + this.backgroundEventProcessor = spy( + new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ) ); } @@ -277,6 +292,7 @@ public ConsumerTestBuilder(Optional groupInfo) { public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); + closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); } public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { @@ -304,42 +320,7 @@ public void close() { } } - public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { - - public final ApplicationEventHandler applicationEventHandler; - public final BackgroundEventProcessor backgroundEventProcessor; - - public ApplicationEventHandlerTestBuilder() { - this(createDefaultGroupInformation()); - } - - public ApplicationEventHandlerTestBuilder(Optional groupInfo) { - super(groupInfo); - this.applicationEventHandler = spy(new ApplicationEventHandler( - logContext, - time, - applicationEventQueue, - () -> applicationEventProcessor, - () -> networkClientDelegate, - () -> requestManagers)); - this.backgroundEventProcessor = spy( - new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ) - ); - } - - @Override - public void close() { - closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); - closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); - } - } - - public static class PrototypeAsyncConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { + public static class PrototypeAsyncConsumerTestBuilder extends ConsumerTestBuilder { final PrototypeAsyncConsumer consumer; @@ -358,6 +339,15 @@ public PrototypeAsyncConsumerTestBuilder(Optional groupInfo) { deserializers, metricsManager, time); + ConsumerNetworkThread networkThread = new ConsumerNetworkThread(logContext, + time, + () -> applicationEventProcessor, + () -> networkClientDelegate, + () -> requestManagers); + this.applicationEventHandler = spy(new InternalApplicationEventHandler( + logContext, + applicationEventQueue, + networkThread)); this.consumer = spy(new PrototypeAsyncConsumer<>( logContext, clientId, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java index 6ef2c1f64a436..0670b8bdb7c3f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java @@ -32,14 +32,14 @@ public class BackgroundEventHandlerTest { - private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; + private ConsumerTestBuilder testBuilder; private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private BackgroundEventProcessor backgroundEventProcessor; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(); + testBuilder = new ConsumerTestBuilder(); backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; backgroundEventProcessor = testBuilder.backgroundEventProcessor; From ad19b36a41e413ba37e58f1d5ed6712bb5338e1a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 26 Oct 2023 07:23:15 -0700 Subject: [PATCH 07/68] Fixed minor whitespace issue --- .../consumer/internals/events/BackgroundEventHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 083accaedeba7..107463d4014b4 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 @@ -30,7 +30,7 @@ public class BackgroundEventHandler extends EventHandler { public BackgroundEventHandler(final LogContext logContext, final BlockingQueue queue) { - this(logContext, queue, () -> {}); + this(logContext, queue, () -> { }); } public BackgroundEventHandler(final LogContext logContext, From a210acf31dd72970fb9d9980ddf2d1e5bd09541b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Nov 2023 16:09:13 -0800 Subject: [PATCH 08/68] Reverted code related to the core reconciliation --- .../internals/AssignmentReconciler.java | 245 ------------------ .../internals/HeartbeatRequestManager.java | 11 +- .../consumer/internals/MembershipManager.java | 63 ----- .../internals/MembershipManagerImpl.java | 104 ++------ .../internals/PrototypeAsyncConsumer.java | 3 +- .../consumer/internals/RequestManagers.java | 14 +- .../events/ApplicationEventProcessor.java | 18 +- .../internals/AssignmentReconcilerTest.java | 191 -------------- .../internals/ConsumerTestBuilder.java | 23 +- .../internals/MembershipManagerImplTest.java | 191 +++++++------- 10 files changed, 123 insertions(+), 740 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java deleted file mode 100644 index c39e33f09d53d..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignmentReconciler.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.PartitionLostStartedEvent; -import org.apache.kafka.clients.consumer.internals.events.RebalanceCompleteEvent; -import org.apache.kafka.clients.consumer.internals.events.RebalanceStartedEvent; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; -import org.apache.kafka.common.utils.LogContext; -import org.slf4j.Logger; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Optional; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.concurrent.BlockingQueue; - -/** - * {@code AssignmentReconciler} performs the work of reconciling this consumer's partition assignment as directed - * by the consumer group coordinator. When the coordinator determines that a change to the partition ownership of - * the group is required, it will communicate with each consumer to relay its respective target - * assignment, that is, the set of partitions for which that consumer should now assume ownership. It is then the - * responsibility of the consumer to work toward that target by performing the necessary internal modifications to - * satisfy the assignment from the coordinator. In practical terms, this means that it must first determine the set - * difference between the {@link SubscriptionState#assignedPartitions() current assignment} and the - * target assignment. - * - *

- * - * Internally, reconciliation requires the following steps: - * - *

    - *
  1. - * On the background thread, upon receipt of a new assignment from the group coordinator, the - * {@link MembershipManager} should call {@link #startReconcile(Set)} to start reconciliation. - *
  2. - *
  3. - * Internally, the partitions to revoke are determined via {@link #getPartitionsToRevoke(Set)}; - * these are the partitions in the current assignment that are not in the target assignment. - * Next, we calculate the partitions to assign using {@link #getPartitionsToAssign(Set)}; - * these are the partitions in the target assignment that are not in the current assignment. - *
  4. - *
  5. - * Send a {@link RebalanceStartedEvent} so that the application thread will execute the - * {@link ConsumerRebalanceListener} callback methods. - *
  6. - *
  7. - * On the application thread, when the {@link RebalanceStartedEvent} is received, execute the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and - * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callback methods - *
  8. - *
  9. - * Enqueue a corresponding {@link RebalanceCompleteEvent} so that the background thread - * will know the listener was invoked and the result of the invocation - *
  10. - *
  11. - * On the background thread, process the {@link RebalanceCompleteEvent}, which should call the - * {@link MembershipManager#completeReconcile(Set, Set, Optional)} method. This method will call - * {@link #completeReconcile(Set, Set)} to remove the revoked partitions and add the assigned partitions in the - * {@link SubscriptionState#assignFromSubscribed(Collection) current assignment} and then make a note to - * send a {@link ConsumerGroupHeartbeatRequest} to the group coordinator on its next pass of - * {@link HeartbeatRequestManager#poll(long)} - *
  12. - *
- */ -public class AssignmentReconciler { - - private final Logger log; - private final SubscriptionState subscriptions; - private final BlockingQueue backgroundEventQueue; - - AssignmentReconciler(LogContext logContext, - SubscriptionState subscriptions, - BlockingQueue backgroundEventQueue) { - this.log = logContext.logger(getClass()); - this.subscriptions = subscriptions; - this.backgroundEventQueue = backgroundEventQueue; - } - - /** - * Performs the step of dropping any assigned {@link TopicPartition partitions} as this consumer is no longer - * to be considered a valid member of the group. - */ - void startLost() { - SortedSet partitionsToLose = getPartitionsToLose(); - - if (partitionsToLose.isEmpty()) { - log.debug("Skipping invocation of {} callbacks as no partitions changed in the new assignment", - ConsumerRebalanceListener.class.getSimpleName()); - return; - } - - markPartitionsPendingRevocation(); - - // TODO: determine if there's a "valid" ConsumerRebalanceListener in use. If not, we can return false - // immediately as there's no need to perform the rebalance callback invocation. - - log.debug("Enqueuing event to invoke {} callbacks", ConsumerRebalanceListener.class.getSimpleName()); - backgroundEventQueue.add(new PartitionLostStartedEvent(partitionsToLose)); - } - - /** - * This method should be invoked to signal the completion of the "{@link TopicPartition lost partition}" - * process. Specifically, it is to be executed on background thread after the - * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback was executed on the application - * thread. It should clear the set of {@link SubscriptionState#assignedPartitions() assigned partitions}, - * regardless of the set of "lost partitions." - * - * @param lostPartitions Set of {@link TopicPartition partitions} that were lost - * @see AssignmentReconciler - */ - void completeLost(Set lostPartitions) { - log.debug("{} callbacks were successfully invoked", ConsumerRebalanceListener.class.getSimpleName()); - subscriptions.assignFromSubscribed(Collections.emptySet()); - } - - /** - * Performs the assignment phase of the reconciliation process as described in the top-level class documentation. - * - * @param target Set of {@link TopicPartition} which represents the target set of topics - */ - void startReconcile(Set target) { - SortedSet partitionsToRevoke = getPartitionsToRevoke(target); - SortedSet partitionsToAssign = getPartitionsToAssign(target); - - if (partitionsToRevoke.isEmpty() && partitionsToAssign.isEmpty()) { - log.debug("Skipping invocation of {} callbacks as no partitions changed in the new assignment", - ConsumerRebalanceListener.class.getSimpleName()); - return; - } - - markPartitionsPendingRevocation(); - - // TODO: determine if there's a "valid" ConsumerRebalanceListener in use. If not, we can return false - // immediately as there's no need to perform the rebalance callback invocation. - - log.debug("Enqueuing event to invoke {} callbacks", ConsumerRebalanceListener.class.getSimpleName()); - RebalanceStartedEvent event = new RebalanceStartedEvent( - partitionsToRevoke, - partitionsToAssign - ); - backgroundEventQueue.add(event); - } - - /** - * This method should be invoked to signal the completion of a successful {@link TopicPartition partition} - * assignment reconciliation. Specifically, it is to be executed on background thread after the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and - * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks have completed execution on - * the application thread. It should update the set of - * {@link SubscriptionState#assignedPartitions() assigned partitions} based on the - * given partitions. - * - * @param revokedPartitions Set of {@link TopicPartition partitions} that were revoked - * @param assignedPartitions Set of {@link TopicPartition partitions} that were assigned - * @see MembershipManager#completeReconcile(Set, Set, Optional) - */ - void completeReconcile(Set revokedPartitions, Set assignedPartitions) { - log.debug("{} callbacks were successfully invoked", ConsumerRebalanceListener.class.getSimpleName()); - - Set newAssignment = new HashSet<>(subscriptions.assignedPartitions()); - newAssignment.addAll(assignedPartitions); - newAssignment.removeAll(revokedPartitions); - subscriptions.assignFromSubscribed(newAssignment); - } - - /** - * Determine which partitions are newly revoked. This is done by comparing the target set from the assignment - * against the {@link SubscriptionState#assignedPartitions() current set}. The returned set of - * {@link TopicPartition partitions} are composed of any partitions that are in the current set but - * are no longer in the target set. - * - * @param target Set of {@link TopicPartition} which represents the target set of topics - * @return Set of partitions to revoke - */ - SortedSet getPartitionsToRevoke(Set target) { - SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); - partitions.addAll(subscriptions.assignedPartitions()); - partitions.removeAll(target); - return partitions; - } - - /** - * Determine which partitions are newly assigned. This is done by comparing the target set from the assignment - * against the {@link SubscriptionState#assignedPartitions() current set}. Any {@link TopicPartition partitions} - * from the target set that are not already in the current set are included in the returned set. - * - * @param target Set of {@link TopicPartition} which represents the target set of topics - * @return Set of partitions to assign - */ - SortedSet getPartitionsToAssign(Set target) { - SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); - partitions.addAll(target); - partitions.removeAll(subscriptions.assignedPartitions()); - return partitions; - } - - /** - * Determine which partitions should be "lost". This is simply the - * {@link SubscriptionState#assignedPartitions() current set} of {@link TopicPartition partitions}. - * - * @return Set of partitions to "lose" - */ - SortedSet getPartitionsToLose() { - SortedSet partitions = new TreeSet<>(new TopicPartitionComparator()); - partitions.addAll(subscriptions.assignedPartitions()); - return partitions; - } - - /** - * When asynchronously committing offsets prior to the revocation of a set of partitions, there will be a - * window of time between when the offset commit is sent and when it returns and revocation completes. It is - * possible for pending fetches for these partitions to return during this time, which means the application's - * position may get ahead of the committed position prior to revocation. This can cause duplicate consumption. - * To prevent this, we mark the partitions as "pending revocation," which stops the Fetcher from sending new - * fetches or returning data from previous fetches to the user. - */ - private void markPartitionsPendingRevocation() { - Set partitions = subscriptions.assignedPartitions(); - log.debug("Marking assigned partitions pending for revocation: {}", partitions); - subscriptions.markPendingRevocation(partitions); - } -} 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 cba27af5ada37..a0a4ca97e16ca 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 @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; -import org.apache.kafka.common.errors.UnknownTopicIdException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; @@ -212,15 +211,7 @@ private void onResponse(final ConsumerGroupHeartbeatResponse response, long curr this.heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); this.heartbeatRequestState.resetTimer(); - - try { - this.membershipManager.updateState(response.data()); - } catch (UnknownTopicIdException e) { - // This can occur when the assignment has a topic ID that we don't know about yet. - // - // TODO: I have no idea what to do here - // metadata.requestUpdate(); - } + this.membershipManager.updateState(response.data()); return; } onErrorResponse(response, currentTimeMs); 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 780656d3c77cf..8a95a80c65991 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 @@ -16,14 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; -import java.util.Collection; import java.util.Optional; -import java.util.Set; /** * A stateful object tracking the state of a single member in relationship to a consumer group: @@ -80,11 +75,6 @@ public interface MembershipManager { */ ConsumerGroupHeartbeatResponseData.Assignment currentAssignment(); - /** - * @return Target assignment for the member. - */ - Optional targetAssignment(); - /** * Update the assignment for the member, indicating that the provided assignment is the new * current assignment. @@ -109,57 +99,4 @@ public interface MembershipManager { * @return True if the member should send heartbeat to the coordinator. */ boolean shouldSendHeartbeat(); - - /** - * This method should be invoked to signal the completion of a successful {@link TopicPartition partition} - * assignment reconciliation. Specifically, it is to be executed on background thread after the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and - * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks have completed execution on - * the application thread. It should perform two tasks: - * - *
    - *
  1. - * Update the set of {@link SubscriptionState#assignedPartitions() assigned partitions} based on the - * given partitions - *
  2. - *
  3. - * Update the necessary internal state to signal to the {@link HeartbeatRequestManager} that it - * should send an acknowledgement heartbeat request to the group coordinator - *
  4. - *
- * - * Note: the partition assignment reconciliation process is started based on the receipt of a new - * {@link ConsumerGroupHeartbeatResponseData.Assignment target assignment}. - * - * @param revokedPartitions Set of {@link TopicPartition partitions} that were revoked - * @param assignedPartitions Set of {@link TopicPartition partitions} that were assigned - * @param callbackError Optional {@link KafkaException error} if an exception was thrown during callbacks - * @see AssignmentReconciler - */ - void completeReconcile(Set revokedPartitions, - Set assignedPartitions, - Optional callbackError); - - /** - * This method should be invoked to signal the completion of the "{@link TopicPartition lost partition}" - * process. Specifically, it is to be executed on background thread after the - * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback was executed on the application - * thread. It should perform two tasks: - * - *
    - *
  1. - * Clear the set of {@link SubscriptionState#assignedPartitions() assigned partitions}, regardless of - * the set of "lost partitions" - *
  2. - *
  3. - * Update the necessary internal state to signal to the {@link HeartbeatRequestManager} that it - * should send an acknowledgement heartbeat request to the group coordinator - *
  4. - *
- * - * @param lostPartitions Set of {@link TopicPartition partitions} that were lost - * @param callbackError Optional {@link KafkaException error} if an exception was thrown during callback - * @see AssignmentReconciler - */ - void completeLost(Set lostPartitions, Optional callbackError); } 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 665022a5c580a..2a9a5d2992daa 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 @@ -17,11 +17,6 @@ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.errors.UnknownTopicIdException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; @@ -29,9 +24,6 @@ import org.slf4j.Logger; import java.util.Optional; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; /** * Membership manager that maintains group membership for a single member, following the new @@ -45,21 +37,6 @@ */ public class MembershipManagerImpl implements MembershipManager { - /** - * Logger. - */ - private final Logger log; - - /** - * For reconciling assignments. - */ - private final AssignmentReconciler assignmentReconciler; - - /** - * Metadata that allows us to create the partitions needed for {@link ConsumerRebalanceListener}. - */ - private final ConsumerMetadata metadata; - /** * Group ID of the consumer group the member will be part of, provided when creating the current * membership manager. @@ -109,26 +86,24 @@ public class MembershipManagerImpl implements MembershipManager { private Optional targetAssignment; /** - * Latest assignment that the member received from the server while a {@link #targetAssignment} - * was in process. + * Logger. */ - private Optional nextTargetAssignment; - - public MembershipManagerImpl(LogContext logContext, - AssignmentReconciler assignmentReconciler, - ConsumerMetadata metadata, - String groupId, - Optional groupInstanceId, - Optional serverAssignor) { - this.log = logContext.logger(MembershipManagerImpl.class); - this.assignmentReconciler = assignmentReconciler; - this.metadata = metadata; + private final Logger log; + + public MembershipManagerImpl(String groupId, LogContext logContext) { + this(groupId, null, null, logContext); + } + + public MembershipManagerImpl(String groupId, + String groupInstanceId, + String serverAssignor, + LogContext logContext) { this.groupId = groupId; this.state = MemberState.UNJOINED; - this.serverAssignor = serverAssignor; - this.groupInstanceId = groupInstanceId; + this.serverAssignor = Optional.ofNullable(serverAssignor); + this.groupInstanceId = Optional.ofNullable(groupInstanceId); this.targetAssignment = Optional.empty(); - this.nextTargetAssignment = Optional.empty(); + this.log = logContext.logger(MembershipManagerImpl.class); } /** @@ -206,7 +181,6 @@ public void updateState(ConsumerGroupHeartbeatResponseData response) { public void transitionToFenced() { resetEpoch(); transitionTo(MemberState.FENCED); - assignmentReconciler.startLost(); } /** @@ -232,33 +206,10 @@ private boolean maybeTransitionToStable() { transitionTo(MemberState.STABLE); } else { transitionTo(MemberState.RECONCILING); - startReconciliation(); } return state.equals(MemberState.STABLE); } - private void startReconciliation() { - if (!targetAssignment.isPresent()) - return; - - SortedSet targetPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); - - for (ConsumerGroupHeartbeatResponseData.TopicPartitions topicPartitions : targetAssignment.get().topicPartitions()) { - Uuid topicId = topicPartitions.topicId(); - String topicName = metadata.topicNames().get(topicId); - - // TODO... I don't think this is right... - if (topicName == null) - throw new UnknownTopicIdException("A topic name for the topic ID " + topicId + " was not found in the local metadata cache"); - - for (Integer partition : topicPartitions.partitions()) { - targetPartitions.add(new TopicPartition(topicName, partition)); - } - } - - assignmentReconciler.startReconcile(targetPartitions); - } - /** * Take new target assignment received from the server and set it as targetAssignment to be * processed. Following the consumer group protocol, the server won't send a new target @@ -318,8 +269,7 @@ public ConsumerGroupHeartbeatResponseData.Assignment currentAssignment() { * @return Assignment that the member received from the server but hasn't completely processed * yet. Visible for testing. */ - @Override - public Optional targetAssignment() { + Optional targetAssignment() { return targetAssignment; } @@ -348,28 +298,4 @@ public void onTargetAssignmentProcessComplete(ConsumerGroupHeartbeatResponseData targetAssignment = Optional.empty(); transitionTo(MemberState.STABLE); } - - @Override - public void completeReconcile(Set revokedPartitions, - Set assignedPartitions, - Optional callbackError) { - if (callbackError.isPresent()) { - // TODO: how to react to callback errors? - } - - assignmentReconciler.completeReconcile(revokedPartitions, assignedPartitions); - transitionTo(MemberState.STABLE); - // TODO: update state to signal the HeartbeatRequestManager to send an ACK heartbeat - } - - @Override - public void completeLost(Set lostPartitions, Optional callbackError) { - if (callbackError.isPresent()) { - // TODO: how to react to callback errors? - } - - assignmentReconciler.completeLost(lostPartitions); - transitionTo(MemberState.UNJOINED); - // TODO: update state to signal the HeartbeatRequestManager to send an ACK heartbeat - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 8af2e36d828ee..893fca492f453 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -234,8 +234,7 @@ public PrototypeAsyncConsumer(final Time time, final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, applicationEventQueue, - requestManagersSupplier, - Optional.empty()); + requestManagersSupplier); ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( subscriptions, metrics, 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 0d4334a3facbd..bfca04c819275 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 @@ -164,19 +164,7 @@ protected RequestManagers create() { backgroundEventHandler, groupState.groupId); commit = new CommitRequestManager(time, logContext, subscriptions, config, coordinator, groupState); - AssignmentReconciler assignmentReconciler = new AssignmentReconciler( - logContext, - subscriptions, - backgroundEventQueue - ); - MembershipManager membershipManager = new MembershipManagerImpl( - logContext, - assignmentReconciler, - metadata, - groupState.groupId, - Optional.empty(), - Optional.empty() - ); + MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext); heartbeatRequestManager = new HeartbeatRequestManager( 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 41666e46427bc..bfc42ebeac101 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -20,7 +20,6 @@ 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.MembershipManager; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; @@ -45,18 +44,15 @@ public class ApplicationEventProcessor extends EventProcessor private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; - private final Optional membershipManager; public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, - final ConsumerMetadata metadata, - final Optional membershipManager) { + final ConsumerMetadata metadata) { super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; - this.membershipManager = membershipManager; } /** @@ -193,11 +189,13 @@ private void process(final TopicMetadataApplicationEvent event) { } private void process(final RebalanceCompleteEvent event) { - membershipManager.ifPresent(mm -> mm.completeReconcile(event.revokedPartitions(), event.assignedPartitions(), event.error())); + // TODO: with this event, we need to signal to the consumer group protocol state machine that the + // rebalance it'd started has completed... } private void process(final PartitionLostCompleteEvent event) { - membershipManager.ifPresent(mm -> mm.completeLost(event.lostPartitions(), event.error())); + // TODO: with this event, we need to signal to the consumer group protocol state machine that the + // partitions have been "lost"... } /** @@ -207,8 +205,7 @@ private void process(final PartitionLostCompleteEvent event) { public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, - final Supplier requestManagersSupplier, - final Optional membershipManager) { + final Supplier requestManagersSupplier) { return new CachedSupplier() { @Override protected ApplicationEventProcessor create() { @@ -217,8 +214,7 @@ protected ApplicationEventProcessor create() { logContext, applicationEventQueue, requestManagers, - metadata, - membershipManager + metadata ); } }; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java deleted file mode 100644 index 21b72e6d9c506..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AssignmentReconcilerTest.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; - -import java.util.Collection; -import java.util.Collections; -import java.util.Optional; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_TOPIC_NAME; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class AssignmentReconcilerTest { - - private ConsumerTestBuilder testBuilder; - private SubscriptionState subscriptions; - private ApplicationEventProcessor applicationEventProcessor; - private BackgroundEventProcessor backgroundEventProcessor; - private AssignmentReconciler reconciler; - - @AfterEach - public void tearDown() { - if (testBuilder != null) { - testBuilder.close(); - } - } - - private void setup() { - setup(Optional.empty()); - } - - private void setup(Optional listener) { - testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); - - // Create our subscriptions and subscribe to the topics. - subscriptions = testBuilder.subscriptions; - subscriptions.subscribe(Collections.singleton(DEFAULT_TOPIC_NAME), listener); - - // We need the background event processor to process events from the background thread (to execute - // the ConsumerRebalanceListener callbacks) and the application event processor to forward the result - // of the callback execution back to the membership manager. - applicationEventProcessor = testBuilder.applicationEventProcessor; - backgroundEventProcessor = testBuilder.backgroundEventProcessor; - reconciler = testBuilder.assignmentReconciler.orElseThrow(() -> new IllegalStateException("Should be in a group")); - } - - @Test - public void testAssignment() { - setup(); - - // Create our initial assignment - Set assignment = newTopicPartitions(0, 1, 2, 3); - - // Start the reconciliation process. At this point, since there are no partitions assigned to our - // subscriptions, we don't need to revoke anything. Validate that after our initial step that we haven't - // prematurely assigned anything to the subscriptions. - reconciler.startReconcile(assignment); - assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - - // Complete the future to signal to the reconciler that the ConsumerRebalanceListener callback - // has completed. This will trigger the "commit" of the partition assignment to the subscriptions. - assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - backgroundEventProcessor.process(); - applicationEventProcessor.process(); - assertEquals(assignment, subscriptions.assignedPartitions()); - } - - @Test - public void testAssignmentAndRevocation() { - setup(); - - Set originalAssignment = newTopicPartitions(0, 1, 2, 3); - - // Create our initial assignment that adds four partitions - { - // Start the reconciliation process. At this point, since there are no partitions assigned to our - // subscriptions, we don't need to revoke anything. Validate that after our initial step that we haven't - // prematurely assigned anything to the subscriptions. - reconciler.startReconcile(originalAssignment); - assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - - // Now process the callback. - backgroundEventProcessor.process(); - applicationEventProcessor.process(); - assertEquals(originalAssignment, subscriptions.assignedPartitions()); - } - - // Create our follow-up assignment that removes two partitions. - { - Set newAssignment = newTopicPartitions(0, 2); - - // We get another assignment. Since we have partitions assigned, we will need to revoke some - // old partitions that are no longer part of the new target assignment. - reconciler.startReconcile(newAssignment); - assertEquals(originalAssignment, subscriptions.assignedPartitions()); - - // Now process the callback. - backgroundEventProcessor.process(); - applicationEventProcessor.process(); - assertEquals(newAssignment, subscriptions.assignedPartitions()); - } - } - - @Test - public void testLose() { - setup(); - - // This mimics having set up an assignment already. - SortedSet partitions = newTopicPartitions(0, 1, 2, 3); - subscriptions.assignFromSubscribed(partitions); - - assertEquals(partitions, subscriptions.assignedPartitions()); - reconciler.startLost(); - assertEquals(partitions, subscriptions.assignedPartitions()); - - backgroundEventProcessor.process(); - applicationEventProcessor.process(); - assertEquals(Collections.emptySet(), subscriptions.assignedPartitions()); - } - - @Test - public void testRevocationFailure() { - ConsumerRebalanceListener failingListener = new ConsumerRebalanceListener() { - @Override - public void onPartitionsRevoked(Collection partitions) { - throw new KafkaException("Simulating callback failure"); - } - - @Override - public void onPartitionsAssigned(Collection partitions) { - - } - }; - - setup(Optional.of(failingListener)); - - // This mimics having set up an assignment of four partitions. - SortedSet partitions = newTopicPartitions(0, 1, 2, 3); - subscriptions.assignFromSubscribed(partitions); - - // When revoking partitions, we get an error. This should not stop the reconciliation process, though. - { - Set assignment = newTopicPartitions(0, 2); - - // Start the reconciliation process. - reconciler.startReconcile(assignment); - - // Now process the callback. It should throw an exception, but it should still finish and allow - // the reconciler to alter the assigned partition set. - assertThrows(KafkaException.class, () -> backgroundEventProcessor.process()); - applicationEventProcessor.process(); - assertEquals(newTopicPartitions(0, 2), subscriptions.assignedPartitions()); - } - } - - private SortedSet newTopicPartitions(Integer... partitions) { - SortedSet topicPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); - - if (partitions != null) { - for (int partition : partitions) - topicPartitions.add(new TopicPartition(DEFAULT_TOPIC_NAME, partition)); - } - - return topicPartitions; - } -} 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 ca35bc060a916..976b28e85b30b 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 @@ -69,7 +69,6 @@ public class ConsumerTestBuilder implements Closeable { static final Uuid DEFAULT_TOPIC_ID = Uuid.randomUuid(); static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id"; static final String DEFAULT_GROUP_ID = "group-id"; - static final String DEFAULT_MEMBER_ID = "test-member-1"; static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; @@ -92,7 +91,6 @@ public class ConsumerTestBuilder implements Closeable { final Optional heartbeatRequestManager; final Optional membershipManager; final Optional heartbeatRequestState; - final Optional assignmentReconciler; final TopicMetadataRequestManager topicMetadataRequestManager; final FetchRequestManager fetchRequestManager; final RequestManagers requestManagers; @@ -100,7 +98,6 @@ public class ConsumerTestBuilder implements Closeable { public final BackgroundEventProcessor backgroundEventProcessor; public ApplicationEventHandler applicationEventHandler; public final BackgroundEventHandler backgroundEventHandler; - public final ConsumerCoordinatorMetrics consumerCoordinatorMetrics; public final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; final MockClient client; final Optional groupInfo; @@ -197,19 +194,12 @@ public ConsumerTestBuilder(Optional groupInfo) { config, coordinator, groupState)); - AssignmentReconciler assignmentReconciler = new AssignmentReconciler( - logContext, - subscriptions, - backgroundEventQueue - ); MembershipManagerImpl mm = spy( new MembershipManagerImpl( - logContext, - assignmentReconciler, - metadata, gi.groupState.groupId, - gi.groupState.groupInstanceId, - Optional.empty() + gi.groupState.groupInstanceId.orElse(null), + null, + logContext ) ); HeartbeatRequestManager.HeartbeatRequestState state = spy(new HeartbeatRequestManager.HeartbeatRequestState(logContext, @@ -233,14 +223,12 @@ public ConsumerTestBuilder(Optional groupInfo) { this.heartbeatRequestManager = Optional.of(heartbeat); this.heartbeatRequestState = Optional.of(state); this.membershipManager = Optional.of(mm); - this.assignmentReconciler = Optional.of(assignmentReconciler); } else { this.coordinatorRequestManager = Optional.empty(); this.commitRequestManager = Optional.empty(); this.heartbeatRequestManager = Optional.empty(); this.heartbeatRequestState = Optional.empty(); this.membershipManager = Optional.empty(); - this.assignmentReconciler = Optional.empty(); } this.fetchBuffer = new FetchBuffer(logContext); @@ -266,10 +254,9 @@ public ConsumerTestBuilder(Optional groupInfo) { logContext, applicationEventQueue, requestManagers, - metadata, - Optional.ofNullable(membershipManager.orElse(null))) + metadata) ); - this.consumerCoordinatorMetrics = new ConsumerCoordinatorMetrics( + ConsumerCoordinatorMetrics consumerCoordinatorMetrics = new ConsumerCoordinatorMetrics( subscriptions, metrics, CONSUMER_METRIC_GROUP_PREFIX 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 da9f2b5983a68..8089cfd8d0bc9 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 @@ -17,20 +17,17 @@ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; +import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; import java.util.Optional; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_MEMBER_ID; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_TOPIC_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -38,57 +35,50 @@ public class MembershipManagerImplTest { + private static final String GROUP_ID = "test-group"; + private static final String MEMBER_ID = "test-member-1"; private static final int MEMBER_EPOCH = 1; - private ConsumerTestBuilder testBuilder; - private MembershipManager membershipManager; - - @BeforeEach - public void setup() { - testBuilder = new ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation()); - membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); - } - - @AfterEach - public void tearDown() { - if (testBuilder != null) { - testBuilder.close(); - } - } + private final LogContext logContext = new LogContext(); @Test public void testMembershipManagerServerAssignor() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(Optional.empty(), membershipManager.serverAssignor()); - membershipManager = new MembershipManagerImpl( - testBuilder.logContext, - testBuilder.assignmentReconciler.orElseThrow(IllegalStateException::new), - testBuilder.metadata, - DEFAULT_GROUP_ID, - Optional.of("instance1"), - Optional.of("Uniform") - ); + membershipManager = new MembershipManagerImpl(GROUP_ID, "instance1", "Uniform", logContext); assertEquals(Optional.of("Uniform"), membershipManager.serverAssignor()); } + @Test + public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { + new MembershipManagerImpl(GROUP_ID, logContext); + new MembershipManagerImpl(GROUP_ID, null, null, logContext); + } + @Test public void testTransitionToReconcilingOnlyIfAssignmentReceived() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); - ConsumerGroupHeartbeatResponse responseWithoutAssignment = createHeartbeatResponse(); + ConsumerGroupHeartbeatResponse responseWithoutAssignment = + createConsumerGroupHeartbeatResponse(null); membershipManager.updateState(responseWithoutAssignment.data()); assertNotEquals(MemberState.RECONCILING, membershipManager.state()); - ConsumerGroupHeartbeatResponse responseWithAssignment = createHeartbeatResponse(createAssignment()); + ConsumerGroupHeartbeatResponse responseWithAssignment = + createConsumerGroupHeartbeatResponse(createAssignment()); membershipManager.updateState(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @Test public void testMemberIdAndEpochResetOnFencedMembers() { - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponse heartbeatResponse = + createConsumerGroupHeartbeatResponse(null); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(DEFAULT_MEMBER_ID, membershipManager.memberId()); + assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); membershipManager.transitionToFenced(); @@ -98,10 +88,12 @@ public void testMemberIdAndEpochResetOnFencedMembers() { @Test public void testTransitionToFailure() { - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponse heartbeatResponse = + createConsumerGroupHeartbeatResponse(null); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(DEFAULT_MEMBER_ID, membershipManager.memberId()); + assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); membershipManager.transitionToFailed(); @@ -110,40 +102,45 @@ public void testTransitionToFailure() { @Test public void testFencingWhenStateIsStable() { - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - testStateUpdateOnFenceError(); + testStateUpdateOnFenceError(membershipManager); } @Test public void testFencingWhenStateIsReconciling() { - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(createAssignment()); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(createAssignment()); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - testStateUpdateOnFenceError(); + testStateUpdateOnFenceError(membershipManager); } @Test public void testFatalFailureWhenStateIsUnjoined() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); - testStateUpdateOnFatalFailure(); + testStateUpdateOnFatalFailure(membershipManager); } @Test public void testFatalFailureWhenStateIsStable() { - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); membershipManager.updateState(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); - testStateUpdateOnFatalFailure(); + testStateUpdateOnFatalFailure(membershipManager); } @Test public void testFencingShouldNotHappenWhenStateIsUnjoined() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); assertEquals(MemberState.UNJOINED, membershipManager.state()); // Getting fenced when the member is not part of the group is not expected and should @@ -153,62 +150,80 @@ public void testFencingShouldNotHappenWhenStateIsUnjoined() { @Test public void testUpdateStateFailsOnResponsesWithErrors() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); // Updating state with a heartbeat response containing errors cannot be performed and // should fail. - ConsumerGroupHeartbeatResponse unknownMemberResponse = createHeartbeatResponse(Errors.UNKNOWN_MEMBER_ID); - assertThrows(IllegalArgumentException.class, () -> membershipManager.updateState(unknownMemberResponse.data())); + ConsumerGroupHeartbeatResponse unknownMemberResponse = + createConsumerGroupHeartbeatResponseWithError(Errors.UNKNOWN_MEMBER_ID); + assertThrows(IllegalArgumentException.class, + () -> membershipManager.updateState(unknownMemberResponse.data())); } @Test public void testAssignmentUpdatedAsReceivedAndProcessed() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); ConsumerGroupHeartbeatResponseData.Assignment newAssignment = createAssignment(); - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(newAssignment); + ConsumerGroupHeartbeatResponse heartbeatResponse = + createConsumerGroupHeartbeatResponse(newAssignment); membershipManager.updateState(heartbeatResponse.data()); // Target assignment should be in the process of being reconciled - checkAssignments(null, newAssignment); + checkAssignments(membershipManager, null, newAssignment); // Mark assignment processing completed membershipManager.onTargetAssignmentProcessComplete(newAssignment); // Target assignment should now be the current assignment - checkAssignments(newAssignment, null); + checkAssignments(membershipManager, newAssignment, null); } @Test public void testMemberFailsIfAssignmentReceivedWhileAnotherOnBeingReconciled() { + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); ConsumerGroupHeartbeatResponseData.Assignment newAssignment1 = createAssignment(); - ConsumerGroupHeartbeatResponseData response1 = createHeartbeatResponse(newAssignment1).data(); - membershipManager.updateState(response1); + membershipManager.updateState(createConsumerGroupHeartbeatResponse(newAssignment1).data()); // First target assignment received should be in the process of being reconciled - checkAssignments(null, newAssignment1); + checkAssignments(membershipManager, null, newAssignment1); // Second target assignment received while there is another one being reconciled ConsumerGroupHeartbeatResponseData.Assignment newAssignment2 = createAssignment(); - ConsumerGroupHeartbeatResponseData response2 = createHeartbeatResponse(newAssignment2).data(); - assertThrows(IllegalStateException.class, () -> membershipManager.updateState(response2)); + assertThrows(IllegalStateException.class, + () -> membershipManager.updateState(createConsumerGroupHeartbeatResponse(newAssignment2).data())); assertEquals(MemberState.FAILED, membershipManager.state()); } @Test public void testAssignmentUpdatedFailsIfAssignmentReconciledDoesNotMatchTargetAssignment() { - ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = createAssignment(0, 1, 2); - ConsumerGroupHeartbeatResponse heartbeatResponse = createHeartbeatResponse(targetAssignment); + MembershipManagerImpl membershipManager = new MembershipManagerImpl(GROUP_ID, logContext); + ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(0, 1, 2)))); + ConsumerGroupHeartbeatResponse heartbeatResponse = + createConsumerGroupHeartbeatResponse(targetAssignment); membershipManager.updateState(heartbeatResponse.data()); // Target assignment should be in the process of being reconciled - checkAssignments(null, targetAssignment); + checkAssignments(membershipManager, null, targetAssignment); // Mark assignment processing completed - ConsumerGroupHeartbeatResponseData.Assignment reconciled = createAssignment(0); + ConsumerGroupHeartbeatResponseData.Assignment reconciled = + new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Collections.singletonList(0)))); assertThrows(IllegalStateException.class, () -> membershipManager.onTargetAssignmentProcessComplete(reconciled)); } - private void checkAssignments(ConsumerGroupHeartbeatResponseData.Assignment expectedCurrentAssignment, - ConsumerGroupHeartbeatResponseData.Assignment expectedTargetAssignment) { + private void checkAssignments( + MembershipManagerImpl membershipManager, + ConsumerGroupHeartbeatResponseData.Assignment expectedCurrentAssignment, + ConsumerGroupHeartbeatResponseData.Assignment expectedTargetAssignment) { assertEquals(expectedCurrentAssignment, membershipManager.currentAssignment()); assertEquals(expectedTargetAssignment, membershipManager.targetAssignment().orElse(null)); } - private void testStateUpdateOnFenceError() { + private void testStateUpdateOnFenceError(MembershipManager membershipManager) { membershipManager.transitionToFenced(); assertEquals(MemberState.FENCED, membershipManager.state()); // Should reset member epoch and keep member id @@ -216,7 +231,7 @@ private void testStateUpdateOnFenceError() { assertEquals(0, membershipManager.memberEpoch()); } - private void testStateUpdateOnFatalFailure() { + private void testStateUpdateOnFatalFailure(MembershipManager membershipManager) { String initialMemberId = membershipManager.memberId(); int initialMemberEpoch = membershipManager.memberEpoch(); membershipManager.transitionToFailed(); @@ -226,50 +241,30 @@ private void testStateUpdateOnFatalFailure() { assertEquals(initialMemberEpoch, membershipManager.memberEpoch()); } - private ConsumerGroupHeartbeatResponse createHeartbeatResponse() { - return new ConsumerGroupHeartbeatResponse(createHeartbeatResponseData()); - } - - private ConsumerGroupHeartbeatResponse createHeartbeatResponse(ConsumerGroupHeartbeatResponseData.Assignment assignment) { - ConsumerGroupHeartbeatResponseData data = createHeartbeatResponseData() - .setAssignment(assignment); - return new ConsumerGroupHeartbeatResponse(data); + private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse(ConsumerGroupHeartbeatResponseData.Assignment assignment) { + return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + .setErrorCode(Errors.NONE.code()) + .setMemberId(MEMBER_ID) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignment(assignment)); } - private ConsumerGroupHeartbeatResponse createHeartbeatResponse(Errors error) { - ConsumerGroupHeartbeatResponseData data = createHeartbeatResponseData() + private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError(Errors error) { + return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setErrorCode(error.code()) - .setMemberEpoch(5); - return new ConsumerGroupHeartbeatResponse(data); - } - - private ConsumerGroupHeartbeatResponseData createHeartbeatResponseData() { - return new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setMemberId(DEFAULT_MEMBER_ID) - .setMemberEpoch(MEMBER_EPOCH); + .setMemberId(MEMBER_ID) + .setMemberEpoch(5)); } private ConsumerGroupHeartbeatResponseData.Assignment createAssignment() { - return createAssignment( - createTopicPartitions(0, 1, 2), - createTopicPartitions(3, 4, 5) - ); - } - - private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(ConsumerGroupHeartbeatResponseData.TopicPartitions... partitions) { return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList(partitions)); - } - - private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(Integer... partitions) { - return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList(createTopicPartitions(partitions))); - } - - private ConsumerGroupHeartbeatResponseData.TopicPartitions createTopicPartitions(Integer... partitions) { - return new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(DEFAULT_TOPIC_ID) - .setPartitions(Arrays.asList(partitions)); + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(0, 1, 2)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(3, 4, 5)) + )); } -} +} \ No newline at end of file From 48dfc464f8da253d22dc77e39dce385580a292f8 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Nov 2023 16:15:12 -0800 Subject: [PATCH 09/68] Removed more unnecessary changes to align with trunk --- .../org/apache/kafka/clients/Metadata.java | 7 ------ .../apache/kafka/clients/MetadataCache.java | 8 ------ .../internals/PrototypeAsyncConsumer.java | 4 +-- .../internals/ConsumerTestBuilder.java | 25 ++++++++----------- .../internals/MembershipManagerImplTest.java | 2 +- 5 files changed, 13 insertions(+), 33 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 932480a83bb95..c0ebcb704f71e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -282,13 +282,6 @@ public synchronized Map topicIds() { return cache.topicIds(); } - /** - * @return a mapping from topic IDs to topic names for all topics with valid IDs in the cache - */ - public synchronized Map topicNames() { - return cache.topicNames(); - } - public synchronized LeaderAndEpoch currentLeader(TopicPartition topicPartition) { Optional maybeMetadata = partitionMetadataIfCurrent(topicPartition); if (!maybeMetadata.isPresent()) diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java index a87a48217d53f..12210b635e541 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java @@ -51,7 +51,6 @@ public class MetadataCache { private final Node controller; private final Map metadataByPartition; private final Map topicIds; - private final Map topicNames; private Cluster clusterInstance; @@ -82,9 +81,6 @@ private MetadataCache(String clusterId, this.internalTopics = internalTopics; this.controller = controller; this.topicIds = Collections.unmodifiableMap(topicIds); - this.topicNames = Collections.unmodifiableMap( - topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) - ); this.metadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -106,10 +102,6 @@ Map topicIds() { return topicIds; } - Map topicNames() { - return topicNames; - } - Optional nodeById(int id) { return Optional.ofNullable(nodes.get(id)); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 893fca492f453..2c22dc77f4f78 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -34,11 +34,11 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; @@ -143,10 +143,10 @@ public class PrototypeAsyncConsumer implements Consumer { private final long defaultApiTimeoutMs; private volatile boolean closed = false; private final List assignors; - private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates private boolean cachedSubscriptionHasAllFetchPositions; + private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); public PrototypeAsyncConsumer(final Properties properties, final Deserializer keyDeserializer, 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 976b28e85b30b..bd03e7e3c0965 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 @@ -28,7 +28,6 @@ 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.BackgroundEventProcessor; -import org.apache.kafka.common.Uuid; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.MetadataResponse; @@ -40,6 +39,7 @@ import java.io.Closeable; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -55,7 +55,6 @@ 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.common.utils.Utils.closeQuietly; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.spy; @SuppressWarnings("ClassDataAbstractionCoupling") @@ -65,8 +64,6 @@ public class ConsumerTestBuilder implements Closeable { static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; static final int DEFAULT_REQUEST_TIMEOUT_MS = 500; static final int DEFAULT_MAX_POLL_INTERVAL_MS = 10000; - static final String DEFAULT_TOPIC_NAME = "sample-topic-name"; - static final Uuid DEFAULT_TOPIC_ID = Uuid.randomUuid(); static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id"; static final String DEFAULT_GROUP_ID = "group-id"; static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; @@ -151,18 +148,16 @@ public ConsumerTestBuilder(Optional groupInfo) { this.metricsManager = createFetchMetricsManager(metrics); this.client = new MockClient(time, metadata); - MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWithIds( - 1, - Collections.singletonMap(DEFAULT_TOPIC_NAME, 1), - Collections.singletonMap(DEFAULT_TOPIC_NAME, DEFAULT_TOPIC_ID) - ); + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap() { + { + String topic1 = "test1"; + put(topic1, 1); + String topic2 = "test2"; + put(topic2, 1); + } + }); this.client.updateMetadata(metadataResponse); - assertEquals(1, metadata.topicNames().size()); - assertEquals(DEFAULT_TOPIC_NAME, metadata.topicNames().get(DEFAULT_TOPIC_ID)); - assertEquals(1, metadata.topicIds().size()); - assertEquals(DEFAULT_TOPIC_ID, metadata.topicIds().get(DEFAULT_TOPIC_NAME)); - this.networkClientDelegate = spy(new NetworkClientDelegate(time, config, logContext, @@ -194,7 +189,7 @@ public ConsumerTestBuilder(Optional groupInfo) { config, coordinator, groupState)); - MembershipManagerImpl mm = spy( + MembershipManager mm = spy( new MembershipManagerImpl( gi.groupState.groupId, gi.groupState.groupInstanceId.orElse(null), 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 8089cfd8d0bc9..d78bbf2ab63ee 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 @@ -267,4 +267,4 @@ private ConsumerGroupHeartbeatResponseData.Assignment createAssignment() { .setPartitions(Arrays.asList(3, 4, 5)) )); } -} \ No newline at end of file +} From ffe3093e8b64aa9711701cd5996149177305745a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Nov 2023 16:21:33 -0800 Subject: [PATCH 10/68] Reverting more unnecessary changes --- .../main/java/org/apache/kafka/clients/MetadataCache.java | 2 +- .../kafka/clients/consumer/internals/RequestManagers.java | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java index 12210b635e541..38a039a0a388f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java @@ -80,7 +80,7 @@ private MetadataCache(String clusterId, this.invalidTopics = invalidTopics; this.internalTopics = internalTopics; this.controller = controller; - this.topicIds = Collections.unmodifiableMap(topicIds); + this.topicIds = topicIds; this.metadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { 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 bfca04c819275..8f9efe35e3919 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 @@ -121,10 +121,7 @@ public static Supplier supplier(final Time time, @Override protected RequestManagers create() { final NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( - logContext, - backgroundEventQueue - ); + final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); 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); From f7b79e3bc8d40df0c382e8a18166d68779353084 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Nov 2023 16:21:52 -0800 Subject: [PATCH 11/68] Making sure the InternalApplicationEventHandler closed its superclass --- .../consumer/internals/InternalApplicationEventHandler.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java index 97be3e806555c..a7b7bad40b0e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java @@ -49,7 +49,10 @@ class InternalApplicationEventHandler extends ApplicationEventHandler { @Override public void close(final Duration timeout) { closer.close( - () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), + () -> { + Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"); + super.close(timeout); + }, () -> log.warn("The application event handler was already closed") ); } From 9ae3616c086223e1d7fa2b8c1ecf8131ac726b8b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 6 Nov 2023 16:22:07 -0800 Subject: [PATCH 12/68] Minor documentation changes in EventHandler.add --- .../kafka/clients/consumer/internals/events/EventHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java index ad087957603f8..afa69bd56b0d7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java @@ -42,8 +42,8 @@ public EventHandler(final LogContext logContext, final BlockingQueue queue, W } /** - * Add an {@link T} to the underlying queue and internally invoke {@link #notifyWatcher} - * to alert the watcher that it has something to process. + * Add an event to the underlying queue and internally invoke {@link #notifyWatcher} to alert the watcher that + * it has an event to process. * * @param event An event to enqueue for later processing */ From a2a4d733e7aecf92cd3c8fc415a790d8641abe95 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 14:42:10 -0800 Subject: [PATCH 13/68] Updates to try to get back to passing tests --- .../internals/AsyncKafkaConsumer.java | 198 ++++++++++-------- .../internals/ConsumerNetworkThread.java | 5 +- .../InternalApplicationEventHandler.java | 59 ------ .../events/ApplicationEventHandler.java | 54 ++++- .../events/ApplicationEventProcessor.java | 8 +- .../events/BackgroundEventHandler.java | 9 +- .../events/BackgroundEventProcessor.java | 23 +- .../internals/events/EventHandler.java | 7 +- .../internals/events/EventProcessor.java | 40 ++-- .../internals/AsyncKafkaConsumerTest.java | 50 +++-- .../internals/ConsumerTestBuilder.java | 56 +++-- .../events/BackgroundEventHandlerTest.java | 4 +- 12 files changed, 265 insertions(+), 248 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.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 d97f795305eba..2d6a43724f18c 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 @@ -87,6 +87,7 @@ import java.util.Optional; import java.util.OptionalLong; import java.util.Set; +import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -238,7 +239,15 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { metadata, applicationEventQueue, requestManagersSupplier); - ConsumerCoordinatorMetrics sensors = new ConsumerCoordinatorMetrics( + this.applicationEventHandler = new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier + ); + ConsumerCoordinatorMetrics coordinatorMetrics = new ConsumerCoordinatorMetrics( subscriptions, metrics, CONSUMER_METRIC_GROUP_PREFIX @@ -247,19 +256,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { logContext, subscriptions, time, - sensors - ); - ConsumerNetworkThread networkThread = new ConsumerNetworkThread( - logContext, - time, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier - ); - this.applicationEventHandler = new InternalApplicationEventHandler( - logContext, - applicationEventQueue, - networkThread + coordinatorMetrics ); this.backgroundEventProcessor = new BackgroundEventProcessor( logContext, @@ -303,45 +300,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { } } - // Visible for testing - AsyncKafkaConsumer(LogContext logContext, - String clientId, - Deserializers deserializers, - FetchBuffer fetchBuffer, - FetchCollector fetchCollector, - ConsumerInterceptors interceptors, - Time time, - ApplicationEventHandler applicationEventHandler, - BackgroundEventProcessor backgroundEventProcessor, - Metrics metrics, - SubscriptionState subscriptions, - ConsumerMetadata metadata, - long retryBackoffMs, - int defaultApiTimeoutMs, - List assignors, - String groupId) { - this.log = logContext.logger(getClass()); - this.subscriptions = subscriptions; - this.clientId = clientId; - this.fetchBuffer = fetchBuffer; - this.fetchCollector = fetchCollector; - this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; - this.interceptors = Objects.requireNonNull(interceptors); - this.time = time; - this.backgroundEventProcessor = backgroundEventProcessor; - this.metrics = metrics; - this.groupId = Optional.ofNullable(groupId); - this.metadata = metadata; - this.retryBackoffMs = retryBackoffMs; - this.defaultApiTimeoutMs = defaultApiTimeoutMs; - this.deserializers = deserializers; - this.applicationEventHandler = applicationEventHandler; - this.assignors = assignors; - this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); - this.groupInstanceId = Optional.empty(); - } - -// // Visible for testing AsyncKafkaConsumer(LogContext logContext, Time time, ConsumerConfig config, @@ -379,8 +337,8 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); ConsumerCoordinatorMetrics coordinatorMetrics = new ConsumerCoordinatorMetrics( @@ -391,38 +349,31 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - this.applicationEventHandler = new ApplicationEventHandler(logContext, applicationEventQueue); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - coordinatorMetrics - ); - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker + logContext, + subscriptions, + time, + coordinatorMetrics ); ApiVersions apiVersions = new ApiVersions(); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( - time, - config, - logContext, - client + time, + config, + logContext, + client ); Supplier requestManagersSupplier = RequestManagers.supplier( - time, - logContext, - backgroundEventQueue, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier + time, + logContext, + backgroundEventQueue, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -430,6 +381,64 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { applicationEventQueue, requestManagersSupplier ); + this.applicationEventHandler = new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier + ); + this.backgroundEventProcessor = new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ); + } + + // Visible for testing + AsyncKafkaConsumer(LogContext logContext, + String clientId, + Deserializers deserializers, + FetchBuffer fetchBuffer, + FetchCollector fetchCollector, + ConsumerInterceptors interceptors, + Time time, + ApplicationEventHandler applicationEventHandler, + BlockingQueue backgroundEventQueue, + ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, + Metrics metrics, + SubscriptionState subscriptions, + ConsumerMetadata metadata, + long retryBackoffMs, + int defaultApiTimeoutMs, + List assignors, + String groupId) { + this.log = logContext.logger(getClass()); + this.subscriptions = subscriptions; + this.clientId = clientId; + this.fetchBuffer = fetchBuffer; + this.fetchCollector = fetchCollector; + this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; + this.interceptors = Objects.requireNonNull(interceptors); + this.time = time; + this.backgroundEventProcessor = new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ); + this.metrics = metrics; + this.groupId = Optional.ofNullable(groupId); + this.metadata = metadata; + this.retryBackoffMs = retryBackoffMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; + this.deserializers = deserializers; + this.applicationEventHandler = applicationEventHandler; + this.assignors = assignors; + this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); + this.groupInstanceId = Optional.empty(); } /** @@ -1057,21 +1066,27 @@ private void updatePatternSubscription(Cluster cluster) { @Override public void unsubscribe() { + log.debug("unsubscribe - 1"); acquireAndEnsureOpen(); + log.debug("unsubscribe - 2"); try { + log.debug("unsubscribe - 3"); fetchBuffer.retainAll(Collections.emptySet()); + log.debug("unsubscribe - 4"); if (groupId.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); - applicationEventHandler.add(unsubscribeApplicationEvent); - try { - unsubscribeApplicationEvent.future().get(); + log.debug("unsubscribe - 5"); + log.debug("unsubscribe - 6"); + log.debug("unsubscribe - 7"); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); + applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); + log.debug("unsubscribe - 8"); log.info("Unsubscribed all topics or patterns and assigned partitions"); - } catch (InterruptedException | ExecutionException e) { - log.error("Failed while waiting for the unsubscribe event to complete", e); - } } + log.debug("unsubscribe - 9"); subscriptions.unsubscribe(); + log.debug("unsubscribe - 10"); } finally { + log.debug("unsubscribe - 11"); release(); } } @@ -1321,14 +1336,21 @@ private void subscribeInternal(Pattern pattern, Optional topics, Optional listener) { + log.debug("1"); acquireAndEnsureOpen(); + log.debug("2"); try { + log.debug("3"); maybeThrowInvalidGroupIdException(); + log.debug("4"); if (topics == null) throw new IllegalArgumentException("Topic collection to subscribe to cannot be null"); + log.debug("5"); if (topics.isEmpty()) { + log.debug("6"); // treat subscribing to empty topic list as the same as unsubscribing unsubscribe(); + log.debug("7"); } else { for (String topic : topics) { if (isBlank(topic)) 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 6519e3ef48ae2..ae9997ba2b1f6 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 @@ -132,8 +132,9 @@ void initializeResources() { * */ void runOnce() { - // If there are errors processing any events, the error will be thrown immediately. This will have - // the effect of closing the background thread. + // 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 long currentTimeMs = time.milliseconds(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java deleted file mode 100644 index a7b7bad40b0e1..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/InternalApplicationEventHandler.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.common.internals.IdempotentCloser; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; - -import java.time.Duration; -import java.util.concurrent.BlockingQueue; - -/** - * An event handler that receives {@link ApplicationEvent application events} from the application thread which - * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. - */ -class InternalApplicationEventHandler extends ApplicationEventHandler { - - private final Logger log; - private final ConsumerNetworkThread networkThread; - private final IdempotentCloser closer = new IdempotentCloser(); - - InternalApplicationEventHandler(final LogContext logContext, - final BlockingQueue applicationEventQueue, - final ConsumerNetworkThread networkThread) { - super(logContext, applicationEventQueue, networkThread::wakeup); - this.log = logContext.logger(InternalApplicationEventHandler.class); - this.networkThread = networkThread; - this.networkThread.start(); - } - - @Override - public void close(final Duration timeout) { - closer.close( - () -> { - Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"); - super.close(timeout); - }, - () -> log.warn("The application event handler was already closed") - ); - } -} 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 85d74b71a900b..aba609e4b5ab0 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,21 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +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; +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; /** * An event handler that receives {@link ApplicationEvent application events} from the application thread which @@ -31,15 +39,26 @@ */ public class ApplicationEventHandler extends EventHandler { - public ApplicationEventHandler(final LogContext logContext, - final BlockingQueue queue) { - this(logContext, queue, () -> { }); - } + private final Logger log; + private final ConsumerNetworkThread networkThread; + private final IdempotentCloser closer = new IdempotentCloser(); public ApplicationEventHandler(final LogContext logContext, - final BlockingQueue queue, - final Watcher watcher) { - super(logContext, queue, watcher); + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier) { + super(logContext, applicationEventQueue); + this.log = logContext.logger(ApplicationEventHandler.class); + this.networkThread = new ConsumerNetworkThread( + logContext, + time, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier + ); + this.networkThread.start(); } /** @@ -61,4 +80,25 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t add(event); return event.get(timer); } + + @Override + public void notifyWatcher() { + networkThread.wakeup(); + } + + @Override + public void close() { + close(Duration.ZERO); + } + + @Override + public void close(final Duration timeout) { + closer.close( + () -> { + Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"); + super.close(timeout); + }, + () -> log.warn("The application event handler was already closed") + ); + } } 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 3897e51a73508..04af02ea7c282 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,9 +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. */ - @Override public void process() { - process((event, error) -> { }); + process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override @@ -126,11 +125,6 @@ public void process(ApplicationEvent event) { } } - @Override - protected Class getEventClass() { - return ApplicationEvent.class; - } - private void process(final PollApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; 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 107463d4014b4..f36628afce75d 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,16 +26,9 @@ * {@link ConsumerNetworkThread network thread} which are then made available to the application thread * via the {@link BackgroundEventProcessor}. */ - public class BackgroundEventHandler extends EventHandler { public BackgroundEventHandler(final LogContext logContext, final BlockingQueue queue) { - this(logContext, queue, () -> { }); - } - - public BackgroundEventHandler(final LogContext logContext, - final BlockingQueue queue, - final Watcher watcher) { - super(logContext, queue, watcher); + super(logContext, queue); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index ba2bcfbba3b18..f614d4ce1511c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -22,13 +22,13 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; import java.util.Optional; import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicReference; - /** * An {@link EventProcessor} that is created and executes in the application thread for the purpose of processing * {@link BackgroundEvent background events} generated by the {@link ConsumerNetworkThread network thread}. @@ -41,6 +41,7 @@ */ public class BackgroundEventProcessor extends EventProcessor { + private final Logger log; private final ApplicationEventHandler applicationEventHandler; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; @@ -49,6 +50,7 @@ public BackgroundEventProcessor(final LogContext logContext, final ApplicationEventHandler applicationEventHandler, final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { super(logContext, backgroundEventQueue); + this.log = logContext.logger(BackgroundEventProcessor.class); this.applicationEventHandler = applicationEventHandler; this.rebalanceListenerInvoker = rebalanceListenerInvoker; } @@ -59,10 +61,20 @@ public BackgroundEventProcessor(final LogContext logContext, * 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 void process() { AtomicReference firstError = new AtomicReference<>(); - process((event, error) -> firstError.compareAndSet(null, error)); + + ProcessHandler processHandler = (event, error) -> { + if (error.isPresent()) { + KafkaException e = error.get(); + + if (!firstError.compareAndSet(null, e)) { + log.warn("An error occurred when processing the event: {}", e.getMessage(), e); + } + } + }; + + process(processHandler); if (firstError.get() != null) throw firstError.get(); @@ -88,11 +100,6 @@ public void process(final BackgroundEvent event) { } } - @Override - protected Class getEventClass() { - return BackgroundEvent.class; - } - private void process(final ErrorBackgroundEvent event) { throw event.error(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java index afa69bd56b0d7..abc579a5c9ea8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java @@ -33,12 +33,10 @@ public class EventHandler implements Closeable { private final Logger log; private final BlockingQueue queue; - private final Watcher watcher; - public EventHandler(final LogContext logContext, final BlockingQueue queue, Watcher watcher) { + public EventHandler(final LogContext logContext, final BlockingQueue queue) { this.log = logContext.logger(EventHandler.class); this.queue = queue; - this.watcher = watcher; } /** @@ -51,11 +49,10 @@ public void add(final T event) { Objects.requireNonNull(event, "Event must be non-null"); log.trace("Enqueued event: {}", event); queue.add(event); - watcher.updated(); } public void notifyWatcher() { - watcher.updated(); + // Do nothing...; } @Override 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 06e7ec28dd7e3..be3335fc22645 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 @@ -25,6 +25,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.BlockingQueue; /** @@ -36,6 +37,11 @@ */ public abstract class EventProcessor implements Closeable { + public interface ProcessHandler { + + void onProcess(T event, Optional error); + } + private final Logger log; private final BlockingQueue eventQueue; private final IdempotentCloser closer; @@ -46,8 +52,6 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } - public abstract void process(); - public abstract void process(T event); @Override @@ -55,34 +59,25 @@ public void close() { closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); } - protected abstract Class getEventClass(); - - protected interface ProcessErrorHandler { - - void onError(T event, KafkaException 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 ProcessErrorHandler}. + * processing the individual events, these are submitted to the given {@link ProcessHandler}. */ - protected void process(ProcessErrorHandler processErrorHandler) { - String eventClassName = getEventClass().getSimpleName(); - closer.assertOpen(() -> String.format("The processor was previously closed, so no further %s processing can occur", eventClassName)); + protected void process(ProcessHandler processHandler) { + closer.assertOpen("The processor was previously closed, so no further processing can occur"); List events = drain(); try { - log.debug("Starting processing of {} {}(s)", events.size(), eventClassName); + log.trace("Starting processing of {} event(s)", events.size()); for (T event : events) { try { - Objects.requireNonNull(event, () -> String.format("Attempted to process a null %s", eventClassName)); - log.debug("Consuming {}: {}", eventClassName, event); + Objects.requireNonNull(event, "Attempted to process a null event"); + log.trace("Consuming event: {}", event); process(event); + processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { - log.warn("An error occurred when processing the {}: {}", eventClassName, t.getMessage(), t); - KafkaException error; if (t instanceof KafkaException) @@ -90,11 +85,11 @@ protected void process(ProcessErrorHandler processErrorHandler) { else error = new KafkaException(t); - processErrorHandler.onError(event, error); + processHandler.onProcess(event, Optional.of(error)); } } } finally { - log.debug("Completed processing of {} {}(s)", events.size(), eventClassName); + log.trace("Completed processing of {} event(s)", events.size()); } } @@ -103,8 +98,7 @@ protected void process(ProcessErrorHandler processErrorHandler) { * this case, we need to throw an exception to notify the user the consumer is closed. */ private void closeInternal() { - String eventClassName = getEventClass().getSimpleName(); - log.trace("Closing event processor for {}", eventClassName); + log.trace("Closing event processor"); List incompleteEvents = drain(); if (incompleteEvents.isEmpty()) @@ -123,7 +117,7 @@ private void closeInternal() { f.completeExceptionally(exception); }); - log.debug("Discarding {} {}s because the consumer is closing", incompleteEvents.size(), eventClassName); + log.debug("Discarding {} event(s) because the consumer is closing", incompleteEvents.size()); } /** diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index f079e8d6e6d0a..1cfbe5adcb382 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 @@ -40,6 +40,7 @@ import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -52,6 +53,7 @@ import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.stubbing.Answer; +import org.slf4j.Logger; import java.time.Duration; import java.util.Collections; @@ -512,27 +514,33 @@ public void testSubscribeGeneratesEvent() { verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); } - @Test - public void testUnsubscribeGeneratesUnsubscribeEvent() { - consumer.unsubscribe(); - - // Verify the unsubscribe event was generated and mock its completion. - final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); - verify(applicationEventHandler).add(captor.capture()); - UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); - unsubscribeApplicationEvent.future().complete(null); - - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testSubscribeToEmptyListActsAsUnsubscribe() { - consumer.subscribe(Collections.emptyList()); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); - } +// @Test +// public void testUnsubscribeGeneratesUnsubscribeEvent() { +// consumer.unsubscribe(); +// +// // Verify the unsubscribe event was generated and mock its completion. +// final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); +// verify(applicationEventHandler).add(captor.capture()); +// UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); +// unsubscribeApplicationEvent.future().complete(null); +// +// assertTrue(consumer.subscription().isEmpty()); +// assertTrue(consumer.assignment().isEmpty()); +// } + +// @Test +// public void testSubscribeToEmptyListActsAsUnsubscribe() { +// Logger logger = new LogContext().logger(getClass()); +// logger.debug("a"); +// consumer.subscribe(Collections.emptyList()); +// logger.debug("b"); +// assertTrue(consumer.subscription().isEmpty()); +// logger.debug("c"); +// assertTrue(consumer.assignment().isEmpty()); +// logger.debug("d"); +// verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); +// logger.debug("e"); +// } @Test public void testSubscribeToNullTopicCollection() { 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 f8c3becbb868c..edf602916a11a 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 @@ -69,7 +69,7 @@ public class ConsumerTestBuilder implements Closeable { static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; - final LogContext logContext = new LogContext(); + public final LogContext logContext = new LogContext(); final Time time = new MockTime(0); public final BlockingQueue applicationEventQueue; public final BlockingQueue backgroundEventQueue; @@ -92,8 +92,6 @@ public class ConsumerTestBuilder implements Closeable { final FetchRequestManager fetchRequestManager; final RequestManagers requestManagers; public final ApplicationEventProcessor applicationEventProcessor; - public final BackgroundEventProcessor backgroundEventProcessor; - public ApplicationEventHandler applicationEventHandler; public final BackgroundEventHandler backgroundEventHandler; public final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; final MockClient client; @@ -107,10 +105,6 @@ public ConsumerTestBuilder(Optional groupInfo) { this.groupInfo = groupInfo; this.applicationEventQueue = new LinkedBlockingQueue<>(); this.backgroundEventQueue = new LinkedBlockingQueue<>(); - this.applicationEventHandler = spy(new ApplicationEventHandler( - logContext, - applicationEventQueue - )); this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( 100, @@ -265,21 +259,12 @@ public ConsumerTestBuilder(Optional groupInfo) { time, consumerCoordinatorMetrics ); - this.backgroundEventProcessor = spy( - new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ) - ); } @Override public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); - closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); } public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { @@ -307,7 +292,41 @@ public void close() { } } - public static class AsyncKafkaConsumerTestBuilder extends ConsumerTestBuilder { + public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { + + public final ApplicationEventHandler applicationEventHandler; + public final BackgroundEventProcessor backgroundEventProcessor; + + public ApplicationEventHandlerTestBuilder() { + this(createDefaultGroupInformation()); + } + + public ApplicationEventHandlerTestBuilder(Optional groupInfo) { + super(groupInfo); + this.applicationEventHandler = spy(new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + () -> applicationEventProcessor, + () -> networkClientDelegate, + () -> requestManagers)); + this.backgroundEventProcessor = spy( + new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ) + ); + } + + @Override + public void close() { + closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); + } + } + + public static class AsyncKafkaConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { final AsyncKafkaConsumer consumer; @@ -335,7 +354,8 @@ public AsyncKafkaConsumerTestBuilder(Optional groupInfo) { new ConsumerInterceptors<>(Collections.emptyList()), time, applicationEventHandler, - backgroundEventProcessor, + backgroundEventQueue, + rebalanceListenerInvoker, metrics, subscriptions, metadata, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java index 0670b8bdb7c3f..6ef2c1f64a436 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java @@ -32,14 +32,14 @@ public class BackgroundEventHandlerTest { - private ConsumerTestBuilder testBuilder; + private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private BackgroundEventProcessor backgroundEventProcessor; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder(); + testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(); backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; backgroundEventProcessor = testBuilder.backgroundEventProcessor; From 926041e688a3386e5550765abb5e4cf0a3d211de Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 14:49:09 -0800 Subject: [PATCH 14/68] More updates to get to a clean slate --- .../internals/AsyncKafkaConsumer.java | 147 ++++++++---------- .../events/ApplicationEventHandler.java | 15 +- .../internals/events/EventHandler.java | 7 +- .../internals/AsyncKafkaConsumerTest.java | 54 +++---- 4 files changed, 107 insertions(+), 116 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 2d6a43724f18c..f4dd2b9c943e1 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 @@ -87,10 +87,8 @@ import java.util.Optional; import java.util.OptionalLong; import java.util.Set; -import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -239,8 +237,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { metadata, applicationEventQueue, requestManagersSupplier); - this.applicationEventHandler = new ApplicationEventHandler( - logContext, + this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, applicationEventProcessorSupplier, @@ -300,6 +297,50 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { } } + // Visible for testing + AsyncKafkaConsumer(LogContext logContext, + String clientId, + Deserializers deserializers, + FetchBuffer fetchBuffer, + FetchCollector fetchCollector, + ConsumerInterceptors interceptors, + Time time, + ApplicationEventHandler applicationEventHandler, + BlockingQueue backgroundEventQueue, + ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, + Metrics metrics, + SubscriptionState subscriptions, + ConsumerMetadata metadata, + long retryBackoffMs, + int defaultApiTimeoutMs, + List assignors, + String groupId) { + this.log = logContext.logger(getClass()); + this.subscriptions = subscriptions; + this.clientId = clientId; + this.fetchBuffer = fetchBuffer; + this.fetchCollector = fetchCollector; + this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; + this.interceptors = Objects.requireNonNull(interceptors); + this.time = time; + this.backgroundEventProcessor = new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + applicationEventHandler, + rebalanceListenerInvoker + ); + this.metrics = metrics; + this.groupId = Optional.ofNullable(groupId); + this.metadata = metadata; + this.retryBackoffMs = retryBackoffMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; + this.deserializers = deserializers; + this.applicationEventHandler = applicationEventHandler; + this.assignors = assignors; + this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); + this.groupInstanceId = Optional.empty(); + } + AsyncKafkaConsumer(LogContext logContext, Time time, ConsumerConfig config, @@ -337,8 +378,8 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( - config, - GroupRebalanceConfig.ProtocolType.CONSUMER + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); ConsumerCoordinatorMetrics coordinatorMetrics = new ConsumerCoordinatorMetrics( @@ -350,10 +391,10 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( - logContext, - subscriptions, - time, - coordinatorMetrics + logContext, + subscriptions, + time, + coordinatorMetrics ); ApiVersions apiVersions = new ApiVersions(); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( @@ -363,17 +404,17 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { client ); Supplier requestManagersSupplier = RequestManagers.supplier( - time, - logContext, - backgroundEventQueue, - metadata, - subscriptions, - fetchBuffer, - config, - groupRebalanceConfig, - apiVersions, - fetchMetricsManager, - networkClientDelegateSupplier + time, + logContext, + backgroundEventQueue, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -397,50 +438,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { ); } - // Visible for testing - AsyncKafkaConsumer(LogContext logContext, - String clientId, - Deserializers deserializers, - FetchBuffer fetchBuffer, - FetchCollector fetchCollector, - ConsumerInterceptors interceptors, - Time time, - ApplicationEventHandler applicationEventHandler, - BlockingQueue backgroundEventQueue, - ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, - Metrics metrics, - SubscriptionState subscriptions, - ConsumerMetadata metadata, - long retryBackoffMs, - int defaultApiTimeoutMs, - List assignors, - String groupId) { - this.log = logContext.logger(getClass()); - this.subscriptions = subscriptions; - this.clientId = clientId; - this.fetchBuffer = fetchBuffer; - this.fetchCollector = fetchCollector; - this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; - this.interceptors = Objects.requireNonNull(interceptors); - this.time = time; - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ); - this.metrics = metrics; - this.groupId = Optional.ofNullable(groupId); - this.metadata = metadata; - this.retryBackoffMs = retryBackoffMs; - this.defaultApiTimeoutMs = defaultApiTimeoutMs; - this.deserializers = deserializers; - this.applicationEventHandler = applicationEventHandler; - this.assignors = assignors; - this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); - this.groupInstanceId = Optional.empty(); - } - /** * poll implementation using {@link ApplicationEventHandler}. * 1. Poll for background events. If there's a fetch response event, process the record and return it. If it is @@ -913,7 +910,7 @@ private void close(Duration timeout, boolean swallowException) { AtomicReference firstException = new AtomicReference<>(); if (applicationEventHandler != null) - closeQuietly(() -> applicationEventHandler.close(timeout), "Failed to close consumer network thread with a timeout(ms)=" + timeout, firstException); + closeQuietly(() -> applicationEventHandler.close(timeout), "Failed to close application event handler with a timeout(ms)=" + timeout, firstException); // Invoke all callbacks after the background thread exists in case if there are unsent async // commits @@ -1066,27 +1063,15 @@ private void updatePatternSubscription(Cluster cluster) { @Override public void unsubscribe() { - log.debug("unsubscribe - 1"); acquireAndEnsureOpen(); - log.debug("unsubscribe - 2"); try { - log.debug("unsubscribe - 3"); fetchBuffer.retainAll(Collections.emptySet()); - log.debug("unsubscribe - 4"); if (groupId.isPresent()) { - log.debug("unsubscribe - 5"); - log.debug("unsubscribe - 6"); - log.debug("unsubscribe - 7"); - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); - applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); - log.debug("unsubscribe - 8"); - log.info("Unsubscribed all topics or patterns and assigned partitions"); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); + applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); } - log.debug("unsubscribe - 9"); subscriptions.unsubscribe(); - log.debug("unsubscribe - 10"); } finally { - log.debug("unsubscribe - 11"); release(); } } @@ -1153,7 +1138,7 @@ private Fetch collectFetch() { final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); // Notify the network thread to wake up and start the next round of fetching. - applicationEventHandler.notifyWatcher(); + applicationEventHandler.wakeupNetworkThread(); return fetch; } 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 aba609e4b5ab0..821e1809ffaee 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 @@ -61,6 +61,18 @@ public ApplicationEventHandler(final LogContext logContext, this.networkThread.start(); } + /** + * Add an event to the underlying queue and internally invoke {@link #wakeupNetworkThread()} to alert it that + * it has an event to process. + * + * @param event An event to enqueue for later processing + */ + @Override + public void add(ApplicationEvent event) { + super.add(event); + wakeupNetworkThread(); + } + /** * Add a {@link CompletableApplicationEvent} to the underlying queue. The method blocks waiting for the result, * and will return the result value upon successful completion; otherwise throws an error. @@ -81,8 +93,7 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t return event.get(timer); } - @Override - public void notifyWatcher() { + public void wakeupNetworkThread() { networkThread.wakeup(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java index abc579a5c9ea8..9dd072c331274 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java @@ -40,8 +40,7 @@ public EventHandler(final LogContext logContext, final BlockingQueue queue) { } /** - * Add an event to the underlying queue and internally invoke {@link #notifyWatcher} to alert the watcher that - * it has an event to process. + * Add an event to the underlying queue. * * @param event An event to enqueue for later processing */ @@ -51,10 +50,6 @@ public void add(final T event) { queue.add(event); } - public void notifyWatcher() { - // Do nothing...; - } - @Override public void close() { close(Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS)); 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 1cfbe5adcb382..662edd74bb544 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 @@ -514,33 +514,33 @@ public void testSubscribeGeneratesEvent() { verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); } -// @Test -// public void testUnsubscribeGeneratesUnsubscribeEvent() { -// consumer.unsubscribe(); -// -// // Verify the unsubscribe event was generated and mock its completion. -// final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); -// verify(applicationEventHandler).add(captor.capture()); -// UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); -// unsubscribeApplicationEvent.future().complete(null); -// -// assertTrue(consumer.subscription().isEmpty()); -// assertTrue(consumer.assignment().isEmpty()); -// } - -// @Test -// public void testSubscribeToEmptyListActsAsUnsubscribe() { -// Logger logger = new LogContext().logger(getClass()); -// logger.debug("a"); -// consumer.subscribe(Collections.emptyList()); -// logger.debug("b"); -// assertTrue(consumer.subscription().isEmpty()); -// logger.debug("c"); -// assertTrue(consumer.assignment().isEmpty()); -// logger.debug("d"); -// verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); -// logger.debug("e"); -// } + @Test + public void testUnsubscribeGeneratesUnsubscribeEvent() { + consumer.unsubscribe(); + + // Verify the unsubscribe event was generated and mock its completion. + final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); + verify(applicationEventHandler).add(captor.capture()); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); + unsubscribeApplicationEvent.future().complete(null); + + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testSubscribeToEmptyListActsAsUnsubscribe() { + Logger logger = new LogContext().logger(getClass()); + logger.debug("a"); + consumer.subscribe(Collections.emptyList()); + logger.debug("b"); + assertTrue(consumer.subscription().isEmpty()); + logger.debug("c"); + assertTrue(consumer.assignment().isEmpty()); + logger.debug("d"); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + logger.debug("e"); + } @Test public void testSubscribeToNullTopicCollection() { From e8571043452f29a9e446f54a284ba25af9c49d9b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 14:53:04 -0800 Subject: [PATCH 15/68] Clean up #1256 --- .../internals/AsyncKafkaConsumer.java | 23 ++++++------ .../events/ApplicationEventHandler.java | 11 +++--- .../internals/ConsumerTestBuilder.java | 35 +++++++++---------- 3 files changed, 33 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 f4dd2b9c943e1..e586ad3b1f491 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 @@ -63,6 +63,7 @@ import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -398,10 +399,10 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { ); ApiVersions apiVersions = new ApiVersions(); Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate( - time, - config, - logContext, - client + time, + config, + logContext, + client ); Supplier requestManagersSupplier = RequestManagers.supplier( time, @@ -1068,7 +1069,12 @@ public void unsubscribe() { fetchBuffer.retainAll(Collections.emptySet()); if (groupId.isPresent()) { UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); - applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); + try { + applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); + log.info("Unsubscribed all topics or patterns and assigned partitions"); + } catch (TimeoutException e) { + log.error("Failed while waiting for the unsubscribe event to complete", e); + } } subscriptions.unsubscribe(); } finally { @@ -1321,21 +1327,14 @@ private void subscribeInternal(Pattern pattern, Optional topics, Optional listener) { - log.debug("1"); acquireAndEnsureOpen(); - log.debug("2"); try { - log.debug("3"); maybeThrowInvalidGroupIdException(); - log.debug("4"); if (topics == null) throw new IllegalArgumentException("Topic collection to subscribe to cannot be null"); - log.debug("5"); if (topics.isEmpty()) { - log.debug("6"); // treat subscribing to empty topic list as the same as unsubscribing unsubscribe(); - log.debug("7"); } else { for (String topic : topics) { if (isBlank(topic)) 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 821e1809ffaee..4dfecd7f660ed 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 @@ -51,12 +51,11 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier requestManagersSupplier) { super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventHandler.class); - this.networkThread = new ConsumerNetworkThread( - logContext, - time, - applicationEventProcessorSupplier, - networkClientDelegateSupplier, - requestManagersSupplier + this.networkThread = new ConsumerNetworkThread(logContext, + time, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier ); this.networkThread.start(); } 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 edf602916a11a..7ce520d3b8c25 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 @@ -346,24 +346,23 @@ public AsyncKafkaConsumerTestBuilder(Optional groupInfo) { metricsManager, time); this.consumer = spy(new AsyncKafkaConsumer<>( - logContext, - clientId, - deserializers, - new FetchBuffer(logContext), - fetchCollector, - new ConsumerInterceptors<>(Collections.emptyList()), - time, - applicationEventHandler, - backgroundEventQueue, - rebalanceListenerInvoker, - metrics, - subscriptions, - metadata, - retryBackoffMs, - 60000, - assignors, - groupInfo.map(groupInformation -> groupInformation.groupState.groupId).orElse(null) - )); + logContext, + clientId, + deserializers, + new FetchBuffer(logContext), + fetchCollector, + new ConsumerInterceptors<>(Collections.emptyList()), + time, + applicationEventHandler, + backgroundEventQueue, + rebalanceListenerInvoker, + metrics, + subscriptions, + metadata, + retryBackoffMs, + 60000, + assignors, + groupInfo.map(groupInformation -> groupInformation.groupState.groupId).orElse(null))); } @Override From 9d372d7f601ab293ce7265a7638ca8b9a9dda785 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 14:59:56 -0800 Subject: [PATCH 16/68] Yet more clean up --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 4 +--- .../consumer/internals/AsyncKafkaConsumerTest.java | 8 -------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e586ad3b1f491..e54cdcee1fe1f 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 @@ -63,7 +63,6 @@ import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -243,8 +242,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { applicationEventQueue, applicationEventProcessorSupplier, networkClientDelegateSupplier, - requestManagersSupplier - ); + requestManagersSupplier); ConsumerCoordinatorMetrics coordinatorMetrics = new ConsumerCoordinatorMetrics( subscriptions, metrics, 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 662edd74bb544..f079e8d6e6d0a 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 @@ -40,7 +40,6 @@ import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ListOffsetsRequest; -import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -53,7 +52,6 @@ import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.stubbing.Answer; -import org.slf4j.Logger; import java.time.Duration; import java.util.Collections; @@ -530,16 +528,10 @@ public void testUnsubscribeGeneratesUnsubscribeEvent() { @Test public void testSubscribeToEmptyListActsAsUnsubscribe() { - Logger logger = new LogContext().logger(getClass()); - logger.debug("a"); consumer.subscribe(Collections.emptyList()); - logger.debug("b"); assertTrue(consumer.subscription().isEmpty()); - logger.debug("c"); assertTrue(consumer.assignment().isEmpty()); - logger.debug("d"); verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); - logger.debug("e"); } @Test From c044b57b5b36a51f79ceb3326c6f3ce7bb22ac4b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 15:05:36 -0800 Subject: [PATCH 17/68] More clean up --- .../events/ApplicationEventProcessor.java | 5 ++++ .../events/BackgroundEventProcessor.java | 5 ++++ .../internals/events/EventHandler.java | 5 ---- .../internals/events/EventProcessor.java | 28 +++++++++++-------- .../internals/ConsumerTestBuilder.java | 3 +- 5 files changed, 28 insertions(+), 18 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 04af02ea7c282..139cf357ab082 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 @@ -125,6 +125,11 @@ public void process(ApplicationEvent event) { } } + @Override + protected Class getEventClass() { + return ApplicationEvent.class; + } + private void process(final PollApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index f614d4ce1511c..52caba861e9dd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -80,6 +80,11 @@ public void process() { throw firstError.get(); } + @Override + protected Class getEventClass() { + return BackgroundEvent.class; + } + @Override public void process(final BackgroundEvent event) { switch (event.type()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java index 9dd072c331274..2c92f190c5118 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java @@ -58,9 +58,4 @@ public void close() { public void close(Duration timeout) { // Do nothing. Available for subclasses. } - - public interface Watcher { - - void updated(); - } } 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 be3335fc22645..2e56a9fede54f 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 @@ -37,11 +37,6 @@ */ public abstract class EventProcessor implements Closeable { - public interface ProcessHandler { - - void onProcess(T event, Optional error); - } - private final Logger log; private final BlockingQueue eventQueue; private final IdempotentCloser closer; @@ -52,29 +47,37 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } - public abstract void process(T event); + protected abstract void process(T event); @Override public void close() { closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); } + protected abstract Class getEventClass(); + + public 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 void process(ProcessHandler processHandler) { - closer.assertOpen("The processor was previously closed, so no further processing can occur"); + String eventClassName = getEventClass().getSimpleName(); + closer.assertOpen(() -> String.format("The processor was previously closed, so no further %s processing can occur", eventClassName)); List events = drain(); try { - log.trace("Starting processing of {} event(s)", events.size()); + log.trace("Starting processing of {} {}(s)", events.size(), eventClassName); for (T event : events) { try { Objects.requireNonNull(event, "Attempted to process a null event"); - log.trace("Consuming event: {}", event); + log.trace("Consuming {}: {}", eventClassName, event); process(event); processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { @@ -89,7 +92,7 @@ protected void process(ProcessHandler processHandler) { } } } finally { - log.trace("Completed processing of {} event(s)", events.size()); + log.debug("Completed processing of {} {}(s)", events.size(), eventClassName); } } @@ -98,7 +101,8 @@ protected void process(ProcessHandler processHandler) { * this case, we need to throw an exception to notify the user the consumer is closed. */ private void closeInternal() { - log.trace("Closing event processor"); + String eventClassName = getEventClass().getSimpleName(); + log.trace("Closing event processor for {}", eventClassName); List incompleteEvents = drain(); if (incompleteEvents.isEmpty()) @@ -117,7 +121,7 @@ private void closeInternal() { f.completeExceptionally(exception); }); - log.debug("Discarding {} event(s) because the consumer is closing", incompleteEvents.size()); + log.debug("Discarding {} {}s because the consumer is closing", incompleteEvents.size(), eventClassName); } /** 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 7ce520d3b8c25..77ca2a1f1c5eb 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 @@ -69,7 +69,7 @@ public class ConsumerTestBuilder implements Closeable { static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; - public final LogContext logContext = new LogContext(); + final LogContext logContext = new LogContext(); final Time time = new MockTime(0); public final BlockingQueue applicationEventQueue; public final BlockingQueue backgroundEventQueue; @@ -322,6 +322,7 @@ public ApplicationEventHandlerTestBuilder(Optional groupInfo) @Override public void close() { + closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); } } From e8287014f68a8634253375abad2ac51a1cc353e1 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 15:06:53 -0800 Subject: [PATCH 18/68] More formatting reverting --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e54cdcee1fe1f..3f88354555e39 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 @@ -421,14 +421,12 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { applicationEventQueue, requestManagersSupplier ); - this.applicationEventHandler = new ApplicationEventHandler( - logContext, + this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, applicationEventProcessorSupplier, networkClientDelegateSupplier, - requestManagersSupplier - ); + requestManagersSupplier); this.backgroundEventProcessor = new BackgroundEventProcessor( logContext, backgroundEventQueue, From ac4b4e9f87fab1dd6fbd044ae09b7e1fca015490 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 15:09:30 -0800 Subject: [PATCH 19/68] More clean up --- .../internals/events/BackgroundEventProcessor.java | 10 +++++----- .../consumer/internals/events/EventProcessor.java | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index 52caba861e9dd..1bf68d1a51ac9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -80,11 +80,6 @@ public void process() { throw firstError.get(); } - @Override - protected Class getEventClass() { - return BackgroundEvent.class; - } - @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -105,6 +100,11 @@ public void process(final BackgroundEvent event) { } } + @Override + protected Class getEventClass() { + return BackgroundEvent.class; + } + private void process(final ErrorBackgroundEvent event) { throw event.error(); } 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 2e56a9fede54f..83cd7bf60933d 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 @@ -56,7 +56,7 @@ public void close() { protected abstract Class getEventClass(); - public interface ProcessHandler { + protected interface ProcessHandler { void onProcess(T event, Optional error); } @@ -76,7 +76,7 @@ protected void process(ProcessHandler processHandler) { for (T event : events) { try { - Objects.requireNonNull(event, "Attempted to process a null event"); + Objects.requireNonNull(event, () -> String.format("Attempted to process a null %s", eventClassName)); log.trace("Consuming {}: {}", eventClassName, event); process(event); processHandler.onProcess(event, Optional.empty()); From 77e8d8fd917adeb5290c6479b158a7d0d8018bcf Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 20 Nov 2023 15:10:51 -0800 Subject: [PATCH 20/68] More formatting clean up --- .../consumer/internals/events/ApplicationEventHandler.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 4dfecd7f660ed..0137ca4f05573 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 @@ -55,8 +55,7 @@ public ApplicationEventHandler(final LogContext logContext, time, applicationEventProcessorSupplier, networkClientDelegateSupplier, - requestManagersSupplier - ); + requestManagersSupplier); this.networkThread.start(); } @@ -67,7 +66,7 @@ public ApplicationEventHandler(final LogContext logContext, * @param event An event to enqueue for later processing */ @Override - public void add(ApplicationEvent event) { + public void add(final ApplicationEvent event) { super.add(event); wakeupNetworkThread(); } From c2e74894519ed63db08033b007cdfcc99e5db9ee Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 21 Nov 2023 16:54:40 -0800 Subject: [PATCH 21/68] Updates for a first pass at end-to-end logic. Still needs tests created/updated. --- ...ConsumerRebalanceListenerCallbackName.java | 21 +++ .../ConsumerRebalanceListenerInvoker.java | 50 +------ .../internals/HeartbeatRequestManager.java | 4 + .../consumer/internals/MembershipManager.java | 16 ++ .../internals/MembershipManagerImpl.java | 139 +++++++++++++++++- .../consumer/internals/RequestManagers.java | 11 +- .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventProcessor.java | 40 ++--- .../internals/events/BackgroundEvent.java | 2 +- .../events/BackgroundEventProcessor.java | 70 ++++----- ...alanceListenerCallbackCompletedEvent.java} | 54 +++---- ...rRebalanceListenerCallbackNeededEvent.java | 84 +++++++++++ .../events/PartitionLostCompleteEvent.java | 82 ----------- .../events/PartitionLostStartedEvent.java | 76 ---------- .../events/RebalanceStartedEvent.java | 88 ----------- .../SubscriptionChangeApplicationEvent.java | 2 +- .../internals/ConsumerTestBuilder.java | 6 +- .../internals/MembershipManagerImplTest.java | 9 +- 18 files changed, 355 insertions(+), 401 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{RebalanceCompleteEvent.java => ConsumerRebalanceListenerCallbackCompletedEvent.java} (51%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java new file mode 100644 index 0000000000000..9652ac9f4c10d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.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; + +public enum ConsumerRebalanceListenerCallbackName { + onPartitionsRevoked, onPartitionsAssigned, onPartitionsLost; +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java index b308c1aec3837..3ac77e08dd965 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.WakeupException; @@ -30,7 +29,6 @@ import java.util.Optional; import java.util.Set; import java.util.SortedSet; -import java.util.concurrent.atomic.AtomicReference; /** * This class encapsulates the invocation of the callback methods defined in the {@link ConsumerRebalanceListener} @@ -54,7 +52,7 @@ public class ConsumerRebalanceListenerInvoker { this.coordinatorMetrics = coordinatorMetrics; } - Exception invokePartitionsAssigned(final SortedSet assignedPartitions) { + public Exception invokePartitionsAssigned(final SortedSet assignedPartitions) { log.info("Adding newly assigned partitions: {}", Utils.join(assignedPartitions, ", ")); Optional listener = subscriptions.rebalanceListener(); @@ -76,7 +74,7 @@ Exception invokePartitionsAssigned(final SortedSet assignedParti return null; } - Exception invokePartitionsRevoked(final SortedSet revokedPartitions) { + public Exception invokePartitionsRevoked(final SortedSet revokedPartitions) { log.info("Revoke previously assigned partitions {}", Utils.join(revokedPartitions, ", ")); Set revokePausedPartitions = subscriptions.pausedPartitions(); revokePausedPartitions.retainAll(revokedPartitions); @@ -102,7 +100,7 @@ Exception invokePartitionsRevoked(final SortedSet revokedPartiti return null; } - Exception invokePartitionsLost(final SortedSet lostPartitions) { + public Exception invokePartitionsLost(final SortedSet lostPartitions) { log.info("Lost previously assigned partitions {}", Utils.join(lostPartitions, ", ")); Set lostPausedPartitions = subscriptions.pausedPartitions(); lostPausedPartitions.retainAll(lostPartitions); @@ -127,46 +125,4 @@ Exception invokePartitionsLost(final SortedSet lostPartitions) { return null; } - - public void rebalance(final SortedSet revokedPartitions, - final SortedSet assignedPartitions) { - final AtomicReference firstException = new AtomicReference<>(null); - - if (!revokedPartitions.isEmpty()) { - // Revoke partitions that were previously owned but no longer assigned; - // note that we should only change the assignment (or update the assignor's state) - // AFTER we've triggered the revoke callback - firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); - } - - if (!assignedPartitions.isEmpty()) { - // Add partitions that were not previously owned but are now assigned - firstException.compareAndSet(null, invokePartitionsAssigned(assignedPartitions)); - } - - if (firstException.get() == null) - return; - - if (firstException.get() instanceof KafkaException) - throw (KafkaException) firstException.get(); - else - throw new KafkaException("User rebalance callback throws an error", firstException.get()); - } - - public void lose(final SortedSet lostPartitions) { - if (lostPartitions.isEmpty()) - return; - - log.info("Giving away all assigned partitions as lost since generation/memberID has been reset, " + - "indicating that consumer is in old state or no longer part of the group"); - Exception e = invokePartitionsLost(lostPartitions); - - if (e == null) - return; - - if (e instanceof KafkaException) - throw (KafkaException) e; - else - throw new KafkaException("User rebalance callback throws an error", e); - } } 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 0390b48122767..d2d272f86f06e 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 @@ -181,6 +181,10 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request)); } + public MembershipManager membershipManager() { + return membershipManager; + } + private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() { // TODO: extract this logic for building the ConsumerGroupHeartbeatRequestData to a // stateful builder (HeartbeatState), that will keep the last data sent, and determine 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 4727daa0f64b8..9882d3d77235f 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 @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; @@ -137,4 +140,17 @@ public interface MembershipManager { * Note that list of topics of the subscription is taken from the shared subscription state. */ void onSubscriptionUpdated(); + + /** + * Signals that a {@link ConsumerRebalanceListener} callback has completed. This is invoked when the + * application thread has completed the callback and has submitted a + * {@link ConsumerRebalanceListenerCallbackCompletedEvent} to the network I/O thread. At this point, we + * notify the state machine that it's complete so that it can move to the next appropriate step of the + * rebalance process. + * + * @param callbackName Method name of the callback that was executed + * @param error Optional error that was thrown by the callback, captured, and forwarded here + */ + void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackName callbackName, + Optional error); } 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 4a98d23a62fe6..bbba9769f20ee 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 @@ -21,6 +21,10 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.Utils.TopicIdPartitionComparator; import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; +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.ConsumerRebalanceListenerCallbackNeededEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.KafkaException; @@ -41,6 +45,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedSet; @@ -101,6 +106,23 @@ */ public class MembershipManagerImpl implements MembershipManager, ClusterResourceListener { + /** + * Used during the rebalancing process to keep track of the {@link ConsumerRebalanceListener} callback + * that is invoked on the application thread. The callback process is "complicated" because it requires + * jumping between the application and the network I/O threads. + */ + private static class ConsumerRebalanceListenerCallbackBreadcrumb { + + private final ConsumerRebalanceListenerCallbackName callbackName; + private final CompletableFuture future; + + public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerCallbackName callbackName, + CompletableFuture future) { + this.callbackName = Objects.requireNonNull(callbackName); + this.future = Objects.requireNonNull(future); + } + } + /** * TopicPartition comparator based on topic name and partition id. */ @@ -237,13 +259,27 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private boolean isRegisteredForMetadataUpdates; + /** + * Serves as the conduit by which we can report events to the application thread. This is needed as we send + * {@link ConsumerRebalanceListenerCallbackNeededEvent callbacks} and, if needed, + * {@link ErrorBackgroundEvent errors} to the application thread. + */ + private final BackgroundEventHandler backgroundEventHandler; + + /** + * Breadcrumb that we can return to as we wait for the completion of the + * {@link ConsumerRebalanceListenerCallbackNeededEvent} that was enqueued during rebalancing. + */ + private ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb; + public MembershipManagerImpl(String groupId, SubscriptionState subscriptions, CommitRequestManager commitRequestManager, ConsumerMetadata metadata, - LogContext logContext) { + LogContext logContext, + BackgroundEventHandler backgroundEventHandler) { this(groupId, Optional.empty(), Optional.empty(), subscriptions, commitRequestManager, metadata, - logContext); + logContext, backgroundEventHandler); } public MembershipManagerImpl(String groupId, @@ -252,7 +288,8 @@ public MembershipManagerImpl(String groupId, SubscriptionState subscriptions, CommitRequestManager commitRequestManager, ConsumerMetadata metadata, - LogContext logContext) { + LogContext logContext, + BackgroundEventHandler backgroundEventHandler) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -265,6 +302,7 @@ public MembershipManagerImpl(String groupId, this.assignmentReadyToReconcile = new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR); this.currentAssignment = new HashSet<>(); this.log = logContext.logger(MembershipManagerImpl.class); + this.backgroundEventHandler = backgroundEventHandler; } /** @@ -957,7 +995,7 @@ private CompletableFuture invokeOnPartitionsRevokedCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsRevoked.isEmpty() && listener.isPresent()) { - throw new UnsupportedOperationException("User-defined callbacks not supported yet"); + return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsRevoked, partitionsRevoked); } else { return CompletableFuture.completedFuture(null); } @@ -968,7 +1006,7 @@ private CompletableFuture invokeOnPartitionsAssignedCallback(Set listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - throw new UnsupportedOperationException("User-defined callbacks not supported yet"); + return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsAssigned, partitionsAssigned); } else { return CompletableFuture.completedFuture(null); } @@ -979,12 +1017,101 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsLost.isEmpty() && listener.isPresent()) { - throw new UnsupportedOperationException("User-defined callbacks not supported yet"); + return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsLost, partitionsLost); } else { return CompletableFuture.completedFuture(null); } } + private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName callbackName, + Set partitions) { + String fullCallbackName = String.format( + "%s.%s", + ConsumerRebalanceListener.class.getSimpleName(), + callbackName + ); + + CompletableFuture future = new CompletableFuture<>(); + ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( + callbackName, + future + ); + + if (breadcrumb == null) { + // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event + // without hesitation. + breadcrumb = newBreadcrumb; + SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + sortedPartitions.addAll(partitions); + BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(callbackName, sortedPartitions); + backgroundEventHandler.add(event); + log.debug("The event to trigger the {} method execution was enqueued successfully", fullCallbackName); + } else { + // In this case, there was an existing breadcrumb, so we need to report the matter back to the user. + String s = "An internal error occurred; an attempt to schedule the " + + fullCallbackName + " method for execution during rebalancing failed because " + + breadcrumb.callbackName + " was already scheduled"; + future.completeExceptionally(new KafkaException(s)); + } + + return future; + } + + @Override + public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackName callbackName, + Optional error) { + String fullCallbackName = String.format( + "%s.%s", + ConsumerRebalanceListener.class.getSimpleName(), + callbackName + ); + + if (breadcrumb != null && breadcrumb.callbackName == callbackName) { + // We have a breadcrumb that matches the callback we expect, so we can proceed to the next step of + // the rebalance process. + CompletableFuture future = breadcrumb.future; + + // We need to clear out our breadcrumb to signal that we've completed this step of the rebalance. + breadcrumb = null; + + if (error.isPresent()) { + KafkaException callbackError = error.get(); + + log.warn( + "The {} method completed with an error; signaling to continue to the next phase of rebalance", + fullCallbackName, + callbackError + ); + + future.completeExceptionally(callbackError); + } else { + log.debug( + "The {} method completed successfully; signaling to continue to the next phase of rebalance", + fullCallbackName + ); + + future.complete(null); + } + } else if (breadcrumb != null) { + // We have a breadcrumb that implicitly does NOT match the one we expect. We need to abort the + // rebalance process, because we're in an inconsistent state. We do that by completing the Future + // with an error. + String s = "An internal error occurred; an attempt to continue rebalance after the execution of the " + + fullCallbackName + " method failed because the expected method was " + breadcrumb.callbackName; + CompletableFuture future = breadcrumb.future; + breadcrumb = null; + future.completeExceptionally(new KafkaException(s)); + } else { + // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. + // Because of that, we don't have a Future that can be completed, so we're left having to report it + // back to the user asynchronously. + String s = "An internal error occurred; the " + fullCallbackName + " method was executed " + + "during rebalancing, but there was no record of it being scheduled"; + BackgroundEvent event = new ErrorBackgroundEvent(new KafkaException(s)); + backgroundEventHandler.add(event); + } + } + /** * Log partitions being revoked that were already paused, since the pause flag will be * effectively lost. 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 7cf6711909c12..3e0d0547397cb 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 @@ -49,7 +49,6 @@ public class RequestManagers implements Closeable { public final Optional coordinatorRequestManager; public final Optional commitRequestManager; public final Optional heartbeatRequestManager; - public final Optional membershipManager; public final OffsetsRequestManager offsetsRequestManager; public final TopicMetadataRequestManager topicMetadataRequestManager; public final FetchRequestManager fetchRequestManager; @@ -62,8 +61,7 @@ public RequestManagers(LogContext logContext, FetchRequestManager fetchRequestManager, Optional coordinatorRequestManager, Optional commitRequestManager, - Optional heartbeatRequestManager, - Optional membershipManager) { + Optional heartbeatRequestManager) { this.log = logContext.logger(RequestManagers.class); this.offsetsRequestManager = requireNonNull(offsetsRequestManager, "OffsetsRequestManager cannot be null"); this.coordinatorRequestManager = coordinatorRequestManager; @@ -71,7 +69,6 @@ public RequestManagers(LogContext logContext, this.topicMetadataRequestManager = topicMetadataRequestManager; this.fetchRequestManager = fetchRequestManager; this.heartbeatRequestManager = heartbeatRequestManager; - this.membershipManager = membershipManager; List> list = new ArrayList<>(); list.add(coordinatorRequestManager); @@ -170,7 +167,8 @@ protected RequestManagers create() { subscriptions, commit, metadata, - logContext); + logContext, + backgroundEventHandler); heartbeatRequestManager = new HeartbeatRequestManager( logContext, time, @@ -188,8 +186,7 @@ protected RequestManagers create() { fetch, Optional.ofNullable(coordinator), Optional.ofNullable(commit), - Optional.ofNullable(heartbeatRequestManager), - Optional.ofNullable(membershipManager) + Optional.ofNullable(heartbeatRequestManager) ); } }; 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 09e71d396a9e1..c896e57bf1f49 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 @@ -26,7 +26,7 @@ public abstract class ApplicationEvent { public enum Type { COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, - UNSUBSCRIBE, PARTITION_RECONCILIATION_COMPLETE, PARTITION_LOST_COMPLETE + UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED } private final Type type; 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 139cf357ab082..1b454afd6b98b 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,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -104,14 +105,6 @@ public void process(ApplicationEvent event) { processValidatePositionsEvent(); return; - case PARTITION_RECONCILIATION_COMPLETE: - process((RebalanceCompleteEvent) event); - return; - - case PARTITION_LOST_COMPLETE: - process((PartitionLostCompleteEvent) event); - return; - case SUBSCRIPTION_CHANGE: processSubscriptionChangeEvent(); return; @@ -120,6 +113,10 @@ public void process(ApplicationEvent event) { processUnsubscribeEvent((UnsubscribeApplicationEvent) event); return; + case CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED: + process((ConsumerRebalanceListenerCallbackCompletedEvent) event); + return; + default: log.warn("Application event type " + event.type() + " was not expected"); } @@ -188,11 +185,11 @@ private void process(final ListOffsetsApplicationEvent event) { * it is already a member. */ private void processSubscriptionChangeEvent() { - if (!requestManagers.membershipManager.isPresent()) { + if (!requestManagers.heartbeatRequestManager.isPresent()) { throw new RuntimeException("Group membership manager not present when processing a " + "subscribe event"); } - MembershipManager membershipManager = requestManagers.membershipManager.get(); + MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); membershipManager.onSubscriptionUpdated(); } @@ -205,11 +202,11 @@ private void processSubscriptionChangeEvent() { * the group is sent out. */ private void processUnsubscribeEvent(UnsubscribeApplicationEvent event) { - if (!requestManagers.membershipManager.isPresent()) { + if (!requestManagers.heartbeatRequestManager.isPresent()) { throw new RuntimeException("Group membership manager not present when processing an " + "unsubscribe event"); } - MembershipManager membershipManager = requestManagers.membershipManager.get(); + MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); CompletableFuture result = membershipManager.leaveGroup(); event.chain(result); } @@ -228,14 +225,17 @@ private void process(final TopicMetadataApplicationEvent event) { event.chain(future); } - private void process(final RebalanceCompleteEvent event) { - // TODO: with this event, we need to signal to the consumer group protocol state machine that the - // rebalance it'd started has completed... - } - - private void process(final PartitionLostCompleteEvent event) { - // TODO: with this event, we need to signal to the consumer group protocol state machine that the - // partitions have been "lost"... + private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { + if (requestManagers.heartbeatRequestManager.isPresent()) { + MembershipManager manager = requestManagers.heartbeatRequestManager.get().membershipManager(); + manager.consumerRebalanceListenerCallbackCompleted(event.callbackName(), event.error()); + } else { + log.warn( + "An internal error occurred; the group membership manager was not present, so the notification of the {}.{} callback's execution could not be sent", + ConsumerRebalanceListener.class.getSimpleName(), + event.callbackName() + ); + } } /** 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 13f2cc4db5bd2..aa21b6e6cc6cd 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 @@ -26,7 +26,7 @@ public abstract class BackgroundEvent { public enum Type { - ERROR, PARTITION_RECONCILIATION_STARTED, PARTITION_LOST_STARTED + ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED } private final Type type; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index 1bf68d1a51ac9..e39622080684b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerInvoker; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; @@ -87,12 +88,8 @@ public void process(final BackgroundEvent event) { process((ErrorBackgroundEvent) event); return; - case PARTITION_RECONCILIATION_STARTED: - process((RebalanceStartedEvent) event); - return; - - case PARTITION_LOST_STARTED: - process((PartitionLostStartedEvent) event); + case CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED: + process((ConsumerRebalanceListenerCallbackNeededEvent) event); return; default: @@ -109,37 +106,40 @@ private void process(final ErrorBackgroundEvent event) { throw event.error(); } - private void process(final RebalanceStartedEvent event) { - final SortedSet revokedPartitions = event.revokedPartitions(); - final SortedSet assignedPartitions = event.assignedPartitions(); - Optional error = Optional.empty(); - - try { - rebalanceListenerInvoker.rebalance(revokedPartitions, assignedPartitions); - } catch (KafkaException e) { - error = Optional.of(e); - throw e; - } finally { - ApplicationEvent invokedEvent = new RebalanceCompleteEvent( - revokedPartitions, - assignedPartitions, - error); - applicationEventHandler.add(invokedEvent); + private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { + SortedSet partitions = event.partitions(); + ConsumerRebalanceListenerCallbackName callbackName = event.callbackName(); + final Exception e; + + switch (callbackName) { + case onPartitionsRevoked: + e = rebalanceListenerInvoker.invokePartitionsRevoked(partitions); + break; + + case onPartitionsAssigned: + e = rebalanceListenerInvoker.invokePartitionsAssigned(partitions); + break; + + case onPartitionsLost: + e = rebalanceListenerInvoker.invokePartitionsLost(partitions); + break; + + default: + throw new IllegalArgumentException("Could not determine the " + ConsumerRebalanceListener.class.getSimpleName() + " to invoke from the callback name " + callbackName); } - } - private void process(final PartitionLostStartedEvent event) { - final SortedSet lostPartitions = event.lostPartitions(); - Optional error = Optional.empty(); - - try { - rebalanceListenerInvoker.lose(lostPartitions); - } catch (KafkaException e) { - error = Optional.of(e); - throw e; - } finally { - ApplicationEvent invokedEvent = new PartitionLostCompleteEvent(lostPartitions, error); - applicationEventHandler.add(invokedEvent); + final Optional error; + + if (e != null) { + if (e instanceof KafkaException) + error = Optional.of((KafkaException) e); + else + error = Optional.of(new KafkaException("User rebalance callback throws an error", e)); + } else { + error = Optional.empty(); } + + ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent(callbackName, partitions, error); + applicationEventHandler.add(invokedEvent); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java similarity index 51% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index 65f6af7a02faf..ee8920dce4443 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceCompleteEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -17,41 +17,41 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.KafkaException; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.KafkaException; import java.util.Collection; import java.util.Collections; +import java.util.Objects; import java.util.Optional; import java.util.SortedSet; /** - * Event that signifies that the application thread has executed the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and - * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks. If either callback execution threw - * an error, the first error encountered is included in the event should any event listener want to know. + * 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 RebalanceCompleteEvent extends ApplicationEvent { +public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent { - private final SortedSet revokedPartitions; - private final SortedSet assignedPartitions; + private final ConsumerRebalanceListenerCallbackName callbackName; + private final SortedSet partitions; private final Optional error; - public RebalanceCompleteEvent(SortedSet revokedPartitions, - SortedSet assignedPartitions, - Optional error) { - super(Type.PARTITION_RECONCILIATION_COMPLETE); - this.revokedPartitions = Collections.unmodifiableSortedSet(revokedPartitions); - this.assignedPartitions = Collections.unmodifiableSortedSet(assignedPartitions); - this.error = error; + public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerCallbackName callbackName, + SortedSet partitions, + Optional error) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); + this.callbackName = Objects.requireNonNull(callbackName); + this.partitions = Collections.unmodifiableSortedSet(partitions); + this.error = Objects.requireNonNull(error); } - public SortedSet revokedPartitions() { - return revokedPartitions; + public ConsumerRebalanceListenerCallbackName callbackName() { + return callbackName; } - public SortedSet assignedPartitions() { - return assignedPartitions; + public SortedSet partitions() { + return partitions; } public Optional error() { @@ -64,27 +64,21 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; if (!super.equals(o)) return false; - RebalanceCompleteEvent that = (RebalanceCompleteEvent) o; + ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - return revokedPartitions.equals(that.revokedPartitions) && - assignedPartitions.equals(that.assignedPartitions) && - error.equals(that.error); + return callbackName == that.callbackName && partitions.equals(that.partitions) && error.equals(that.error); } @Override public int hashCode() { - int result = super.hashCode(); - result = 31 * result + revokedPartitions.hashCode(); - result = 31 * result + assignedPartitions.hashCode(); - result = 31 * result + error.hashCode(); - return result; + return Objects.hash(callbackName, partitions, error); } @Override protected String toStringBase() { return super.toStringBase() + - ", revokedPartitions=" + revokedPartitions + - ", assignedPartitions=" + assignedPartitions + + ", callbackName=" + callbackName + + ", partitions=" + partitions + ", error=" + error; } 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 new file mode 100644 index 0000000000000..7a1983a57a23e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -0,0 +1,84 @@ +/* + * 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.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; +import org.apache.kafka.common.TopicPartition; + +import java.time.Duration; +import java.util.Collections; +import java.util.Objects; +import java.util.SortedSet; + +/** + * 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 + * {@link Consumer#poll(Duration)} call is performed by the user. When processed, the application thread should + * invoke the appropriate callback method (based on {@link #callbackName()}) with the given partitions. + */ +public class ConsumerRebalanceListenerCallbackNeededEvent extends BackgroundEvent { + + private final ConsumerRebalanceListenerCallbackName callbackName; + private final SortedSet partitions; + + public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerCallbackName callbackName, + SortedSet partitions) { + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); + this.callbackName = Objects.requireNonNull(callbackName); + this.partitions = Collections.unmodifiableSortedSet(partitions); + } + + public ConsumerRebalanceListenerCallbackName callbackName() { + return callbackName; + } + + 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 callbackName == that.callbackName && partitions.equals(that.partitions); + } + + @Override + public int hashCode() { + return Objects.hash(callbackName, partitions); + } + + @Override + protected String toStringBase() { + return super.toStringBase() + + ", callbackName=" + callbackName + + ", partitions=" + partitions; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + toStringBase() + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java deleted file mode 100644 index a087cce8151f0..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostCompleteEvent.java +++ /dev/null @@ -1,82 +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.ConsumerRebalanceListener; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.KafkaException; - -import java.util.Collection; -import java.util.Collections; -import java.util.Optional; -import java.util.SortedSet; - -/** - * Event that signifies that the application thread has executed the - * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback. If the callback execution threw an error, - * it is included in the event should any event listener want to know. - */ -public class PartitionLostCompleteEvent extends ApplicationEvent { - - private final SortedSet lostPartitions; - private final Optional error; - - public PartitionLostCompleteEvent(SortedSet lostPartitions, Optional error) { - super(Type.PARTITION_LOST_COMPLETE); - this.lostPartitions = Collections.unmodifiableSortedSet(lostPartitions); - this.error = error; - } - - public SortedSet lostPartitions() { - return lostPartitions; - } - - 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; - - PartitionLostCompleteEvent that = (PartitionLostCompleteEvent) o; - - return lostPartitions.equals(that.lostPartitions) && error.equals(that.error); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + lostPartitions.hashCode(); - result = 31 * result + error.hashCode(); - return result; - } - - @Override - protected String toStringBase() { - return super.toStringBase() + ", lostPartitions=" + lostPartitions + ", error=" + error; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java deleted file mode 100644 index ecad0fa7aaa57..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PartitionLostStartedEvent.java +++ /dev/null @@ -1,76 +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.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.TopicPartition; - -import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.SortedSet; - -/** - * Event that signifies that the background thread has determined that the member should abandon its partition - * assignment. This event will be processed by the application thread when the next {@link Consumer#poll(Duration)} - * call is performed by the user. When processed, the application thread should invoke the - * {@link ConsumerRebalanceListener#onPartitionsLost(Collection)} callback with the given partitions. - */ -public class PartitionLostStartedEvent extends BackgroundEvent { - - private final SortedSet lostPartitions; - - public PartitionLostStartedEvent(SortedSet lostPartitions) { - super(Type.PARTITION_LOST_STARTED); - this.lostPartitions = Collections.unmodifiableSortedSet(lostPartitions); - } - - public SortedSet lostPartitions() { - return lostPartitions; - } - - @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; - - PartitionLostStartedEvent that = (PartitionLostStartedEvent) o; - - return lostPartitions.equals(that.lostPartitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + lostPartitions.hashCode(); - return result; - } - - @Override - protected String toStringBase() { - return super.toStringBase() + ", lostPartitions=" + lostPartitions; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java deleted file mode 100644 index bf0ae25b9c506..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/RebalanceStartedEvent.java +++ /dev/null @@ -1,88 +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.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.common.TopicPartition; - -import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.SortedSet; - -/** - * Event that signifies that the background thread has started the partition assignment process. This event will - * be processed by the application thread when the next {@link Consumer#poll(Duration)} call is performed by the - * user. When processed, the application thread should invoke both the - * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} and - * {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} callbacks with the given partitions. - */ -public class RebalanceStartedEvent extends BackgroundEvent { - - private final SortedSet revokedPartitions; - private final SortedSet assignedPartitions; - - public RebalanceStartedEvent(SortedSet revokedPartitions, - SortedSet assignedPartitions) { - super(Type.PARTITION_RECONCILIATION_STARTED); - this.revokedPartitions = Collections.unmodifiableSortedSet(revokedPartitions); - this.assignedPartitions = Collections.unmodifiableSortedSet(assignedPartitions); - } - - public SortedSet revokedPartitions() { - return revokedPartitions; - } - - public SortedSet assignedPartitions() { - return assignedPartitions; - } - - @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; - - RebalanceStartedEvent that = (RebalanceStartedEvent) o; - - return revokedPartitions.equals(that.revokedPartitions) && - assignedPartitions.equals(that.assignedPartitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + revokedPartitions.hashCode(); - result = 31 * result + assignedPartitions.hashCode(); - return result; - } - - @Override - protected String toStringBase() { - return super.toStringBase() + - ", revokedPartitions=" + revokedPartitions + - ", assignedPartitions=" + assignedPartitions; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } -} 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..9a903e5c53223 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 @@ -22,7 +22,7 @@ * calls the subscribe API. This will make the consumer join a consumer group if not part of it * yet, or just send the updated subscription to the broker if it's already a member of the group. */ -public class SubscriptionChangeApplicationEvent extends ApplicationEvent { +public class SubscriptionChangeApplicationEvent extends CompletableApplicationEvent { public SubscriptionChangeApplicationEvent() { super(Type.SUBSCRIPTION_CHANGE); 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 77ca2a1f1c5eb..522f6399a28da 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 @@ -193,7 +193,8 @@ public ConsumerTestBuilder(Optional groupInfo) { subscriptions, commit, metadata, - logContext + logContext, + backgroundEventHandler ) ); HeartbeatRequestManager.HeartbeatRequestState state = spy(new HeartbeatRequestManager.HeartbeatRequestState(logContext, @@ -240,8 +241,7 @@ public ConsumerTestBuilder(Optional groupInfo) { fetchRequestManager, coordinatorRequestManager, commitRequestManager, - heartbeatRequestManager, - membershipManager); + heartbeatRequestManager); this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, applicationEventQueue, 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 8d3f61ea26ce2..971fb89eafb35 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 @@ -86,7 +86,7 @@ public void tearDown() { private MembershipManagerImpl createMembershipManagerJoiningGroup() { MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, subscriptionState, commitRequestManager, - metadata, testBuilder.logContext)); + metadata, testBuilder.logContext, testBuilder.backgroundEventHandler)); manager.transitionToJoining(); return manager; } @@ -95,7 +95,8 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn String serverAssignor) { MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(serverAssignor), - subscriptionState, commitRequestManager, metadata, testBuilder.logContext); + subscriptionState, commitRequestManager, metadata, testBuilder.logContext, + testBuilder.backgroundEventHandler); manager.transitionToJoining(); return manager; } @@ -120,7 +121,7 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() // First join should register to get metadata updates MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, subscriptionState, commitRequestManager, - metadata, testBuilder.logContext); + metadata, testBuilder.logContext, testBuilder.backgroundEventHandler); manager.transitionToJoining(); verify(metadata).addClusterUpdateListener(manager); clearInvocations(metadata); @@ -198,7 +199,7 @@ public void testTransitionToFatal() { public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( GROUP_ID, subscriptionState, commitRequestManager, metadata, - testBuilder.logContext); + testBuilder.logContext, testBuilder.backgroundEventHandler); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); From 6428ba63ee01c0953bc4081e1d2a3fd992f9824c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 27 Nov 2023 16:25:14 -0800 Subject: [PATCH 22/68] Changed ConsumerRebalanceListenerCallbackName to ConsumerRebalanceListenerMethodName --- ... ConsumerRebalanceListenerMethodName.java} | 2 +- .../consumer/internals/MembershipManager.java | 4 +- .../internals/MembershipManagerImpl.java | 57 ++++++++++--------- .../events/ApplicationEventProcessor.java | 4 +- .../events/BackgroundEventProcessor.java | 10 ++-- ...balanceListenerCallbackCompletedEvent.java | 19 +++---- ...rRebalanceListenerCallbackNeededEvent.java | 20 +++---- 7 files changed, 60 insertions(+), 56 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{ConsumerRebalanceListenerCallbackName.java => ConsumerRebalanceListenerMethodName.java} (94%) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java similarity index 94% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java index 9652ac9f4c10d..3d99b134ca0a6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerCallbackName.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java @@ -16,6 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -public enum ConsumerRebalanceListenerCallbackName { +public enum ConsumerRebalanceListenerMethodName { onPartitionsRevoked, onPartitionsAssigned, onPartitionsLost; } 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 9882d3d77235f..c44405dcf56f4 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 @@ -148,9 +148,9 @@ public interface MembershipManager { * notify the state machine that it's complete so that it can move to the next appropriate step of the * rebalance process. * - * @param callbackName Method name of the callback that was executed + * @param methodName Method name of the callback that was executed * @param error Optional error that was thrown by the callback, captured, and forwarded here */ - void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackName callbackName, + void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error); } 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 bbba9769f20ee..dcefebd20280c 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 @@ -113,12 +113,12 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private static class ConsumerRebalanceListenerCallbackBreadcrumb { - private final ConsumerRebalanceListenerCallbackName callbackName; + private final ConsumerRebalanceListenerMethodName methodName; private final CompletableFuture future; - public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerCallbackName callbackName, + public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMethodName methodName, CompletableFuture future) { - this.callbackName = Objects.requireNonNull(callbackName); + this.methodName = Objects.requireNonNull(methodName); this.future = Objects.requireNonNull(future); } } @@ -995,7 +995,7 @@ private CompletableFuture invokeOnPartitionsRevokedCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsRevoked.isEmpty() && listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsRevoked, partitionsRevoked); + return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsRevoked, partitionsRevoked); } else { return CompletableFuture.completedFuture(null); } @@ -1006,7 +1006,7 @@ private CompletableFuture invokeOnPartitionsAssignedCallback(Set listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsAssigned, partitionsAssigned); + return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsAssigned, partitionsAssigned); } else { return CompletableFuture.completedFuture(null); } @@ -1017,23 +1017,23 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsLost.isEmpty() && listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName.onPartitionsLost, partitionsLost); + return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsLost, partitionsLost); } else { return CompletableFuture.completedFuture(null); } } - private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerCallbackName callbackName, + private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerMethodName methodName, Set partitions) { - String fullCallbackName = String.format( + String fullMethodName = String.format( "%s.%s", ConsumerRebalanceListener.class.getSimpleName(), - callbackName + methodName ); CompletableFuture future = new CompletableFuture<>(); ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( - callbackName, + methodName, future ); @@ -1043,14 +1043,17 @@ private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerCa breadcrumb = newBreadcrumb; SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); - BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(callbackName, sortedPartitions); + BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", fullCallbackName); + log.debug("The event to trigger the {} method execution was enqueued successfully", fullMethodName); + if (true) + throw new IllegalStateException(); + } else { // In this case, there was an existing breadcrumb, so we need to report the matter back to the user. String s = "An internal error occurred; an attempt to schedule the " + - fullCallbackName + " method for execution during rebalancing failed because " + - breadcrumb.callbackName + " was already scheduled"; + fullMethodName + " method for execution during rebalancing failed because " + + breadcrumb.methodName + " was already scheduled"; future.completeExceptionally(new KafkaException(s)); } @@ -1058,15 +1061,15 @@ private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerCa } @Override - public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackName callbackName, + public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error) { - String fullCallbackName = String.format( - "%s.%s", - ConsumerRebalanceListener.class.getSimpleName(), - callbackName + String fullMethodName = String.format( + "%s.%s", + ConsumerRebalanceListener.class.getSimpleName(), + methodName ); - if (breadcrumb != null && breadcrumb.callbackName == callbackName) { + if (breadcrumb != null && breadcrumb.methodName == methodName) { // We have a breadcrumb that matches the callback we expect, so we can proceed to the next step of // the rebalance process. CompletableFuture future = breadcrumb.future; @@ -1079,15 +1082,18 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener log.warn( "The {} method completed with an error; signaling to continue to the next phase of rebalance", - fullCallbackName, + fullMethodName, callbackError ); future.completeExceptionally(callbackError); } else { + if (true) + throw new IllegalStateException(); + log.debug( "The {} method completed successfully; signaling to continue to the next phase of rebalance", - fullCallbackName + fullMethodName ); future.complete(null); @@ -1097,7 +1103,7 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener // rebalance process, because we're in an inconsistent state. We do that by completing the Future // with an error. String s = "An internal error occurred; an attempt to continue rebalance after the execution of the " + - fullCallbackName + " method failed because the expected method was " + breadcrumb.callbackName; + fullMethodName + " method failed because the expected method was " + breadcrumb.methodName; CompletableFuture future = breadcrumb.future; breadcrumb = null; future.completeExceptionally(new KafkaException(s)); @@ -1105,10 +1111,9 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. // Because of that, we don't have a Future that can be completed, so we're left having to report it // back to the user asynchronously. - String s = "An internal error occurred; the " + fullCallbackName + " method was executed " + + String s = "An internal error occurred; the " + fullMethodName + " method was executed " + "during rebalancing, but there was no record of it being scheduled"; - BackgroundEvent event = new ErrorBackgroundEvent(new KafkaException(s)); - backgroundEventHandler.add(event); + backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); } } 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 1b454afd6b98b..e13650c48b3b5 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 @@ -228,12 +228,12 @@ private void process(final TopicMetadataApplicationEvent event) { private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { if (requestManagers.heartbeatRequestManager.isPresent()) { MembershipManager manager = requestManagers.heartbeatRequestManager.get().membershipManager(); - manager.consumerRebalanceListenerCallbackCompleted(event.callbackName(), event.error()); + manager.consumerRebalanceListenerCallbackCompleted(event.methodName(), event.error()); } else { log.warn( "An internal error occurred; the group membership manager was not present, so the notification of the {}.{} callback's execution could not be sent", ConsumerRebalanceListener.class.getSimpleName(), - event.callbackName() + event.methodName() ); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index e39622080684b..3aad547d8a147 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerInvoker; -import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; @@ -108,10 +108,10 @@ private void process(final ErrorBackgroundEvent event) { private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { SortedSet partitions = event.partitions(); - ConsumerRebalanceListenerCallbackName callbackName = event.callbackName(); + ConsumerRebalanceListenerMethodName methodName = event.methodName(); final Exception e; - switch (callbackName) { + switch (methodName) { case onPartitionsRevoked: e = rebalanceListenerInvoker.invokePartitionsRevoked(partitions); break; @@ -125,7 +125,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { break; default: - throw new IllegalArgumentException("Could not determine the " + ConsumerRebalanceListener.class.getSimpleName() + " to invoke from the callback name " + callbackName); + throw new IllegalArgumentException("Could not determine the " + ConsumerRebalanceListener.class.getSimpleName() + " to invoke from the callback method " + methodName); } final Optional error; @@ -139,7 +139,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { error = Optional.empty(); } - ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent(callbackName, partitions, error); + ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent(methodName, partitions, error); applicationEventHandler.add(invokedEvent); } } 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 ee8920dce4443..7bd71b91b238d 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 @@ -17,11 +17,10 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.KafkaException; -import java.util.Collection; import java.util.Collections; import java.util.Objects; import java.util.Optional; @@ -33,21 +32,21 @@ */ public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent { - private final ConsumerRebalanceListenerCallbackName callbackName; + private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; private final Optional error; - public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerCallbackName callbackName, + public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, SortedSet partitions, Optional error) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); - this.callbackName = Objects.requireNonNull(callbackName); + this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); this.error = Objects.requireNonNull(error); } - public ConsumerRebalanceListenerCallbackName callbackName() { - return callbackName; + public ConsumerRebalanceListenerMethodName methodName() { + return methodName; } public SortedSet partitions() { @@ -66,18 +65,18 @@ public boolean equals(Object o) { ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - return callbackName == that.callbackName && partitions.equals(that.partitions) && error.equals(that.error); + return methodName == that.methodName && partitions.equals(that.partitions) && error.equals(that.error); } @Override public int hashCode() { - return Objects.hash(callbackName, partitions, error); + return Objects.hash(methodName, partitions, error); } @Override protected String toStringBase() { return super.toStringBase() + - ", callbackName=" + callbackName + + ", methodName=" + methodName + ", partitions=" + partitions + ", error=" + error; } 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 7a1983a57a23e..754f8e7c96399 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 @@ -18,7 +18,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerCallbackName; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName; import org.apache.kafka.common.TopicPartition; import java.time.Duration; @@ -30,22 +30,22 @@ * 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 * {@link Consumer#poll(Duration)} call is performed by the user. When processed, the application thread should - * invoke the appropriate callback method (based on {@link #callbackName()}) with the given partitions. + * invoke the appropriate callback method (based on {@link #methodName()}) with the given partitions. */ public class ConsumerRebalanceListenerCallbackNeededEvent extends BackgroundEvent { - private final ConsumerRebalanceListenerCallbackName callbackName; + private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; - public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerCallbackName callbackName, + public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, SortedSet partitions) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); - this.callbackName = Objects.requireNonNull(callbackName); + this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } - public ConsumerRebalanceListenerCallbackName callbackName() { - return callbackName; + public ConsumerRebalanceListenerMethodName methodName() { + return methodName; } public SortedSet partitions() { @@ -60,18 +60,18 @@ public boolean equals(Object o) { ConsumerRebalanceListenerCallbackNeededEvent that = (ConsumerRebalanceListenerCallbackNeededEvent) o; - return callbackName == that.callbackName && partitions.equals(that.partitions); + return methodName == that.methodName && partitions.equals(that.partitions); } @Override public int hashCode() { - return Objects.hash(callbackName, partitions); + return Objects.hash(methodName, partitions); } @Override protected String toStringBase() { return super.toStringBase() + - ", callbackName=" + callbackName + + ", methodName=" + methodName + ", partitions=" + partitions; } From 06d4897666258fd644d0d50bf6b2305def22d962 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 27 Nov 2023 16:25:32 -0800 Subject: [PATCH 23/68] Updated AsyncKafkaConsumer.unsubscribe() to block --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 7 +++++-- 1 file changed, 5 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 3f88354555e39..ab6bd1d18e71f 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 @@ -1064,9 +1064,12 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupId.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); try { - applicationEventHandler.addAndGet(unsubscribeApplicationEvent, time.timer(Duration.ofMillis(Long.MAX_VALUE))); + // The unsubscribe logic will issue the request to leave the group and then execute any callbacks + // that the user provided. The existing implementation (LegacyKafkaConsumer) blocks the application + // thread when it calls the user's callback. We will implement in kind to maintain compatibility. + Timer timer = time.timer(Duration.ofMillis(Long.MAX_VALUE)); + applicationEventHandler.addAndGet(new UnsubscribeApplicationEvent(), 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", e); From 993c040e35fe664ee661e6e9f6a7fc7795c15f3f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 28 Nov 2023 20:11:11 -0800 Subject: [PATCH 24/68] Updates to implement blocking for the consumer to join the group, if not already a member --- .../internals/AsyncKafkaConsumer.java | 75 ++++++++++-- .../ConsumerRebalanceListenerInvoker.java | 18 ++- .../consumer/internals/MembershipManager.java | 9 ++ .../internals/MembershipManagerImpl.java | 111 ++++++++++++------ .../consumer/internals/RequestManagers.java | 3 +- .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventProcessor.java | 39 ++++-- .../events/BackgroundEventProcessor.java | 7 +- .../internals/events/EventHandler.java | 2 +- .../internals/events/EventProcessor.java | 23 ++-- .../WaitForJoinGroupApplicationEvent.java | 32 +++++ .../internals/ConsumerTestBuilder.java | 1 + 12 files changed, 241 insertions(+), 81 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.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 fac7313203e92..831896ccfcc9f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -41,6 +41,7 @@ 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.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; @@ -50,6 +51,7 @@ import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.WaitForJoinGroupApplicationEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; @@ -212,6 +214,10 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { ApiVersions apiVersions = new ApiVersions(); final BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); final BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( + logContext, + backgroundEventQueue + ); // This FetchBuffer is shared between the application and network threads. this.fetchBuffer = new FetchBuffer(logContext); @@ -224,7 +230,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { fetchMetricsManager); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, - backgroundEventQueue, + backgroundEventHandler, metadata, subscriptions, fetchBuffer, @@ -236,6 +242,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, applicationEventQueue, + backgroundEventHandler, requestManagersSupplier); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, @@ -389,6 +396,10 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler( + logContext, + backgroundEventQueue + ); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( logContext, subscriptions, @@ -405,7 +416,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, - backgroundEventQueue, + backgroundEventHandler, metadata, subscriptions, fetchBuffer, @@ -419,6 +430,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { logContext, metadata, applicationEventQueue, + backgroundEventHandler, requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, @@ -1259,14 +1271,63 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd @Override public boolean updateAssignmentMetadataIfNeeded(Timer timer) { + if (isCommittedOffsetsManagementEnabled() && !coordinatorPoll(timer)) { + return false; + } + + return updateFetchPositions(timer); + } + + /** + * This method is an approximation of the logic that the {@link ConsumerCoordinator#poll(Timer, boolean)} method + * performs when it is called by the {@link LegacyKafkaConsumer#poll(Duration)} code. + * + *

+ * + * To mimic the behavior of the {@link ConsumerCoordinator#ensureActiveGroup(Timer)}, we want to block the + * application thread until the consumer fully joins the group. However, we can't block the application thread + * for the full length of the given {@link Timer timer} because we need the application thread to perform + * any {@link ConsumerRebalanceListener} callbacks that are invoked as part of the initial assignment. So we + * have a loop that executes the following until the timer expires: + * + *

    + *
  • Block for a 100 ms.
  • + *
  • Execute any callbacks
  • + *
+ * + * @param timer Timer that limits the time the method will wait to join the group + * @return {@code true} if the join was successful, {@code false} otherwise + */ + private boolean coordinatorPoll(Timer timer) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); - backgroundEventProcessor.process(); - // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as - // in the previous implementation, because it will eventually involve group coordination - // logic - return updateFetchPositions(timer); + WaitForJoinGroupApplicationEvent event = null; + + do { + backgroundEventProcessor.process(); + + try { + log.debug("Waiting for consumer to join group and transition to {} state", MemberState.STABLE); + + if (event == null) { + // Create and enqueue the event once, but repeatedly wait on the same Future until complete + // or the timer expires. + event = new WaitForJoinGroupApplicationEvent(); + applicationEventHandler.add(event); + } + + ConsumerUtils.getResult(event.future(), time.timer(100)); + + return true; + } catch (TimeoutException e) { + // Ignore this as we will retry the event until the timeout expires. + } finally { + timer.update(time.milliseconds()); + } + } while (timer.notExpired()); + + return false; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java index 3ac77e08dd965..48fea2e3d618d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java @@ -58,15 +58,17 @@ public Exception invokePartitionsAssigned(final SortedSet assign Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { + ConsumerRebalanceListener consumerRebalanceListener = listener.get(); + try { final long startMs = time.milliseconds(); - listener.get().onPartitionsAssigned(assignedPartitions); + consumerRebalanceListener.onPartitionsAssigned(assignedPartitions); coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}", - listener.getClass().getName(), assignedPartitions, e); + consumerRebalanceListener.getClass().getName(), assignedPartitions, e); return e; } } @@ -84,15 +86,17 @@ public Exception invokePartitionsRevoked(final SortedSet revoked Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { + ConsumerRebalanceListener consumerRebalanceListener = listener.get(); + try { final long startMs = time.milliseconds(); - listener.get().onPartitionsRevoked(revokedPartitions); + consumerRebalanceListener.onPartitionsRevoked(revokedPartitions); coordinatorMetrics.revokeCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}", - listener.getClass().getName(), revokedPartitions, e); + consumerRebalanceListener.getClass().getName(), revokedPartitions, e); return e; } } @@ -110,15 +114,17 @@ public Exception invokePartitionsLost(final SortedSet lostPartit Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { + ConsumerRebalanceListener consumerRebalanceListener = listener.get(); + try { final long startMs = time.milliseconds(); - listener.get().onPartitionsLost(lostPartitions); + consumerRebalanceListener.onPartitionsLost(lostPartitions); coordinatorMetrics.loseCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}", - listener.getClass().getName(), lostPartitions, e); + consumerRebalanceListener.getClass().getName(), lostPartitions, e); return e; } } 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 c44405dcf56f4..541ea7d7398a5 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 @@ -153,4 +153,13 @@ public interface MembershipManager { */ void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error); + + /** + * Allows the caller to block until the member has joined the consumer group in the + * {@link MemberState#STABLE stable} state. + * + * @param future Future that will be completed when the member has fully joined the consumer group + * (exceptionally or normally) + */ + void notifyOnStable(CompletableFuture future); } 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 b4136fd7fafd4..29050847ae221 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.utils.Utils; import org.slf4j.Logger; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -251,6 +252,13 @@ public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMeth */ private Optional> leaveGroupInProgress; + /** + * The Consumer may wish to wait until it's officially part of a {@link MemberState#STABLE stable} + * consumer group. Code that runs within the {@link ConsumerNetworkThread background thread} can elect + * to be notified when this occurs via the {@link #notifyOnStable(CompletableFuture)} method. + */ + private final List> notifyOnStableFutures = new ArrayList<>(); + /** * True if the member has registered to be notified when the cluster metadata is updated. * This is initially false, as the member that is not part of a consumer group does not @@ -316,7 +324,7 @@ private void transitionTo(MemberState nextState) { throw new IllegalStateException(String.format("Invalid state transition from %s to %s", state, nextState)); } - log.trace("Member {} transitioned from {} to {}.", memberId, state, nextState); + log.trace("Member {} transitioned from {} to {}.", memberIdForLogging(), state, nextState); this.state = nextState; } @@ -344,6 +352,10 @@ public String memberId() { return memberId; } + private String memberIdForLogging() { + return memberId != null && !memberId.trim().isEmpty() ? memberId : ""; + } + /** * {@inheritDoc} */ @@ -374,7 +386,28 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo resolveMetadataForUnresolvedAssignment(); reconcile(); } else if (allPendingAssignmentsReconciled()) { - transitionTo(MemberState.STABLE); + transitionToStable(); + } + } + + @Override + public void notifyOnStable(CompletableFuture future) { + if (state == MemberState.STABLE) { + future.complete(null); + log.debug( + "Completed future {} since member {} is already in the {} state", + future, + memberIdForLogging(), + MemberState.STABLE + ); + } else { + notifyOnStableFutures.add(future); + log.debug( + "Registered future {} for notification when member {} transitions to the {} state", + future, + memberIdForLogging(), + MemberState.STABLE + ); } } @@ -406,10 +439,10 @@ public void transitionToFenced() { transitionTo(MemberState.FENCED); resetEpoch(); log.debug("Member {} with epoch {} transitioned to {} state. It will release its " + - "assignment and rejoin the group.", memberId, memberEpoch, MemberState.FENCED); + "assignment and rejoin the group.", memberIdForLogging(), memberEpoch, MemberState.FENCED); // Release assignment - CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); + CompletableFuture callbackResult = enqueueOnPartitionsLostCallback(subscriptions.assignedPartitions()); callbackResult.whenComplete((result, error) -> { if (error != null) { log.error("onPartitionsLost callback invocation failed while releasing assignment" + @@ -426,10 +459,10 @@ public void transitionToFenced() { @Override public void transitionToFatal() { transitionTo(MemberState.FATAL); - log.error("Member {} with epoch {} transitioned to {} state", memberId, memberEpoch, MemberState.FATAL); + log.error("Member {} with epoch {} transitioned to {} state", memberIdForLogging(), memberEpoch, MemberState.FATAL); // Release assignment - CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); + CompletableFuture callbackResult = enqueueOnPartitionsLostCallback(subscriptions.assignedPartitions()); callbackResult.whenComplete((result, error) -> { if (error != null) { log.error("onPartitionsLost callback invocation failed while releasing assignment" + @@ -480,6 +513,20 @@ void transitionToJoining() { registerForMetadataUpdates(); } + private void transitionToStable() { + transitionTo(MemberState.STABLE); + notifyOnStableFutures.forEach(future -> { + future.complete(null); + log.debug( + "Completed future {} as member {} has transitioned to the {} state", + future, + memberIdForLogging(), + MemberState.STABLE + ); + }); + notifyOnStableFutures.clear(); + } + /** * Register to get notified when the cluster metadata is updated, via the * {@link #onUpdate(ClusterResource)}. Register only if the manager is not register already. @@ -512,7 +559,7 @@ public CompletableFuture leaveGroup() { CompletableFuture leaveResult = new CompletableFuture<>(); leaveGroupInProgress = Optional.of(leaveResult); - CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); + CompletableFuture callbackResult = enqueueOnPartitionsRevokedOrLostToReleaseAssignment(); callbackResult.whenComplete((result, error) -> { // Clear the subscription, no matter if the callback execution failed or succeeded. updateSubscription(Collections.emptySet(), true); @@ -542,7 +589,7 @@ public CompletableFuture leaveGroup() { * * @return Future that will complete when the callback execution completes. */ - private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignment() { + private CompletableFuture enqueueOnPartitionsRevokedOrLostToReleaseAssignment() { SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); droppedPartitions.addAll(subscriptions.assignedPartitions()); @@ -557,7 +604,7 @@ private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignme callbackResult = revokePartitions(droppedPartitions); } else { // Member is not part of the group anymore. Invoke onPartitionsLost. - callbackResult = invokeOnPartitionsLostCallback(droppedPartitions); + callbackResult = enqueueOnPartitionsLostCallback(droppedPartitions); } } return callbackResult; @@ -591,11 +638,11 @@ public void onHeartbeatRequestSent() { MemberState state = state(); if (state == MemberState.ACKNOWLEDGING) { if (allPendingAssignmentsReconciled()) { - transitionTo(MemberState.STABLE); + transitionToStable(); } else { log.debug("Member {} with epoch {} transitioned to {} after a heartbeat was sent " + "to ack a previous reconciliation. New assignments are ready to " + - "be reconciled.", memberId, memberEpoch, MemberState.RECONCILING); + "be reconciled.", memberIdForLogging(), memberEpoch, MemberState.RECONCILING); transitionTo(MemberState.RECONCILING); } } else if (state == MemberState.LEAVING) { @@ -610,7 +657,7 @@ public void onHeartbeatRequestSent() { public void onHeartbeatRequestSkipped() { if (state == MemberState.LEAVING) { log.debug("Heartbeat for leaving group could not be sent. Member {} with epoch {} will transition to {}.", - memberId, memberEpoch, MemberState.UNSUBSCRIBED); + memberIdForLogging(), memberEpoch, MemberState.UNSUBSCRIBED); transitionToUnsubscribed(); } } @@ -932,7 +979,7 @@ private CompletableFuture revokePartitions(Set revokedPart " proceed with the revocation anyway.", error); } - CompletableFuture userCallbackResult = invokeOnPartitionsRevokedCallback(revokedPartitions); + CompletableFuture userCallbackResult = enqueueOnPartitionsRevokedCallback(revokedPartitions); userCallbackResult.whenComplete((callbackResult, callbackError) -> { if (callbackError != null) { log.error("onPartitionsRevoked callback invocation failed for partitions {}", @@ -967,7 +1014,7 @@ private CompletableFuture assignPartitions( updateSubscription(assignedPartitions, false); // Invoke user call back - return invokeOnPartitionsAssignedCallback(addedPartitions); + return enqueueOnPartitionsAssignedCallback(addedPartitions); } /** @@ -990,7 +1037,7 @@ private void markPendingRevocationToPauseFetching(Set partitions subscriptions.markPendingRevocation(partitionsToRevoke); } - private CompletableFuture invokeOnPartitionsRevokedCallback(Set partitionsRevoked) { + private CompletableFuture enqueueOnPartitionsRevokedCallback(Set partitionsRevoked) { // This should not trigger the callback if partitionsRevoked is empty, to keep the // current behaviour. Optional listener = subscriptions.rebalanceListener(); @@ -1001,7 +1048,7 @@ private CompletableFuture invokeOnPartitionsRevokedCallback(Set invokeOnPartitionsAssignedCallback(Set partitionsAssigned) { + private CompletableFuture enqueueOnPartitionsAssignedCallback(Set partitionsAssigned) { // This should always trigger the callback, even if partitionsAssigned is empty, to keep // the current behaviour. Optional listener = subscriptions.rebalanceListener(); @@ -1012,7 +1059,7 @@ private CompletableFuture invokeOnPartitionsAssignedCallback(Set invokeOnPartitionsLostCallback(Set partitionsLost) { + private CompletableFuture enqueueOnPartitionsLostCallback(Set partitionsLost) { // This should not trigger the callback if partitionsLost is empty, to keep the current // behaviour. Optional listener = subscriptions.rebalanceListener(); @@ -1026,14 +1073,14 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set enqueueCallbackEvent(ConsumerRebalanceListenerMethodName methodName, Set partitions) { String fullMethodName = String.format( - "%s.%s", - ConsumerRebalanceListener.class.getSimpleName(), - methodName + "%s.%s", + ConsumerRebalanceListener.class.getSimpleName(), + methodName ); CompletableFuture future = new CompletableFuture<>(); ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( - methodName, + methodName, future ); @@ -1046,9 +1093,6 @@ private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerMe BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", fullMethodName); - if (true) - throw new IllegalStateException(); - } else { // In this case, there was an existing breadcrumb, so we need to report the matter back to the user. String s = "An internal error occurred; an attempt to schedule the " + @@ -1064,9 +1108,9 @@ private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerMe public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error) { String fullMethodName = String.format( - "%s.%s", - ConsumerRebalanceListener.class.getSimpleName(), - methodName + "%s.%s", + ConsumerRebalanceListener.class.getSimpleName(), + methodName ); if (breadcrumb != null && breadcrumb.methodName == methodName) { @@ -1078,26 +1122,19 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener breadcrumb = null; if (error.isPresent()) { - KafkaException callbackError = error.get(); - log.warn( "The {} method completed with an error; signaling to continue to the next phase of rebalance", fullMethodName, - callbackError + error.get() ); - - future.completeExceptionally(callbackError); } else { - if (true) - throw new IllegalStateException(); - log.debug( "The {} method completed successfully; signaling to continue to the next phase of rebalance", fullMethodName ); - - future.complete(null); } + + future.complete(null); } else if (breadcrumb != null) { // We have a breadcrumb that implicitly does NOT match the one we expect. We need to abort the // rebalance process, because we're in an inconsistent state. We do that by completing the 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 3a485422ee620..0cb817c9bd014 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 @@ -108,7 +108,7 @@ public void close() { */ public static Supplier supplier(final Time time, final LogContext logContext, - final BlockingQueue backgroundEventQueue, + final BackgroundEventHandler backgroundEventHandler, final ConsumerMetadata metadata, final SubscriptionState subscriptions, final FetchBuffer fetchBuffer, @@ -121,7 +121,6 @@ public static Supplier supplier(final Time time, @Override protected RequestManagers create() { final NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); - final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); 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); 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 c896e57bf1f49..c16d3153d3579 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 @@ -26,7 +26,7 @@ public abstract class ApplicationEvent { public enum Type { COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, - UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED + UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, WAIT_FOR_JOIN_GROUP } private final Type type; 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 e13650c48b3b5..2c91c970c4590 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 @@ -46,13 +46,16 @@ public class ApplicationEventProcessor extends EventProcessor private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; + private final BackgroundEventHandler backgroundEventHandler; public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, + final BackgroundEventHandler backgroundEventHandler, final RequestManagers requestManagers, final ConsumerMetadata metadata) { - super(logContext, applicationEventQueue); + super(new LogContext("[Application event processor]" + (logContext.logPrefix() != null ? " " + logContext.logPrefix() : "")), applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); + this.backgroundEventHandler = backgroundEventHandler; this.requestManagers = requestManagers; this.metadata = metadata; } @@ -117,16 +120,15 @@ public void process(ApplicationEvent event) { process((ConsumerRebalanceListenerCallbackCompletedEvent) event); return; + case WAIT_FOR_JOIN_GROUP: + process((WaitForJoinGroupApplicationEvent) event); + return; + default: log.warn("Application event type " + event.type() + " was not expected"); } } - @Override - protected Class getEventClass() { - return ApplicationEvent.class; - } - private void process(final PollApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; @@ -186,9 +188,11 @@ private void process(final ListOffsetsApplicationEvent event) { */ private void processSubscriptionChangeEvent() { if (!requestManagers.heartbeatRequestManager.isPresent()) { - throw new RuntimeException("Group membership manager not present when processing a " + - "subscribe event"); + KafkaException error = new KafkaException("Group membership manager not present when processing a subscribe event"); + backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + return; } + MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); membershipManager.onSubscriptionUpdated(); } @@ -203,9 +207,11 @@ private void processSubscriptionChangeEvent() { */ private void processUnsubscribeEvent(UnsubscribeApplicationEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { - throw new RuntimeException("Group membership manager not present when processing an " + - "unsubscribe event"); + KafkaException error = new KafkaException("Group membership manager not present when processing an unsubscribe event"); + backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + return; } + MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); CompletableFuture result = membershipManager.leaveGroup(); event.chain(result); @@ -238,6 +244,17 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event } } + private void process(final WaitForJoinGroupApplicationEvent event) { + if (!requestManagers.heartbeatRequestManager.isPresent()) { + KafkaException error = new KafkaException("Group membership manager not present when waiting to join a group"); + backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + return; + } + + MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); + membershipManager.notifyOnStable(event.future()); + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. @@ -245,6 +262,7 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, + final BackgroundEventHandler backgroundEventHandler, final Supplier requestManagersSupplier) { return new CachedSupplier() { @Override @@ -253,6 +271,7 @@ protected ApplicationEventProcessor create() { return new ApplicationEventProcessor( logContext, applicationEventQueue, + backgroundEventHandler, requestManagers, metadata ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index 80b2b9ec00584..1a0d82ca7378f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -50,7 +50,7 @@ public BackgroundEventProcessor(final LogContext logContext, final BlockingQueue backgroundEventQueue, final ApplicationEventHandler applicationEventHandler, final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { - super(logContext, backgroundEventQueue); + super(new LogContext("[Background event processor]" + (logContext.logPrefix() != null ? " " + logContext.logPrefix() : "")), backgroundEventQueue); this.log = logContext.logger(BackgroundEventProcessor.class); this.applicationEventHandler = applicationEventHandler; this.rebalanceListenerInvoker = rebalanceListenerInvoker; @@ -101,11 +101,6 @@ public void process(final BackgroundEvent event) { } } - @Override - protected Class getEventClass() { - return BackgroundEvent.class; - } - private void process(final ErrorBackgroundEvent event) { throw event.error(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java index 2c92f190c5118..c68f89f2324a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java @@ -46,8 +46,8 @@ public EventHandler(final LogContext logContext, final BlockingQueue queue) { */ public void add(final T event) { Objects.requireNonNull(event, "Event must be non-null"); - log.trace("Enqueued event: {}", event); queue.add(event); + log.trace("Enqueued event: {}", event); } @Override 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 83cd7bf60933d..0eb8b5b7a8832 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 @@ -54,8 +54,6 @@ public void close() { closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); } - protected abstract Class getEventClass(); - protected interface ProcessHandler { void onProcess(T event, Optional error); @@ -66,18 +64,22 @@ protected interface ProcessHandler { * processing the individual events, these are submitted to the given {@link ProcessHandler}. */ protected void process(ProcessHandler processHandler) { - String eventClassName = getEventClass().getSimpleName(); - closer.assertOpen(() -> String.format("The processor was previously closed, so no further %s processing can occur", eventClassName)); + 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; + } + try { - log.trace("Starting processing of {} {}(s)", events.size(), eventClassName); + log.trace("Starting processing of {} event{}", events.size(), events.size() == 1 ? "" : "s"); for (T event : events) { try { - Objects.requireNonNull(event, () -> String.format("Attempted to process a null %s", eventClassName)); - log.trace("Consuming {}: {}", eventClassName, event); + Objects.requireNonNull(event, "Attempted to process a null event"); + log.trace("Processing event: {}", event); process(event); processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { @@ -92,7 +94,7 @@ protected void process(ProcessHandler processHandler) { } } } finally { - log.debug("Completed processing of {} {}(s)", events.size(), eventClassName); + log.trace("Completed processing"); } } @@ -101,8 +103,7 @@ protected void process(ProcessHandler processHandler) { * this case, we need to throw an exception to notify the user the consumer is closed. */ private void closeInternal() { - String eventClassName = getEventClass().getSimpleName(); - log.trace("Closing event processor for {}", eventClassName); + log.trace("Closing event processor"); List incompleteEvents = drain(); if (incompleteEvents.isEmpty()) @@ -121,7 +122,7 @@ private void closeInternal() { f.completeExceptionally(exception); }); - log.debug("Discarding {} {}s because the consumer is closing", incompleteEvents.size(), eventClassName); + log.debug("Discarding {} events because the consumer is closing", incompleteEvents.size()); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java new file mode 100644 index 0000000000000..05fdf47ef73c7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java @@ -0,0 +1,32 @@ +/* + * 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.LegacyKafkaConsumer; + +/** + * This event serves as a mechanism to block the application thread until the consumer has joined the consumer group. + * It is used to mimic the blocking behavior of the {@link LegacyKafkaConsumer existing Consumer}. The application + * thread submits this event and then can block on the {@link #future()} to wait for the result. + */ +public class WaitForJoinGroupApplicationEvent extends CompletableApplicationEvent { + + public WaitForJoinGroupApplicationEvent() { + super(Type.WAIT_FOR_JOIN_GROUP); + } +} + 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 7500c53cd7a51..c52705ac9512a 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 @@ -246,6 +246,7 @@ public ConsumerTestBuilder(Optional groupInfo) { this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, applicationEventQueue, + backgroundEventHandler, requestManagers, metadata) ); From 5bdedaa7ab4d157698ca6fd82abe2ed0b232639c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 20:32:52 -0800 Subject: [PATCH 25/68] Update MembershipManagerImpl to print out partitions w/o topic IDs --- .../kafka/clients/consumer/internals/MembershipManagerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 29050847ae221..be7dbb882a256 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 @@ -735,7 +735,7 @@ boolean reconcile() { "\tCurrent owned partitions: {}\n" + "\tAdded partitions (assigned - owned): {}\n" + "\tRevoked partitions (owned - assigned): {}\n", - assignedTopicIdPartitions, + assignedTopicPartition, ownedPartitions, addedPartitions, revokedPartitions From 86adb9a267ca0c4079affab88fe5bf1164ebfbc7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 20:33:22 -0800 Subject: [PATCH 26/68] Updated ConsumerNetworkThread to block for new events (should be spun out into a separate PR) --- .../internals/ConsumerNetworkThread.java | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) 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 240746e52cae4..a317b16008745 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -39,6 +40,10 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -63,6 +68,9 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private RequestManagers requestManagers; private volatile boolean running; private final IdempotentCloser closer = new IdempotentCloser(); + private final Lock lock = new ReentrantLock(); + private final Condition wakeupCondition = lock.newCondition(); + private volatile Duration closeTimeout = Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS); public ConsumerNetworkThread(LogContext logContext, @@ -131,6 +139,8 @@ void initializeResources() { * */ void runOnce() { + waitForNewEvents(MAX_POLL_TIMEOUT_MS); + // 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. @@ -204,6 +214,8 @@ public void wakeup() { // The network client can be null if the initializeResources method has not yet been called. if (networkClientDelegate != null) networkClientDelegate.wakeup(); + + notifyOfNewEvents(); } @Override @@ -328,4 +340,33 @@ private void findCoordinatorSync(final Timer timer) { timer.update(); } } + + /** + * Used by the main {@link #runOnce()} method to wait for the arrival of new events to process OR for the + * timeout to expire. This prevents our thread from busy spinning when there aren't events process. + */ + private void waitForNewEvents(long timeoutMs) { + try { + lock.lock(); + + if (wakeupCondition.await(timeoutMs, TimeUnit.MILLISECONDS)) + log.trace("Consumer network I/O thread awoken prematurely"); + } catch (InterruptedException e) { + throw new InterruptException("Interrupted while waiting for new events", e); + } finally { + lock.unlock(); + } + } + + /** + * This is called from {@link #wakeup()} to notify our thread that there are new events to process. + */ + private void notifyOfNewEvents() { + try { + lock.lock(); + wakeupCondition.signalAll(); + } finally { + lock.unlock(); + } + } } From 110c7ebac78464910c16a7c1a34716a2d68856f2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 21:48:30 -0800 Subject: [PATCH 27/68] Downgraded logging in MembershipManagerImpl for registering futures to trace --- .../clients/consumer/internals/MembershipManagerImpl.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 be7dbb882a256..15711948b9c71 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 @@ -394,7 +394,7 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo public void notifyOnStable(CompletableFuture future) { if (state == MemberState.STABLE) { future.complete(null); - log.debug( + log.trace( "Completed future {} since member {} is already in the {} state", future, memberIdForLogging(), @@ -402,7 +402,7 @@ public void notifyOnStable(CompletableFuture future) { ); } else { notifyOnStableFutures.add(future); - log.debug( + log.trace( "Registered future {} for notification when member {} transitions to the {} state", future, memberIdForLogging(), @@ -517,7 +517,7 @@ private void transitionToStable() { transitionTo(MemberState.STABLE); notifyOnStableFutures.forEach(future -> { future.complete(null); - log.debug( + log.trace( "Completed future {} as member {} has transitioned to the {} state", future, memberIdForLogging(), From 3019a9cbac0974968ddc932290a0784f968974b6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 21:49:06 -0800 Subject: [PATCH 28/68] Added processBackgroundEvents to AsyncKafkaConsumer --- .../internals/AsyncKafkaConsumer.java | 70 +++++++++---------- 1 file changed, 35 insertions(+), 35 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 831896ccfcc9f..abcd700b35ef4 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 @@ -44,6 +44,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; 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; import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; @@ -1101,16 +1102,15 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupId.isPresent()) { - try { - // The unsubscribe logic will issue the request to leave the group and then execute any callbacks - // that the user provided. The existing implementation (LegacyKafkaConsumer) blocks the application - // thread when it calls the user's callback. We will implement in kind to maintain compatibility. - Timer timer = time.timer(Duration.ofMillis(Long.MAX_VALUE)); - applicationEventHandler.addAndGet(new UnsubscribeApplicationEvent(), timer); + log.info("Unsubscribing all topics or patterns and assigned partitions"); + Timer timer = time.timer(Long.MAX_VALUE); + UnsubscribeApplicationEvent event = new UnsubscribeApplicationEvent(); + applicationEventHandler.add(event); + + if (processBackgroundEvents(event, 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", e); - } + else + log.info("Timeout expired while unsubscribing"); } subscriptions.unsubscribe(); } finally { @@ -1302,32 +1302,9 @@ private boolean coordinatorPoll(Timer timer) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); - WaitForJoinGroupApplicationEvent event = null; - - do { - backgroundEventProcessor.process(); - - try { - log.debug("Waiting for consumer to join group and transition to {} state", MemberState.STABLE); - - if (event == null) { - // Create and enqueue the event once, but repeatedly wait on the same Future until complete - // or the timer expires. - event = new WaitForJoinGroupApplicationEvent(); - applicationEventHandler.add(event); - } - - ConsumerUtils.getResult(event.future(), time.timer(100)); - - return true; - } catch (TimeoutException e) { - // Ignore this as we will retry the event until the timeout expires. - } finally { - timer.update(time.milliseconds()); - } - } while (timer.notExpired()); - - return false; + WaitForJoinGroupApplicationEvent event = new WaitForJoinGroupApplicationEvent(); + applicationEventHandler.add(event); + return processBackgroundEvents(event, timer); } @Override @@ -1451,6 +1428,29 @@ private void subscribeInternal(Collection topics, Optional event, Timer timer) { + log.trace("Enqueuing event {} for processing; will wait up to {} ms to complete", event, timer.remainingMs()); + + do { + backgroundEventProcessor.process(); + + try { + Timer pollInterval = time.timer(100L); + log.trace("Waiting {} ms for event {} to complete", event, pollInterval.remainingMs()); + ConsumerUtils.getResult(event.future(), pollInterval); + log.trace("Event {} completed successfully", event); + return true; + } catch (TimeoutException e) { + // Ignore this as we will retry the event until the timeout expires. + } finally { + timer.update(time.milliseconds()); + } + } while (timer.notExpired()); + + log.trace("Event {} did not complete within timeout", event); + return false; + } + @Override public String clientId() { return clientId; From 0bb10ee77c2bcc4fc2bf516efa0e3d5dd852ada6 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 21:49:30 -0800 Subject: [PATCH 29/68] PlaintextConsumerTest - enabling testMultiConsumerSessionTimeoutOnClose and testUnsubscribeTopic --- .../scala/integration/kafka/api/PlaintextConsumerTest.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 2663da05bd8c2..12384a27343b7 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -1202,9 +1202,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { runMultiConsumerSessionTimeoutTest(false) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = { runMultiConsumerSessionTimeoutTest(true) } @@ -1419,9 +1418,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(2, topics.get(topic3).size) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testUnsubscribeTopic(quorum: String, groupProtocol: String): Unit = { this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") From 925624b30e9b4661e7b7d337b7cec6b808fcfb4f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 29 Nov 2023 22:10:23 -0800 Subject: [PATCH 30/68] Updates to make disabled tests TODOs --- .../kafka/api/BaseConsumerTest.scala | 2 +- .../kafka/api/PlaintextConsumerTest.scala | 36 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 8c25b394fe7a2..476fab7c6b185 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -78,7 +78,7 @@ abstract class BaseConsumerTest extends AbstractConsumerTest { assertNotEquals(0, BaseConsumerTest.updateConsumerCount.get()) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when partitionsFor is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testCoordinatorFailover(quorum: String, groupProtocol: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 12384a27343b7..c82a2c5d100eb 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -167,7 +167,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { startingTimestamp = startingTimestamp) } - // ConsumerRebalanceListener temporarily not supported for the consumer group protocol + // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = { @@ -194,7 +194,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(1, listener.callsToRevoked) } - // ConsumerRebalanceListener temporarily not supported for the consumer group protocol + // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = { @@ -236,7 +236,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertTrue(commitCompleted) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMaxPollIntervalMsDelayInAssignment(quorum: String, groupProtocol: String): Unit = { @@ -262,7 +262,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { ensureNoRebalance(consumer, listener) } - // Consumer group protocol temporarily does not commit offsets on consumer close + // TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testAutoCommitOnClose(quorum: String, groupProtocol: String): Unit = { @@ -287,7 +287,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(500, anotherConsumer.committed(Set(tp2).asJava).get(tp2).offset) } - // Consumer group protocol temporarily does not commit offsets on consumer close + // TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testAutoCommitOnCloseAfterWakeup(quorum: String, groupProtocol: String): Unit = { @@ -349,7 +349,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { * metadata refresh the consumer becomes subscribed to this new topic and all partitions * of that topic are assigned to it. */ - // Pattern subscriptions temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPatternSubscription(quorum: String, groupProtocol: String): Unit = { @@ -408,7 +408,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { * The metadata refresh interval is intentionally increased to a large enough value to guarantee * that it is the subscription call that triggers a metadata refresh, and not the timeout. */ - // Pattern subscriptions temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = { @@ -461,7 +461,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { * When consumer unsubscribes from all its subscriptions, it is expected that its * assignments are cleared right away. */ - // Pattern subscriptions temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = { @@ -546,7 +546,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { awaitAssignment(consumer, expandedAssignment) } - // Consumer group protocol temporarily does not properly handle assignment change + // TODO: enable this test for the consumer group protocol when proper support for assignment change is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { @@ -562,7 +562,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { awaitAssignment(consumer, shrunkenAssignment) } - // partitionsFor not implemented in consumer group protocol + // TODO: enable this test for the consumer group protocol when partitionsFor is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPartitionsFor(quorum: String, groupProtocol: String): Unit = { @@ -574,7 +574,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(2, parts.size) } - // partitionsFor not implemented in consumer group protocol + // TODO: enable this test for the consumer group protocol when partitionsFor is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPartitionsForAutoCreate(quorum: String, groupProtocol: String): Unit = { @@ -586,7 +586,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { }, s"Timed out while awaiting non empty partitions.") } - // partitionsFor not implemented in consumer group protocol + // TODO: enable this test for the consumer group protocol when partitionsFor is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPartitionsForInvalidTopic(quorum: String, groupProtocol: String): Unit = { @@ -1195,7 +1195,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = { @@ -1396,7 +1396,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { startingTimestamp = startTime, timestampType = TimestampType.LOG_APPEND_TIME) } - // listTopics temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when listTopics is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testListTopics(quorum: String, groupProtocol: String): Unit = { @@ -1485,7 +1485,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(7, consumer.committed(Set(tp2).asJava).get(tp2).offset) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when auto-commit support is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testAutoCommitOnRebalance(quorum: String, groupProtocol: String): Unit = { @@ -1526,7 +1526,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(500, consumer.committed(Set(tp2).asJava).get(tp2).offset) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPerPartitionLeadMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = { @@ -1567,7 +1567,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertNull(consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags2))) } - // ConsumerRebalanceListener temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testPerPartitionLagMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = { @@ -2030,7 +2030,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertThrows(classOf[InvalidGroupIdException], () => consumer1.commitSync()) } - // Static membership temporarily not supported in consumer group protocol + // TODO: enable this test for the consumer group protocol when static membership is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testStaticConsumerDetectsNewPartitionCreatedAfterRestart(quorum:String, groupProtocol: String): Unit = { From 18adb349433400a4d0526cf67364b49757934cc7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 30 Nov 2023 15:23:18 -0800 Subject: [PATCH 31/68] Added documentation --- .../internals/AsyncKafkaConsumer.java | 39 +++++++++++++++++++ 1 file changed, 39 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 abcd700b35ef4..8b1a44777c402 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 @@ -45,6 +45,8 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; 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.ConsumerRebalanceListenerCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; @@ -1428,6 +1430,43 @@ private void subscribeInternal(Collection topics, Optional + *
  • Process background events, if any
  • + *
  • Briefly wait for {@link CompletableApplicationEvent an event} to complete
  • + * + * + *

    + * + * Each iteration gives the application thread an opportunity to process background events, which may be + * necessary to complete the overall processing. + * + *

    + * + * As an example, take {@link #unsubscribe()}. To start unsubscribing, the application thread enqueues an + * {@link UnsubscribeApplicationEvent} 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, + * 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 + * execution of the rebalancing logic. The rebalancing logic cannot complete until the + * {@link ConsumerRebalanceListener} callback is performed. + * + * @param event 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 the application thread will wait for the event to complete + * @return {@code true} if the event completed within the timeout, {@code false} otherwise + */ private boolean processBackgroundEvents(CompletableApplicationEvent event, Timer timer) { log.trace("Enqueuing event {} for processing; will wait up to {} ms to complete", event, timer.remainingMs()); From 7e2cc611b05acead9c7026714ca8497eff2a5a1b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 30 Nov 2023 15:32:58 -0800 Subject: [PATCH 32/68] Reverting unrelated changes --- .../internals/AsyncKafkaConsumer.java | 45 ++++-------------- .../internals/ConsumerNetworkThread.java | 41 ---------------- .../consumer/internals/MembershipManager.java | 9 ---- .../internals/MembershipManagerImpl.java | 47 +------------------ .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventProcessor.java | 15 ------ .../SubscriptionChangeApplicationEvent.java | 2 +- .../WaitForJoinGroupApplicationEvent.java | 32 ------------- 8 files changed, 13 insertions(+), 180 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.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 8b1a44777c402..03c64618275a9 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 @@ -54,7 +54,6 @@ import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.WaitForJoinGroupApplicationEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; @@ -1104,15 +1103,15 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupId.isPresent()) { + UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); + applicationEventHandler.add(unsubscribeApplicationEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); Timer timer = time.timer(Long.MAX_VALUE); - UnsubscribeApplicationEvent event = new UnsubscribeApplicationEvent(); - applicationEventHandler.add(event); - if (processBackgroundEvents(event, timer)) + if (processBackgroundEvents(unsubscribeApplicationEvent, timer)) log.info("Unsubscribed all topics or patterns and assigned partitions"); else - log.info("Timeout expired while unsubscribing"); + log.error("Failed while waiting for the unsubscribe event to complete"); } subscriptions.unsubscribe(); } finally { @@ -1273,40 +1272,14 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd @Override public boolean updateAssignmentMetadataIfNeeded(Timer timer) { - if (isCommittedOffsetsManagementEnabled() && !coordinatorPoll(timer)) { - return false; - } - - return updateFetchPositions(timer); - } - - /** - * This method is an approximation of the logic that the {@link ConsumerCoordinator#poll(Timer, boolean)} method - * performs when it is called by the {@link LegacyKafkaConsumer#poll(Duration)} code. - * - *

    - * - * To mimic the behavior of the {@link ConsumerCoordinator#ensureActiveGroup(Timer)}, we want to block the - * application thread until the consumer fully joins the group. However, we can't block the application thread - * for the full length of the given {@link Timer timer} because we need the application thread to perform - * any {@link ConsumerRebalanceListener} callbacks that are invoked as part of the initial assignment. So we - * have a loop that executes the following until the timer expires: - * - *

      - *
    • Block for a 100 ms.
    • - *
    • Execute any callbacks
    • - *
    - * - * @param timer Timer that limits the time the method will wait to join the group - * @return {@code true} if the join was successful, {@code false} otherwise - */ - private boolean coordinatorPoll(Timer timer) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); + backgroundEventProcessor.process(); - WaitForJoinGroupApplicationEvent event = new WaitForJoinGroupApplicationEvent(); - applicationEventHandler.add(event); - return processBackgroundEvents(event, timer); + // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as + // in the previous implementation, because it will eventually involve group coordination + // logic + return updateFetchPositions(timer); } @Override 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 a317b16008745..240746e52cae4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.common.Node; -import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -40,10 +39,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -68,9 +63,6 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private RequestManagers requestManagers; private volatile boolean running; private final IdempotentCloser closer = new IdempotentCloser(); - private final Lock lock = new ReentrantLock(); - private final Condition wakeupCondition = lock.newCondition(); - private volatile Duration closeTimeout = Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS); public ConsumerNetworkThread(LogContext logContext, @@ -139,8 +131,6 @@ void initializeResources() { * */ void runOnce() { - waitForNewEvents(MAX_POLL_TIMEOUT_MS); - // 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. @@ -214,8 +204,6 @@ public void wakeup() { // The network client can be null if the initializeResources method has not yet been called. if (networkClientDelegate != null) networkClientDelegate.wakeup(); - - notifyOfNewEvents(); } @Override @@ -340,33 +328,4 @@ private void findCoordinatorSync(final Timer timer) { timer.update(); } } - - /** - * Used by the main {@link #runOnce()} method to wait for the arrival of new events to process OR for the - * timeout to expire. This prevents our thread from busy spinning when there aren't events process. - */ - private void waitForNewEvents(long timeoutMs) { - try { - lock.lock(); - - if (wakeupCondition.await(timeoutMs, TimeUnit.MILLISECONDS)) - log.trace("Consumer network I/O thread awoken prematurely"); - } catch (InterruptedException e) { - throw new InterruptException("Interrupted while waiting for new events", e); - } finally { - lock.unlock(); - } - } - - /** - * This is called from {@link #wakeup()} to notify our thread that there are new events to process. - */ - private void notifyOfNewEvents() { - try { - lock.lock(); - wakeupCondition.signalAll(); - } finally { - lock.unlock(); - } - } } 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 541ea7d7398a5..c44405dcf56f4 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 @@ -153,13 +153,4 @@ public interface MembershipManager { */ void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error); - - /** - * Allows the caller to block until the member has joined the consumer group in the - * {@link MemberState#STABLE stable} state. - * - * @param future Future that will be completed when the member has fully joined the consumer group - * (exceptionally or normally) - */ - void notifyOnStable(CompletableFuture future); } 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 15711948b9c71..2543950b83465 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.utils.Utils; import org.slf4j.Logger; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -252,13 +251,6 @@ public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMeth */ private Optional> leaveGroupInProgress; - /** - * The Consumer may wish to wait until it's officially part of a {@link MemberState#STABLE stable} - * consumer group. Code that runs within the {@link ConsumerNetworkThread background thread} can elect - * to be notified when this occurs via the {@link #notifyOnStable(CompletableFuture)} method. - */ - private final List> notifyOnStableFutures = new ArrayList<>(); - /** * True if the member has registered to be notified when the cluster metadata is updated. * This is initially false, as the member that is not part of a consumer group does not @@ -386,28 +378,7 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo resolveMetadataForUnresolvedAssignment(); reconcile(); } else if (allPendingAssignmentsReconciled()) { - transitionToStable(); - } - } - - @Override - public void notifyOnStable(CompletableFuture future) { - if (state == MemberState.STABLE) { - future.complete(null); - log.trace( - "Completed future {} since member {} is already in the {} state", - future, - memberIdForLogging(), - MemberState.STABLE - ); - } else { - notifyOnStableFutures.add(future); - log.trace( - "Registered future {} for notification when member {} transitions to the {} state", - future, - memberIdForLogging(), - MemberState.STABLE - ); + transitionTo(MemberState.STABLE); } } @@ -513,20 +484,6 @@ void transitionToJoining() { registerForMetadataUpdates(); } - private void transitionToStable() { - transitionTo(MemberState.STABLE); - notifyOnStableFutures.forEach(future -> { - future.complete(null); - log.trace( - "Completed future {} as member {} has transitioned to the {} state", - future, - memberIdForLogging(), - MemberState.STABLE - ); - }); - notifyOnStableFutures.clear(); - } - /** * Register to get notified when the cluster metadata is updated, via the * {@link #onUpdate(ClusterResource)}. Register only if the manager is not register already. @@ -638,7 +595,7 @@ public void onHeartbeatRequestSent() { MemberState state = state(); if (state == MemberState.ACKNOWLEDGING) { if (allPendingAssignmentsReconciled()) { - transitionToStable(); + transitionTo(MemberState.STABLE); } else { log.debug("Member {} with epoch {} transitioned to {} after a heartbeat was sent " + "to ack a previous reconciliation. New assignments are ready to " + 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 c16d3153d3579..c896e57bf1f49 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 @@ -26,7 +26,7 @@ public abstract class ApplicationEvent { public enum Type { COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, - UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, WAIT_FOR_JOIN_GROUP + UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED } private final Type type; 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 2c91c970c4590..aa62aa09d6e8d 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 @@ -120,10 +120,6 @@ public void process(ApplicationEvent event) { process((ConsumerRebalanceListenerCallbackCompletedEvent) event); return; - case WAIT_FOR_JOIN_GROUP: - process((WaitForJoinGroupApplicationEvent) event); - return; - default: log.warn("Application event type " + event.type() + " was not expected"); } @@ -244,17 +240,6 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event } } - private void process(final WaitForJoinGroupApplicationEvent event) { - if (!requestManagers.heartbeatRequestManager.isPresent()) { - KafkaException error = new KafkaException("Group membership manager not present when waiting to join a group"); - backgroundEventHandler.add(new ErrorBackgroundEvent(error)); - return; - } - - MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - membershipManager.notifyOnStable(event.future()); - } - /** * 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/SubscriptionChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java index 9a903e5c53223..73fd15fb14408 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 @@ -22,7 +22,7 @@ * calls the subscribe API. This will make the consumer join a consumer group if not part of it * yet, or just send the updated subscription to the broker if it's already a member of the group. */ -public class SubscriptionChangeApplicationEvent extends CompletableApplicationEvent { +public class SubscriptionChangeApplicationEvent extends ApplicationEvent { public SubscriptionChangeApplicationEvent() { super(Type.SUBSCRIPTION_CHANGE); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java deleted file mode 100644 index 05fdf47ef73c7..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/WaitForJoinGroupApplicationEvent.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.clients.consumer.internals.LegacyKafkaConsumer; - -/** - * This event serves as a mechanism to block the application thread until the consumer has joined the consumer group. - * It is used to mimic the blocking behavior of the {@link LegacyKafkaConsumer existing Consumer}. The application - * thread submits this event and then can block on the {@link #future()} to wait for the result. - */ -public class WaitForJoinGroupApplicationEvent extends CompletableApplicationEvent { - - public WaitForJoinGroupApplicationEvent() { - super(Type.WAIT_FOR_JOIN_GROUP); - } -} - From 22e92c063921af02bfe9f9d76b596659b79b8c28 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 30 Nov 2023 16:04:05 -0800 Subject: [PATCH 33/68] More clean up --- .../internals/MembershipManagerImpl.java | 166 +++++++++++------- .../consumer/internals/RequestManagers.java | 2 - 2 files changed, 104 insertions(+), 64 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 2543950b83465..cd8ac45bcf1e6 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 @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; 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; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.ClusterResource; @@ -413,7 +414,7 @@ public void transitionToFenced() { "assignment and rejoin the group.", memberIdForLogging(), memberEpoch, MemberState.FENCED); // Release assignment - CompletableFuture callbackResult = enqueueOnPartitionsLostCallback(subscriptions.assignedPartitions()); + CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); callbackResult.whenComplete((result, error) -> { if (error != null) { log.error("onPartitionsLost callback invocation failed while releasing assignment" + @@ -433,7 +434,7 @@ public void transitionToFatal() { log.error("Member {} with epoch {} transitioned to {} state", memberIdForLogging(), memberEpoch, MemberState.FATAL); // Release assignment - CompletableFuture callbackResult = enqueueOnPartitionsLostCallback(subscriptions.assignedPartitions()); + CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); callbackResult.whenComplete((result, error) -> { if (error != null) { log.error("onPartitionsLost callback invocation failed while releasing assignment" + @@ -516,7 +517,7 @@ public CompletableFuture leaveGroup() { CompletableFuture leaveResult = new CompletableFuture<>(); leaveGroupInProgress = Optional.of(leaveResult); - CompletableFuture callbackResult = enqueueOnPartitionsRevokedOrLostToReleaseAssignment(); + CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); callbackResult.whenComplete((result, error) -> { // Clear the subscription, no matter if the callback execution failed or succeeded. updateSubscription(Collections.emptySet(), true); @@ -546,7 +547,7 @@ public CompletableFuture leaveGroup() { * * @return Future that will complete when the callback execution completes. */ - private CompletableFuture enqueueOnPartitionsRevokedOrLostToReleaseAssignment() { + private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignment() { SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); droppedPartitions.addAll(subscriptions.assignedPartitions()); @@ -561,7 +562,7 @@ private CompletableFuture enqueueOnPartitionsRevokedOrLostToReleaseAssignm callbackResult = revokePartitions(droppedPartitions); } else { // Member is not part of the group anymore. Invoke onPartitionsLost. - callbackResult = enqueueOnPartitionsLostCallback(droppedPartitions); + callbackResult = invokeOnPartitionsLostCallback(droppedPartitions); } } return callbackResult; @@ -936,7 +937,7 @@ private CompletableFuture revokePartitions(Set revokedPart " proceed with the revocation anyway.", error); } - CompletableFuture userCallbackResult = enqueueOnPartitionsRevokedCallback(revokedPartitions); + CompletableFuture userCallbackResult = invokeOnPartitionsRevokedCallback(revokedPartitions); userCallbackResult.whenComplete((callbackResult, callbackError) -> { if (callbackError != null) { log.error("onPartitionsRevoked callback invocation failed for partitions {}", @@ -971,7 +972,7 @@ private CompletableFuture assignPartitions( updateSubscription(assignedPartitions, false); // Invoke user call back - return enqueueOnPartitionsAssignedCallback(addedPartitions); + return invokeOnPartitionsAssignedCallback(addedPartitions); } /** @@ -994,73 +995,108 @@ private void markPendingRevocationToPauseFetching(Set partitions subscriptions.markPendingRevocation(partitionsToRevoke); } - private CompletableFuture enqueueOnPartitionsRevokedCallback(Set partitionsRevoked) { + private CompletableFuture invokeOnPartitionsRevokedCallback(Set partitionsRevoked) { // This should not trigger the callback if partitionsRevoked is empty, to keep the // current behaviour. Optional listener = subscriptions.rebalanceListener(); if (!partitionsRevoked.isEmpty() && listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsRevoked, partitionsRevoked); + return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsRevoked, partitionsRevoked); } else { return CompletableFuture.completedFuture(null); } } - private CompletableFuture enqueueOnPartitionsAssignedCallback(Set partitionsAssigned) { + private CompletableFuture invokeOnPartitionsAssignedCallback(Set partitionsAssigned) { // This should always trigger the callback, even if partitionsAssigned is empty, to keep // the current behaviour. Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsAssigned, partitionsAssigned); + return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsAssigned, partitionsAssigned); } else { return CompletableFuture.completedFuture(null); } } - private CompletableFuture enqueueOnPartitionsLostCallback(Set partitionsLost) { + private CompletableFuture invokeOnPartitionsLostCallback(Set partitionsLost) { // This should not trigger the callback if partitionsLost is empty, to keep the current // behaviour. Optional listener = subscriptions.rebalanceListener(); if (!partitionsLost.isEmpty() && listener.isPresent()) { - return enqueueCallbackEvent(ConsumerRebalanceListenerMethodName.onPartitionsLost, partitionsLost); + return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsLost, partitionsLost); } else { return CompletableFuture.completedFuture(null); } } - private CompletableFuture enqueueCallbackEvent(ConsumerRebalanceListenerMethodName methodName, - Set partitions) { + /** + * Enqueue a {@link ConsumerRebalanceListenerCallbackNeededEvent} to trigger the execution of the + * appropriate {@link ConsumerRebalanceListener} {@link ConsumerRebalanceListenerMethodName method} on the + * application thread. + * + *

    + * + * This method is essentially "giving" the baton from the background thread to the application thread for + * processing of the reconciliation logic. It will "receive" the "baton" back via the + * {@link #consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName, Optional)} method. + * + *

    + * + * Because the reconciliation process (run in the background thread) will be blocked by the application thread + * until it completes this, we need to leave a {@link ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb} + * by which to remember where we left off. + * + * @param methodName Callback method that needs to be executed on the application thread + * @param partitions Partitions to supply to the callback method + * @return Future that will be chained within the rest of the reconciliation logic + */ + private CompletableFuture enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, + Set partitions) { String fullMethodName = String.format( "%s.%s", ConsumerRebalanceListener.class.getSimpleName(), methodName ); - CompletableFuture future = new CompletableFuture<>(); - ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( - methodName, - future - ); - if (breadcrumb == null) { - // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event - // without hesitation. - breadcrumb = newBreadcrumb; - SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - sortedPartitions.addAll(partitions); - BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); - backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", fullMethodName); - } else { - // In this case, there was an existing breadcrumb, so we need to report the matter back to the user. + if (breadcrumb != null) { + // In this case, there was already an existing breadcrumb, so we need to report the matter back to the user. String s = "An internal error occurred; an attempt to schedule the " + - fullMethodName + " method for execution during rebalancing failed because " + - breadcrumb.methodName + " was already scheduled"; + fullMethodName + " method for execution during rebalancing failed because " + + breadcrumb.methodName + " was already scheduled"; + CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new KafkaException(s)); + return future; } + // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event + // without hesitation. + CompletableFuture future = new CompletableFuture<>(); + breadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb(methodName, future); + SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + sortedPartitions.addAll(partitions); + BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + backgroundEventHandler.add(event); + log.debug("The event to trigger the {} method execution was enqueued successfully", fullMethodName); + return future; } + /** + * Signals that a {@link ConsumerRebalanceListener} callback has completed. This is invoked when the + * application thread has completed the callback and has submitted a + * {@link ConsumerRebalanceListenerCallbackCompletedEvent} to the network I/O thread. At this point, we + * notify the state machine that it's complete so that it can move to the next appropriate step of the + * rebalance process. + * + *

    + * + * This method is "receiving" the baton back from the application thread after having "given" it to the + * application thread via the + * {@link #enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName, Set)} method. + * + * @param methodName Method name of the callback that was executed + * @param error Optional error that was thrown by the callback, captured, and forwarded here + */ @Override public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error) { @@ -1070,45 +1106,51 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener methodName ); - if (breadcrumb != null && breadcrumb.methodName == methodName) { - // We have a breadcrumb that matches the callback we expect, so we can proceed to the next step of - // the rebalance process. - CompletableFuture future = breadcrumb.future; - - // We need to clear out our breadcrumb to signal that we've completed this step of the rebalance. - breadcrumb = null; - - if (error.isPresent()) { - log.warn( - "The {} method completed with an error; signaling to continue to the next phase of rebalance", - fullMethodName, - error.get() - ); - } else { - log.debug( - "The {} method completed successfully; signaling to continue to the next phase of rebalance", - fullMethodName - ); - } + if (breadcrumb == null) { + // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. + // Because of that, we don't have a Future that can be completed, so we're left having to report it + // back to the user asynchronously. + String s = "An internal error occurred; the " + fullMethodName + " method was executed " + + "during rebalancing, but there was no record of it being scheduled"; + backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); + return; + } - future.complete(null); - } else if (breadcrumb != null) { + if (breadcrumb.methodName != methodName) { // We have a breadcrumb that implicitly does NOT match the one we expect. We need to abort the // rebalance process, because we're in an inconsistent state. We do that by completing the Future // with an error. String s = "An internal error occurred; an attempt to continue rebalance after the execution of the " + - fullMethodName + " method failed because the expected method was " + breadcrumb.methodName; + fullMethodName + " method failed because the expected method was " + breadcrumb.methodName; CompletableFuture future = breadcrumb.future; + + // Set the breadcrumb to null to clear our state. breadcrumb = null; future.completeExceptionally(new KafkaException(s)); + return; + } + + // We have a breadcrumb that matches the callback we expect, so we can proceed to the next step of + // the rebalance process. + CompletableFuture future = breadcrumb.future; + + // We need to clear out our breadcrumb to signal that we've completed this step of the rebalance. + breadcrumb = null; + + if (error.isPresent()) { + log.warn( + "The {} method completed with an error; signaling to continue to the next phase of rebalance", + fullMethodName, + error.get() + ); } else { - // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. - // Because of that, we don't have a Future that can be completed, so we're left having to report it - // back to the user asynchronously. - String s = "An internal error occurred; the " + fullMethodName + " method was executed " + - "during rebalancing, but there was no record of it being scheduled"; - backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); + log.debug( + "The {} method completed successfully; signaling to continue to the next phase of rebalance", + fullMethodName + ); } + + future.complete(null); } /** 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 0cb817c9bd014..58fc1f6bf2a13 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 @@ -19,7 +19,6 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.GroupRebalanceConfig; 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.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; @@ -31,7 +30,6 @@ import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; import static org.apache.kafka.common.utils.Utils.closeQuietly; From 0f3809517861ccfba6b0f8ab25b707c8bdef40b5 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 17:01:01 -0800 Subject: [PATCH 34/68] Minor clean up --- .../internals/AsyncKafkaConsumer.java | 1 + .../ConsumerRebalanceListenerInvoker.java | 18 ++++------ .../ConsumerRebalanceListenerMethodName.java | 18 ++++++++++ .../internals/MembershipManagerImpl.java | 35 +++++++------------ 4 files changed, 38 insertions(+), 34 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 1a99012d17d76..6c87c2a2d67d1 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 @@ -492,6 +492,7 @@ public ConsumerRecords poll(final Duration timeout) { // of the fetches. A wakeup between returned fetches and returning records would lead to never // returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches. wakeupTrigger.maybeTriggerWakeup(); + updateAssignmentMetadataIfNeeded(timer); final Fetch fetch = pollForFetches(timer); if (!fetch.isEmpty()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java index 48fea2e3d618d..d0e8df6a67b41 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java @@ -58,17 +58,15 @@ public Exception invokePartitionsAssigned(final SortedSet assign Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - ConsumerRebalanceListener consumerRebalanceListener = listener.get(); - try { final long startMs = time.milliseconds(); - consumerRebalanceListener.onPartitionsAssigned(assignedPartitions); + listener.get().onPartitionsAssigned(assignedPartitions); coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}", - consumerRebalanceListener.getClass().getName(), assignedPartitions, e); + listener.get().getClass().getName(), assignedPartitions, e); return e; } } @@ -86,17 +84,15 @@ public Exception invokePartitionsRevoked(final SortedSet revoked Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - ConsumerRebalanceListener consumerRebalanceListener = listener.get(); - try { final long startMs = time.milliseconds(); - consumerRebalanceListener.onPartitionsRevoked(revokedPartitions); + listener.get().onPartitionsRevoked(revokedPartitions); coordinatorMetrics.revokeCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}", - consumerRebalanceListener.getClass().getName(), revokedPartitions, e); + listener.get().getClass().getName(), revokedPartitions, e); return e; } } @@ -114,17 +110,15 @@ public Exception invokePartitionsLost(final SortedSet lostPartit Optional listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - ConsumerRebalanceListener consumerRebalanceListener = listener.get(); - try { final long startMs = time.milliseconds(); - consumerRebalanceListener.onPartitionsLost(lostPartitions); + listener.get().onPartitionsLost(lostPartitions); coordinatorMetrics.loseCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}", - consumerRebalanceListener.getClass().getName(), lostPartitions, e); + listener.get().getClass().getName(), lostPartitions, e); return e; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java index 3d99b134ca0a6..1ffb924ecb06b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java @@ -16,6 +16,24 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; + +/** + * This class just provides a static name for the methods in the {@link ConsumerRebalanceListener} interface + * for a bit more compile time assurance. + */ public enum ConsumerRebalanceListenerMethodName { + onPartitionsRevoked, onPartitionsAssigned, onPartitionsLost; + + /** + * Provides the fully-qualified method name, e.g. {@code ConsumerRebalanceListener.onPartitionsRevoked}. This + * is used for log messages. + * + * @return Full-qualified method name + */ + @Override + public String toString() { + return String.format("%s.%s", ConsumerRebalanceListener.class.getSimpleName(), name()); + } } 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 9c55c5963c2ad..bfb9dc223ed41 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 @@ -53,6 +53,10 @@ import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsAssigned; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsLost; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsRevoked; + /** * Group manager for a single consumer that has a group id defined in the config * {@link ConsumerConfig#GROUP_ID_CONFIG}, to use the Kafka-based offset management capability, @@ -1000,7 +1004,7 @@ private CompletableFuture invokeOnPartitionsRevokedCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsRevoked.isEmpty() && listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsRevoked, partitionsRevoked); + return enqueueConsumerRebalanceListenerCallback(onPartitionsRevoked, partitionsRevoked); } else { return CompletableFuture.completedFuture(null); } @@ -1011,7 +1015,7 @@ private CompletableFuture invokeOnPartitionsAssignedCallback(Set listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsAssigned, partitionsAssigned); + return enqueueConsumerRebalanceListenerCallback(onPartitionsAssigned, partitionsAssigned); } else { return CompletableFuture.completedFuture(null); } @@ -1022,7 +1026,7 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsLost.isEmpty() && listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName.onPartitionsLost, partitionsLost); + return enqueueConsumerRebalanceListenerCallback(onPartitionsLost, partitionsLost); } else { return CompletableFuture.completedFuture(null); } @@ -1051,17 +1055,10 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, Set partitions) { - String fullMethodName = String.format( - "%s.%s", - ConsumerRebalanceListener.class.getSimpleName(), - methodName - ); - - if (breadcrumb != null) { // In this case, there was already an existing breadcrumb, so we need to report the matter back to the user. String s = "An internal error occurred; an attempt to schedule the " + - fullMethodName + " method for execution during rebalancing failed because " + + methodName + " method for execution during rebalancing failed because " + breadcrumb.methodName + " was already scheduled"; CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new KafkaException(s)); @@ -1076,7 +1073,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume sortedPartitions.addAll(partitions); BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", fullMethodName); + log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); return future; } @@ -1100,17 +1097,11 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume @Override public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error) { - String fullMethodName = String.format( - "%s.%s", - ConsumerRebalanceListener.class.getSimpleName(), - methodName - ); - if (breadcrumb == null) { // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. // Because of that, we don't have a Future that can be completed, so we're left having to report it // back to the user asynchronously. - String s = "An internal error occurred; the " + fullMethodName + " method was executed " + + String s = "An internal error occurred; the " + methodName + " method was executed " + "during rebalancing, but there was no record of it being scheduled"; backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); return; @@ -1121,7 +1112,7 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener // rebalance process, because we're in an inconsistent state. We do that by completing the Future // with an error. String s = "An internal error occurred; an attempt to continue rebalance after the execution of the " + - fullMethodName + " method failed because the expected method was " + breadcrumb.methodName; + methodName + " method failed because the expected method was " + breadcrumb.methodName; CompletableFuture future = breadcrumb.future; // Set the breadcrumb to null to clear our state. @@ -1140,13 +1131,13 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener if (error.isPresent()) { log.warn( "The {} method completed with an error; signaling to continue to the next phase of rebalance", - fullMethodName, + methodName, error.get() ); } else { log.debug( "The {} method completed successfully; signaling to continue to the next phase of rebalance", - fullMethodName + methodName ); } From 3babe24c1812850b60fbb1ee841bb38543c688e4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 17:02:34 -0800 Subject: [PATCH 35/68] More clean up to remove unnecessary diffs --- .../consumer/internals/events/ApplicationEventProcessor.java | 5 +---- 1 file changed, 1 insertion(+), 4 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 aa62aa09d6e8d..0b60d43017c64 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 @@ -188,7 +188,6 @@ private void processSubscriptionChangeEvent() { backgroundEventHandler.add(new ErrorBackgroundEvent(error)); return; } - MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); membershipManager.onSubscriptionUpdated(); } @@ -207,7 +206,6 @@ private void processUnsubscribeEvent(UnsubscribeApplicationEvent event) { backgroundEventHandler.add(new ErrorBackgroundEvent(error)); return; } - MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); CompletableFuture result = membershipManager.leaveGroup(); event.chain(result); @@ -233,8 +231,7 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event manager.consumerRebalanceListenerCallbackCompleted(event.methodName(), event.error()); } else { log.warn( - "An internal error occurred; the group membership manager was not present, so the notification of the {}.{} callback's execution could not be sent", - ConsumerRebalanceListener.class.getSimpleName(), + "An internal error occurred; the group membership manager was not present, so the notification of the {} callback execution could not be sent", event.methodName() ); } From 8ef0f2e6c3f6a2f3350fef0f9a747a86e5e52fab Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 17:09:01 -0800 Subject: [PATCH 36/68] More clean up --- .../internals/AsyncKafkaConsumer.java | 2 -- .../events/ApplicationEventProcessor.java | 23 +++++++------------ .../internals/ConsumerTestBuilder.java | 1 - 3 files changed, 8 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 6c87c2a2d67d1..9e214283fb66a 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 @@ -243,7 +243,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, applicationEventQueue, - backgroundEventHandler, requestManagersSupplier); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, @@ -431,7 +430,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { logContext, metadata, applicationEventQueue, - backgroundEventHandler, requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(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 0b60d43017c64..d23d2a90315cc 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.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -46,16 +45,13 @@ public class ApplicationEventProcessor extends EventProcessor private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; - private final BackgroundEventHandler backgroundEventHandler; public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, - final BackgroundEventHandler backgroundEventHandler, final RequestManagers requestManagers, final ConsumerMetadata metadata) { super(new LogContext("[Application event processor]" + (logContext.logPrefix() != null ? " " + logContext.logPrefix() : "")), applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); - this.backgroundEventHandler = backgroundEventHandler; this.requestManagers = requestManagers; this.metadata = metadata; } @@ -109,7 +105,7 @@ public void process(ApplicationEvent event) { return; case SUBSCRIPTION_CHANGE: - processSubscriptionChangeEvent(); + processSubscriptionChangeEvent((SubscriptionChangeApplicationEvent) event); return; case UNSUBSCRIBE: @@ -182,10 +178,9 @@ private void process(final ListOffsetsApplicationEvent event) { * consumer join the group if it is not part of it yet, or send the updated subscription if * it is already a member. */ - private void processSubscriptionChangeEvent() { + private void processSubscriptionChangeEvent(SubscriptionChangeApplicationEvent __) { if (!requestManagers.heartbeatRequestManager.isPresent()) { - KafkaException error = new KafkaException("Group membership manager not present when processing a subscribe event"); - backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + log.warn("Group membership manager not present when processing a subscribe event"); return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); @@ -203,7 +198,7 @@ private void processSubscriptionChangeEvent() { private void processUnsubscribeEvent(UnsubscribeApplicationEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { KafkaException error = new KafkaException("Group membership manager not present when processing an unsubscribe event"); - backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + event.future().completeExceptionally(error); return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); @@ -226,15 +221,15 @@ private void process(final TopicMetadataApplicationEvent event) { } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { - if (requestManagers.heartbeatRequestManager.isPresent()) { - MembershipManager manager = requestManagers.heartbeatRequestManager.get().membershipManager(); - manager.consumerRebalanceListenerCallbackCompleted(event.methodName(), event.error()); - } else { + if (!requestManagers.heartbeatRequestManager.isPresent()) { log.warn( "An internal error occurred; the group membership manager was not present, so the notification of the {} callback execution could not be sent", event.methodName() ); + return; } + MembershipManager manager = requestManagers.heartbeatRequestManager.get().membershipManager(); + manager.consumerRebalanceListenerCallbackCompleted(event.methodName(), event.error()); } /** @@ -244,7 +239,6 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, final BlockingQueue applicationEventQueue, - final BackgroundEventHandler backgroundEventHandler, final Supplier requestManagersSupplier) { return new CachedSupplier() { @Override @@ -253,7 +247,6 @@ protected ApplicationEventProcessor create() { return new ApplicationEventProcessor( logContext, applicationEventQueue, - backgroundEventHandler, requestManagers, metadata ); 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 5353eeab1cbcc..3ae231e692233 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 @@ -262,7 +262,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, applicationEventQueue, - backgroundEventHandler, requestManagers, metadata) ); From b2be1e53cd23526411786079fd10eb54b6c195bb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:20:29 -0800 Subject: [PATCH 37/68] More clean up --- .../internals/MembershipManagerImpl.java | 112 +++++++++--------- .../events/ApplicationEventProcessor.java | 4 +- 2 files changed, 60 insertions(+), 56 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 bfb9dc223ed41..108c5453d1667 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 @@ -52,6 +52,7 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsAssigned; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsLost; @@ -121,8 +122,8 @@ private static class ConsumerRebalanceListenerCallbackBreadcrumb { private final ConsumerRebalanceListenerMethodName methodName; private final CompletableFuture future; - public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMethodName methodName, - CompletableFuture future) { + private ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMethodName methodName, + CompletableFuture future) { this.methodName = Objects.requireNonNull(methodName); this.future = Objects.requireNonNull(future); } @@ -273,9 +274,11 @@ public ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMeth /** * Breadcrumb that we can return to as we wait for the completion of the - * {@link ConsumerRebalanceListenerCallbackNeededEvent} that was enqueued during rebalancing. + * {@link ConsumerRebalanceListenerCallbackNeededEvent} that was enqueued during rebalancing. This + * is an {@link AtomicReference} as it is updated by both the {@link ConsumerNetworkThread background thread} + * and the application thread. */ - private ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb; + private final AtomicReference breadcrumbRef; public MembershipManagerImpl(String groupId, SubscriptionState subscriptions, @@ -308,6 +311,7 @@ public MembershipManagerImpl(String groupId, this.currentAssignment = new HashSet<>(); this.log = logContext.logger(MembershipManagerImpl.class); this.backgroundEventHandler = backgroundEventHandler; + this.breadcrumbRef = new AtomicReference<>(); } /** @@ -1055,26 +1059,36 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, Set partitions) { - if (breadcrumb != null) { + CompletableFuture future = new CompletableFuture<>(); + ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( + methodName, + future + ); + + if (breadcrumbRef.compareAndSet(null, newBreadcrumb)) { + // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event + // without hesitation. + SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + sortedPartitions.addAll(partitions); + BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + backgroundEventHandler.add(event); + log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); + } else { + ConsumerRebalanceListenerCallbackBreadcrumb unexpected = breadcrumbRef.get(); + + // If our CAS above failed it's because the breadcrumb wasn't null, that is, there was already a + // breadcrumb set. For our error, we try to get some info from that existing breadcrumb for better + // debugging. But there's technically a race condition, which means existingBreadcrumb could be null + // by the time we try to access it. In that case, we just use a generic name. + String unexpectedMethodName = unexpected != null ? unexpected.methodName.toString() : "another invocation"; + // In this case, there was already an existing breadcrumb, so we need to report the matter back to the user. - String s = "An internal error occurred; an attempt to schedule the " + + String s = "An internal error occurred: an attempt to schedule the " + methodName + " method for execution during rebalancing failed because " + - breadcrumb.methodName + " was already scheduled"; - CompletableFuture future = new CompletableFuture<>(); + unexpectedMethodName + " was already scheduled"; future.completeExceptionally(new KafkaException(s)); - return future; } - // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event - // without hesitation. - CompletableFuture future = new CompletableFuture<>(); - breadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb(methodName, future); - SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - sortedPartitions.addAll(partitions); - BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); - backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); - return future; } @@ -1097,51 +1111,41 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume @Override public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, Optional error) { + // Get any existing breadcrumb out and clear the state for the next reconciliation. + ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb = breadcrumbRef.getAndSet(null); + if (breadcrumb == null) { - // In this case, we're somehow completing a callback for which we don't have a recorded breadcrumb. + // Case 1: we're somehow completing a callback for which we don't have a recorded breadcrumb. // Because of that, we don't have a Future that can be completed, so we're left having to report it // back to the user asynchronously. - String s = "An internal error occurred; the " + methodName + " method was executed " + + String s = "An internal error occurred: the " + methodName + " method was executed " + "during rebalancing, but there was no record of it being scheduled"; backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); - return; - } - - if (breadcrumb.methodName != methodName) { - // We have a breadcrumb that implicitly does NOT match the one we expect. We need to abort the + } else if (breadcrumb.methodName != methodName) { + // Case 2: we have a breadcrumb, but it does NOT match the one we expect. We need to abort the // rebalance process, because we're in an inconsistent state. We do that by completing the Future // with an error. - String s = "An internal error occurred; an attempt to continue rebalance after the execution of the " + + String s = "An internal error occurred: an attempt to continue rebalance after the execution of the " + methodName + " method failed because the expected method was " + breadcrumb.methodName; - CompletableFuture future = breadcrumb.future; - - // Set the breadcrumb to null to clear our state. - breadcrumb = null; - future.completeExceptionally(new KafkaException(s)); - return; - } - - // We have a breadcrumb that matches the callback we expect, so we can proceed to the next step of - // the rebalance process. - CompletableFuture future = breadcrumb.future; - - // We need to clear out our breadcrumb to signal that we've completed this step of the rebalance. - breadcrumb = null; - - if (error.isPresent()) { - log.warn( - "The {} method completed with an error; signaling to continue to the next phase of rebalance", - methodName, - error.get() - ); + breadcrumb.future.completeExceptionally(new KafkaException(s)); } else { - log.debug( - "The {} method completed successfully; signaling to continue to the next phase of rebalance", - methodName - ); - } + // Case 3: the happy path. We have a breadcrumb that matches what we expect, so we can proceed to + // the next step of the rebalance process. + if (error.isPresent()) { + log.warn( + "The {} method completed with an error ({}). signaling to continue to the next phase of rebalance", + methodName, + error.get().getMessage() + ); + } else { + log.debug( + "The {} method completed successfully; signaling to continue to the next phase of rebalance", + methodName + ); + } - future.complete(null); + breadcrumb.future.complete(null); + } } /** 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 d23d2a90315cc..7ddd9c40ca2db 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 @@ -105,7 +105,7 @@ public void process(ApplicationEvent event) { return; case SUBSCRIPTION_CHANGE: - processSubscriptionChangeEvent((SubscriptionChangeApplicationEvent) event); + processSubscriptionChangeEvent(); return; case UNSUBSCRIBE: @@ -178,7 +178,7 @@ private void process(final ListOffsetsApplicationEvent event) { * consumer join the group if it is not part of it yet, or send the updated subscription if * it is already a member. */ - private void processSubscriptionChangeEvent(SubscriptionChangeApplicationEvent __) { + private void processSubscriptionChangeEvent() { if (!requestManagers.heartbeatRequestManager.isPresent()) { log.warn("Group membership manager not present when processing a subscribe event"); return; From f3b092c47787956cd6302428eda65a24008e0d67 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:24:30 -0800 Subject: [PATCH 38/68] Removed memberIdForLogging() as memberId() is exposed and used for logging in HeartbeatRequestManager, so it's inconsistent --- .../consumer/internals/MembershipManagerImpl.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 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 108c5453d1667..8fa7909c459e1 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 @@ -325,7 +325,7 @@ private void transitionTo(MemberState nextState) { throw new IllegalStateException(String.format("Invalid state transition from %s to %s", state, nextState)); } - log.trace("Member {} transitioned from {} to {}.", memberIdForLogging(), state, nextState); + log.trace("Member {} transitioned from {} to {}.", memberId, state, nextState); this.state = nextState; } @@ -353,10 +353,6 @@ public String memberId() { return memberId; } - private String memberIdForLogging() { - return memberId != null && !memberId.trim().isEmpty() ? memberId : ""; - } - /** * {@inheritDoc} */ @@ -419,7 +415,7 @@ public void transitionToFenced() { transitionTo(MemberState.FENCED); resetEpoch(); log.debug("Member {} with epoch {} transitioned to {} state. It will release its " + - "assignment and rejoin the group.", memberIdForLogging(), memberEpoch, MemberState.FENCED); + "assignment and rejoin the group.", memberId, memberEpoch, MemberState.FENCED); // Release assignment CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); @@ -439,7 +435,7 @@ public void transitionToFenced() { @Override public void transitionToFatal() { transitionTo(MemberState.FATAL); - log.error("Member {} with epoch {} transitioned to {} state", memberIdForLogging(), memberEpoch, MemberState.FATAL); + log.error("Member {} with epoch {} transitioned to {} state", memberId, memberEpoch, MemberState.FATAL); // Release assignment CompletableFuture callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); @@ -608,7 +604,7 @@ public void onHeartbeatRequestSent() { } else { log.debug("Member {} with epoch {} transitioned to {} after a heartbeat was sent " + "to ack a previous reconciliation. New assignments are ready to " + - "be reconciled.", memberIdForLogging(), memberEpoch, MemberState.RECONCILING); + "be reconciled.", memberId, memberEpoch, MemberState.RECONCILING); transitionTo(MemberState.RECONCILING); } } else if (state == MemberState.LEAVING) { @@ -623,7 +619,7 @@ public void onHeartbeatRequestSent() { public void onHeartbeatRequestSkipped() { if (state == MemberState.LEAVING) { log.debug("Heartbeat for leaving group could not be sent. Member {} with epoch {} will transition to {}.", - memberIdForLogging(), memberEpoch, MemberState.UNSUBSCRIBED); + memberId, memberEpoch, MemberState.UNSUBSCRIBED); transitionToUnsubscribed(); } } From 134bfdb2cd616a7b7f545636deed397a8e2e80a2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:34:59 -0800 Subject: [PATCH 39/68] Reverted the generalization of the EventHandler as it didn't gain much --- .../events/ApplicationEventHandler.java | 23 ++++--- .../events/BackgroundEventHandler.java | 23 ++++++- .../internals/events/EventHandler.java | 61 ------------------- 3 files changed, 31 insertions(+), 76 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java 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 e81195159f4eb..f32da7f232ddd 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 @@ -26,6 +26,7 @@ import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; +import java.io.Closeable; import java.time.Duration; import java.util.Objects; import java.util.concurrent.BlockingQueue; @@ -37,9 +38,10 @@ * An event handler that receives {@link ApplicationEvent application events} from the application thread which * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. */ -public class ApplicationEventHandler extends EventHandler { +public class ApplicationEventHandler implements Closeable { private final Logger log; + private final BlockingQueue applicationEventQueue; private final ConsumerNetworkThread networkThread; private final IdempotentCloser closer = new IdempotentCloser(); @@ -49,8 +51,8 @@ public ApplicationEventHandler(final LogContext logContext, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier) { - super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventHandler.class); + this.applicationEventQueue = applicationEventQueue; this.networkThread = new ConsumerNetworkThread(logContext, time, applicationEventProcessorSupplier, @@ -60,14 +62,15 @@ public ApplicationEventHandler(final LogContext logContext, } /** - * Add an event to the underlying queue and internally invoke {@link #wakeupNetworkThread()} to alert it that - * it has an event to process. + * Add an {@link ApplicationEvent} to the underlying queue and internally invoke {@link #wakeupNetworkThread()} + * to alert the network I/O thread that it has one or more events to process. * - * @param event An event to enqueue for later processing + * @param event An {@link ApplicationEvent} created by the application thread */ - @Override public void add(final ApplicationEvent event) { - super.add(event); + Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null"); + applicationEventQueue.add(event); + log.trace("Enqueued event: {}", event); wakeupNetworkThread(); } @@ -115,13 +118,9 @@ public void close() { close(Duration.ZERO); } - @Override public void close(final Duration timeout) { closer.close( - () -> { - Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"); - super.close(timeout); - }, + () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), () -> log.warn("The application event handler was already closed") ); } 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 f36628afce75d..033811f12b7ef 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 @@ -18,7 +18,9 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; +import java.util.Objects; import java.util.concurrent.BlockingQueue; /** @@ -26,9 +28,24 @@ * {@link ConsumerNetworkThread network thread} which are then made available to the application thread * via the {@link BackgroundEventProcessor}. */ -public class BackgroundEventHandler extends EventHandler { +public class BackgroundEventHandler { - public BackgroundEventHandler(final LogContext logContext, final BlockingQueue queue) { - super(logContext, queue); + private final Logger log; + private final BlockingQueue backgroundEventQueue; + + public BackgroundEventHandler(final LogContext logContext, final BlockingQueue backgroundEventQueue) { + this.log = logContext.logger(BackgroundEventHandler.class); + this.backgroundEventQueue = backgroundEventQueue; + } + + /** + * Add a {@link BackgroundEvent} to the handler. + * + * @param event A {@link BackgroundEvent} created by the {@link ConsumerNetworkThread network thread} + */ + public void add(BackgroundEvent event) { + Objects.requireNonNull(event, "BackgroundEvent provided to add must be non-null"); + backgroundEventQueue.add(event); + log.trace("Enqueued event: {}", event); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java deleted file mode 100644 index c68f89f2324a4..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ /dev/null @@ -1,61 +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.common.utils.LogContext; -import org.slf4j.Logger; - -import java.io.Closeable; -import java.time.Duration; -import java.util.Objects; -import java.util.concurrent.BlockingQueue; - -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; - -/** - * An event handler is used to publish events from one thread which are then consumed from another thread. - */ -public class EventHandler implements Closeable { - - private final Logger log; - private final BlockingQueue queue; - - public EventHandler(final LogContext logContext, final BlockingQueue queue) { - this.log = logContext.logger(EventHandler.class); - this.queue = queue; - } - - /** - * Add an event to the underlying queue. - * - * @param event An event to enqueue for later processing - */ - public void add(final T event) { - Objects.requireNonNull(event, "Event must be non-null"); - queue.add(event); - log.trace("Enqueued event: {}", event); - } - - @Override - public void close() { - close(Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS)); - } - - public void close(Duration timeout) { - // Do nothing. Available for subclasses. - } -} From edbffc4d6161468cc005014e24bca728513d81dd Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:36:10 -0800 Subject: [PATCH 40/68] Reverted some unnecessary diffs for BackgroundEventHandler --- .../consumer/internals/events/BackgroundEventHandler.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 033811f12b7ef..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 @@ -21,19 +21,20 @@ import org.slf4j.Logger; import java.util.Objects; -import java.util.concurrent.BlockingQueue; +import java.util.Queue; /** * 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}. */ + public class BackgroundEventHandler { private final Logger log; - private final BlockingQueue backgroundEventQueue; + private final Queue backgroundEventQueue; - public BackgroundEventHandler(final LogContext logContext, final BlockingQueue backgroundEventQueue) { + public BackgroundEventHandler(final LogContext logContext, final Queue backgroundEventQueue) { this.log = logContext.logger(BackgroundEventHandler.class); this.backgroundEventQueue = backgroundEventQueue; } From 69f80735887c13006c23f059b7c41b4813168d7a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:38:12 -0800 Subject: [PATCH 41/68] Removed more unnecessary changes --- .../consumer/internals/events/ApplicationEventHandler.java | 4 ++-- .../consumer/internals/events/ApplicationEventProcessor.java | 2 +- .../consumer/internals/events/BackgroundEventProcessor.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) 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 f32da7f232ddd..df2710374d56d 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 @@ -62,8 +62,8 @@ public ApplicationEventHandler(final LogContext logContext, } /** - * Add an {@link ApplicationEvent} to the underlying queue and internally invoke {@link #wakeupNetworkThread()} - * to alert the network I/O thread that it has one or more events to process. + * Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread()} + * to alert the network I/O thread that it has something to process. * * @param event An {@link ApplicationEvent} created by the application thread */ 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 7ddd9c40ca2db..2274dae455519 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 @@ -50,7 +50,7 @@ public ApplicationEventProcessor(final LogContext logContext, final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { - super(new LogContext("[Application event processor]" + (logContext.logPrefix() != null ? " " + logContext.logPrefix() : "")), applicationEventQueue); + super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index eafaae15a8eec..1f56165d007f5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -50,7 +50,7 @@ public BackgroundEventProcessor(final LogContext logContext, final BlockingQueue backgroundEventQueue, final ApplicationEventHandler applicationEventHandler, final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { - super(new LogContext("[Background event processor]" + (logContext.logPrefix() != null ? " " + logContext.logPrefix() : "")), backgroundEventQueue); + super(logContext, backgroundEventQueue); this.log = logContext.logger(BackgroundEventProcessor.class); this.applicationEventHandler = applicationEventHandler; this.rebalanceListenerInvoker = rebalanceListenerInvoker; From 7183d91af83d1cf88ed5ddb019141791900fa100 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 1 Dec 2023 18:44:13 -0800 Subject: [PATCH 42/68] More clean up --- .../consumer/internals/events/ApplicationEventHandler.java | 4 ++-- .../consumer/internals/events/ApplicationEventProcessor.java | 1 + .../consumer/internals/events/BackgroundEventProcessor.java | 3 ++- .../clients/consumer/internals/events/EventProcessor.java | 2 ++ 4 files changed, 7 insertions(+), 3 deletions(-) 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 df2710374d56d..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 @@ -94,8 +94,8 @@ public long maximumTimeToWait() { } /** - * Add a {@link CompletableApplicationEvent} to the underlying queue. The method blocks waiting for the result, - * and will return the result value upon successful completion; otherwise throws an error. + * 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. * *

    * 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 2274dae455519..8e5787edc06cc 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,6 +61,7 @@ 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. */ + @Override public void process() { process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index 1f56165d007f5..08a86559e759b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -62,6 +62,7 @@ public BackgroundEventProcessor(final LogContext logContext, * 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 void process() { AtomicReference firstError = new AtomicReference<>(); @@ -120,7 +121,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { break; default: - throw new IllegalArgumentException("Could not determine the " + ConsumerRebalanceListener.class.getSimpleName() + " to invoke from the callback method " + methodName); + throw new IllegalArgumentException("The method " + methodName + " to invoke was not expected"); } final Optional error; 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 0eb8b5b7a8832..64aa6f2f77b8e 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 @@ -47,6 +47,8 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } + public abstract void process(); + protected abstract void process(T event); @Override From 9645a24ec223f4f6a5294074918b7ec664e0f6a4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 4 Dec 2023 15:03:31 -0800 Subject: [PATCH 43/68] Updates based on feedback --- .../ConsumerRebalanceListenerMethodName.java | 30 +++- .../consumer/internals/MembershipManager.java | 6 +- .../internals/MembershipManagerImpl.java | 158 ++++-------------- .../events/ApplicationEventProcessor.java | 2 +- .../events/BackgroundEventProcessor.java | 12 +- .../events/CompletableBackgroundEvent.java | 86 ++++++++++ ...balanceListenerCallbackCompletedEvent.java | 16 +- ...rRebalanceListenerCallbackNeededEvent.java | 7 +- 8 files changed, 173 insertions(+), 144 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java index 1ffb924ecb06b..1a25ecefa7a7a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java @@ -24,16 +24,36 @@ */ public enum ConsumerRebalanceListenerMethodName { - onPartitionsRevoked, onPartitionsAssigned, onPartitionsLost; + ON_PARTITIONS_REVOKED("onPartitionsRevoked"), + ON_PARTITIONS_ASSIGNED("onPartitionsAssigned"), + ON_PARTITIONS_LOST("onPartitionsLost"); + + private final String methodName; + + private final String fullyQualifiedMethodName; + + ConsumerRebalanceListenerMethodName(String methodName) { + this.methodName = methodName; + this.fullyQualifiedMethodName = String.format("%s.%s", ConsumerRebalanceListener.class.getSimpleName(), methodName); + } + + /** + * Provides the method name, e.g. {@code onPartitionsRevoked}. + * + * @return Method name + */ + + public String methodName() { + return methodName; + } /** * Provides the fully-qualified method name, e.g. {@code ConsumerRebalanceListener.onPartitionsRevoked}. This * is used for log messages. * - * @return Full-qualified method name + * @return Fully-qualified method name */ - @Override - public String toString() { - return String.format("%s.%s", ConsumerRebalanceListener.class.getSimpleName(), name()); + public String fullyQualifiedMethodName() { + return fullyQualifiedMethodName; } } 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 c44405dcf56f4..88a697dc74236 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 @@ -148,9 +148,7 @@ public interface MembershipManager { * notify the state machine that it's complete so that it can move to the next appropriate step of the * rebalance process. * - * @param methodName Method name of the callback that was executed - * @param error Optional error that was thrown by the callback, captured, and forwarded here + * @param event Event with details about the callback that was executed */ - void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, - Optional error); + void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackCompletedEvent event); } 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 8fa7909c459e1..46024f973bcd8 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 @@ -21,8 +21,8 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.Utils.TopicIdPartitionComparator; import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; -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.CompletableBackgroundEvent; 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.ErrorBackgroundEvent; @@ -46,17 +46,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsAssigned; -import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsLost; -import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.onPartitionsRevoked; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED; /** * Group manager for a single consumer that has a group id defined in the config @@ -112,23 +110,6 @@ */ public class MembershipManagerImpl implements MembershipManager, ClusterResourceListener { - /** - * Used during the rebalancing process to keep track of the {@link ConsumerRebalanceListener} callback - * that is invoked on the application thread. The callback process is "complicated" because it requires - * jumping between the application and the network I/O threads. - */ - private static class ConsumerRebalanceListenerCallbackBreadcrumb { - - private final ConsumerRebalanceListenerMethodName methodName; - private final CompletableFuture future; - - private ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMethodName methodName, - CompletableFuture future) { - this.methodName = Objects.requireNonNull(methodName); - this.future = Objects.requireNonNull(future); - } - } - /** * TopicPartition comparator based on topic name and partition id. */ @@ -272,14 +253,6 @@ private ConsumerRebalanceListenerCallbackBreadcrumb(ConsumerRebalanceListenerMet */ private final BackgroundEventHandler backgroundEventHandler; - /** - * Breadcrumb that we can return to as we wait for the completion of the - * {@link ConsumerRebalanceListenerCallbackNeededEvent} that was enqueued during rebalancing. This - * is an {@link AtomicReference} as it is updated by both the {@link ConsumerNetworkThread background thread} - * and the application thread. - */ - private final AtomicReference breadcrumbRef; - public MembershipManagerImpl(String groupId, SubscriptionState subscriptions, CommitRequestManager commitRequestManager, @@ -311,7 +284,6 @@ public MembershipManagerImpl(String groupId, this.currentAssignment = new HashSet<>(); this.log = logContext.logger(MembershipManagerImpl.class); this.backgroundEventHandler = backgroundEventHandler; - this.breadcrumbRef = new AtomicReference<>(); } /** @@ -697,7 +669,7 @@ boolean reconcile() { "\tCurrent owned partitions: {}\n" + "\tAdded partitions (assigned - owned): {}\n" + "\tRevoked partitions (owned - assigned): {}\n", - assignedTopicPartitions, + assignedTopicIdPartitions, ownedPartitions, addedPartitions, revokedPartitions @@ -1004,7 +976,7 @@ private CompletableFuture invokeOnPartitionsRevokedCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsRevoked.isEmpty() && listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(onPartitionsRevoked, partitionsRevoked); + return enqueueConsumerRebalanceListenerCallback(ON_PARTITIONS_REVOKED, partitionsRevoked); } else { return CompletableFuture.completedFuture(null); } @@ -1015,7 +987,7 @@ private CompletableFuture invokeOnPartitionsAssignedCallback(Set listener = subscriptions.rebalanceListener(); if (listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(onPartitionsAssigned, partitionsAssigned); + return enqueueConsumerRebalanceListenerCallback(ON_PARTITIONS_ASSIGNED, partitionsAssigned); } else { return CompletableFuture.completedFuture(null); } @@ -1026,7 +998,7 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set listener = subscriptions.rebalanceListener(); if (!partitionsLost.isEmpty() && listener.isPresent()) { - return enqueueConsumerRebalanceListenerCallback(onPartitionsLost, partitionsLost); + return enqueueConsumerRebalanceListenerCallback(ON_PARTITIONS_LOST, partitionsLost); } else { return CompletableFuture.completedFuture(null); } @@ -1039,15 +1011,8 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set * - * This method is essentially "giving" the baton from the background thread to the application thread for - * processing of the reconciliation logic. It will "receive" the "baton" back via the - * {@link #consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName, Optional)} method. - * - *

    - * * Because the reconciliation process (run in the background thread) will be blocked by the application thread - * until it completes this, we need to leave a {@link ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb} - * by which to remember where we left off. + * until it completes this, we need to provide a {@link CompletableFuture} by which to remember where we left off. * * @param methodName Callback method that needs to be executed on the application thread * @param partitions Partitions to supply to the callback method @@ -1055,93 +1020,34 @@ private CompletableFuture invokeOnPartitionsLostCallback(Set enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName methodName, Set partitions) { - CompletableFuture future = new CompletableFuture<>(); - ConsumerRebalanceListenerCallbackBreadcrumb newBreadcrumb = new ConsumerRebalanceListenerCallbackBreadcrumb( - methodName, - future - ); - - if (breadcrumbRef.compareAndSet(null, newBreadcrumb)) { - // This is the happy path—there isn't an existing breadcrumb, so we can schedule our new event - // without hesitation. - SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - sortedPartitions.addAll(partitions); - BackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); - backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); - } else { - ConsumerRebalanceListenerCallbackBreadcrumb unexpected = breadcrumbRef.get(); - - // If our CAS above failed it's because the breadcrumb wasn't null, that is, there was already a - // breadcrumb set. For our error, we try to get some info from that existing breadcrumb for better - // debugging. But there's technically a race condition, which means existingBreadcrumb could be null - // by the time we try to access it. In that case, we just use a generic name. - String unexpectedMethodName = unexpected != null ? unexpected.methodName.toString() : "another invocation"; - - // In this case, there was already an existing breadcrumb, so we need to report the matter back to the user. - String s = "An internal error occurred: an attempt to schedule the " + - methodName + " method for execution during rebalancing failed because " + - unexpectedMethodName + " was already scheduled"; - future.completeExceptionally(new KafkaException(s)); - } - - return future; + SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + sortedPartitions.addAll(partitions); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); + backgroundEventHandler.add(event); + log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); + return event.future(); } - /** - * Signals that a {@link ConsumerRebalanceListener} callback has completed. This is invoked when the - * application thread has completed the callback and has submitted a - * {@link ConsumerRebalanceListenerCallbackCompletedEvent} to the network I/O thread. At this point, we - * notify the state machine that it's complete so that it can move to the next appropriate step of the - * rebalance process. - * - *

    - * - * This method is "receiving" the baton back from the application thread after having "given" it to the - * application thread via the - * {@link #enqueueConsumerRebalanceListenerCallback(ConsumerRebalanceListenerMethodName, Set)} method. - * - * @param methodName Method name of the callback that was executed - * @param error Optional error that was thrown by the callback, captured, and forwarded here - */ @Override - public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerMethodName methodName, - Optional error) { - // Get any existing breadcrumb out and clear the state for the next reconciliation. - ConsumerRebalanceListenerCallbackBreadcrumb breadcrumb = breadcrumbRef.getAndSet(null); - - if (breadcrumb == null) { - // Case 1: we're somehow completing a callback for which we don't have a recorded breadcrumb. - // Because of that, we don't have a Future that can be completed, so we're left having to report it - // back to the user asynchronously. - String s = "An internal error occurred: the " + methodName + " method was executed " + - "during rebalancing, but there was no record of it being scheduled"; - backgroundEventHandler.add(new ErrorBackgroundEvent(new KafkaException(s))); - } else if (breadcrumb.methodName != methodName) { - // Case 2: we have a breadcrumb, but it does NOT match the one we expect. We need to abort the - // rebalance process, because we're in an inconsistent state. We do that by completing the Future - // with an error. - String s = "An internal error occurred: an attempt to continue rebalance after the execution of the " + - methodName + " method failed because the expected method was " + breadcrumb.methodName; - breadcrumb.future.completeExceptionally(new KafkaException(s)); + public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListenerCallbackCompletedEvent event) { + ConsumerRebalanceListenerMethodName methodName = event.methodName(); + Optional error = event.error(); + CompletableFuture future = event.future(); + + if (error.isPresent()) { + log.warn( + "The {} method completed with an error ({}). signaling to continue to the next phase of rebalance", + methodName, + error.get().getMessage() + ); } else { - // Case 3: the happy path. We have a breadcrumb that matches what we expect, so we can proceed to - // the next step of the rebalance process. - if (error.isPresent()) { - log.warn( - "The {} method completed with an error ({}). signaling to continue to the next phase of rebalance", - methodName, - error.get().getMessage() - ); - } else { - log.debug( - "The {} method completed successfully; signaling to continue to the next phase of rebalance", - methodName - ); - } - - breadcrumb.future.complete(null); + log.debug( + "The {} method completed successfully; signaling to continue to the next phase of rebalance", + methodName + ); } + + future.complete(null); } /** 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 8e5787edc06cc..7e35878f938f6 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 @@ -230,7 +230,7 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event return; } MembershipManager manager = requestManagers.heartbeatRequestManager.get().membershipManager(); - manager.consumerRebalanceListenerCallbackCompleted(event.methodName(), event.error()); + manager.consumerRebalanceListenerCallbackCompleted(event); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java index 08a86559e759b..d8a2237ded561 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -108,15 +108,15 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final Exception e; switch (methodName) { - case onPartitionsRevoked: + case ON_PARTITIONS_REVOKED: e = rebalanceListenerInvoker.invokePartitionsRevoked(partitions); break; - case onPartitionsAssigned: + case ON_PARTITIONS_ASSIGNED: e = rebalanceListenerInvoker.invokePartitionsAssigned(partitions); break; - case onPartitionsLost: + case ON_PARTITIONS_LOST: e = rebalanceListenerInvoker.invokePartitionsLost(partitions); break; @@ -135,7 +135,11 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { error = Optional.empty(); } - ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent(methodName, partitions, error); + ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent( + methodName, + partitions, + event.future(), + error); applicationEventHandler.add(invokedEvent); } } 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 new file mode 100644 index 0000000000000..61ec322cb3a55 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -0,0 +1,86 @@ +/* + * 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.utils.Timer; + +import java.util.concurrent.CompletableFuture; + +/** + * Background event with a result in the form of a future, that can be retrieved within a + * timeout based on completion. + * + * @param + */ +public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { + + private final CompletableFuture future; + + protected CompletableBackgroundEvent(Type type) { + super(type); + this.future = new CompletableFuture<>(); + } + + 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 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; + } + + @Override + protected String toStringBase() { + 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 7bd71b91b238d..96ec17a8538d8 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 @@ -25,6 +25,7 @@ import java.util.Objects; import java.util.Optional; import java.util.SortedSet; +import java.util.concurrent.CompletableFuture; /** * Event that signifies that the application thread has executed the {@link ConsumerRebalanceListener} callback. If @@ -34,14 +35,17 @@ public class ConsumerRebalanceListenerCallbackCompletedEvent extends Application private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; + private final CompletableFuture future; private final Optional error; public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, SortedSet partitions, + CompletableFuture future, Optional error) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); + this.future = Objects.requireNonNull(future); this.error = Objects.requireNonNull(error); } @@ -53,6 +57,10 @@ public SortedSet partitions() { return partitions; } + public CompletableFuture future() { + return future; + } + public Optional error() { return error; } @@ -65,12 +73,15 @@ public boolean equals(Object o) { ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - return methodName == that.methodName && partitions.equals(that.partitions) && error.equals(that.error); + return methodName == that.methodName && + partitions.equals(that.partitions) && + future.equals(this.future) && + error.equals(that.error); } @Override public int hashCode() { - return Objects.hash(methodName, partitions, error); + return Objects.hash(methodName, partitions, future, error); } @Override @@ -78,6 +89,7 @@ protected String toStringBase() { return super.toStringBase() + ", methodName=" + methodName + ", partitions=" + partitions + + ", future=" + future + ", error=" + error; } 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 754f8e7c96399..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 @@ -32,7 +32,7 @@ * {@link Consumer#poll(Duration)} call is performed by the user. When processed, the application thread should * invoke the appropriate callback method (based on {@link #methodName()}) with the given partitions. */ -public class ConsumerRebalanceListenerCallbackNeededEvent extends BackgroundEvent { +public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBackgroundEvent { private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; @@ -65,7 +65,10 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(methodName, partitions); + int result = super.hashCode(); + result = 31 * result + methodName.hashCode(); + result = 31 * result + partitions.hashCode(); + return result; } @Override From 45cf7f5dd675a5394ee2bced838bc1c009ca8809 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 4 Dec 2023 15:18:42 -0800 Subject: [PATCH 44/68] Clean up --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 ++-- .../kafka/clients/consumer/internals/MembershipManager.java | 1 - .../ConsumerRebalanceListenerCallbackCompletedEvent.java | 2 +- 3 files changed, 3 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 08cfe78e06554..80036617f8a66 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 @@ -136,7 +136,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private final ApplicationEventHandler applicationEventHandler; private final Time time; - private final Optional groupMetadata; + private Optional groupMetadata; private final KafkaConsumerMetrics kafkaConsumerMetrics; private Logger log; private final String clientId; @@ -1477,7 +1477,7 @@ private boolean processBackgroundEvents(CompletableApplicationEvent event, Ti } catch (TimeoutException e) { // Ignore this as we will retry the event until the timeout expires. } finally { - timer.update(time.milliseconds()); + timer.update(); } } while (timer.notExpired()); 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 88a697dc74236..c6f6e18a192be 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 @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; 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 96ec17a8538d8..a065b5bec27c2 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 @@ -75,7 +75,7 @@ public boolean equals(Object o) { return methodName == that.methodName && partitions.equals(that.partitions) && - future.equals(this.future) && + future.equals(that.future) && error.equals(that.error); } From 730ee8654e6eef32b8fe6b7e5a8e97bcdd700d7c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 4 Dec 2023 17:16:23 -0800 Subject: [PATCH 45/68] Changed a period to a semicolon in a log message --- .../kafka/clients/consumer/internals/MembershipManagerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 46024f973bcd8..f8be333695f02 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 @@ -1036,7 +1036,7 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener if (error.isPresent()) { log.warn( - "The {} method completed with an error ({}). signaling to continue to the next phase of rebalance", + "The {} method completed with an error ({}); signaling to continue to the next phase of rebalance", methodName, error.get().getMessage() ); From 7e00f6d26ed040c83e757158a1b8bb558d46a31c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 4 Dec 2023 18:05:51 -0800 Subject: [PATCH 46/68] Minor clean up and comments --- .../consumer/internals/HeartbeatRequestManager.java | 5 +++++ .../consumer/internals/MembershipManagerImpl.java | 11 ++++++----- 2 files changed, 11 insertions(+), 5 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 99f75c2319be4..9bd950fd39041 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.TopicIdPartition; @@ -183,6 +184,10 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request)); } + /** + * Returns the {@link MembershipManager} that this request manager is using to track the state of the group. + * This is provided so that the {@link ApplicationEventProcessor} can access the state for querying or updating. + */ public MembershipManager membershipManager() { return membershipManager; } 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 f8be333695f02..0bca3c277b592 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 @@ -1035,15 +1035,16 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener CompletableFuture future = event.future(); if (error.isPresent()) { + String message = error.get().getMessage(); log.warn( - "The {} method completed with an error ({}); signaling to continue to the next phase of rebalance", - methodName, - error.get().getMessage() + "The {} method completed with an error ({}); signaling to continue to the next phase of rebalance", + methodName, + message ); } else { log.debug( - "The {} method completed successfully; signaling to continue to the next phase of rebalance", - methodName + "The {} method completed successfully; signaling to continue to the next phase of rebalance", + methodName ); } From 8e04846ecaee5ce5219886b741511f537b37f0fb Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 4 Dec 2023 18:06:14 -0800 Subject: [PATCH 47/68] Added unit tests for ConsumerRebalanceListener callbacks since the integration tests are disabled --- .../internals/MembershipManagerImplTest.java | 243 ++++++++++++++++++ 1 file changed, 243 insertions(+) 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 971fb89eafb35..7f0fb5ee47516 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 @@ -17,18 +17,28 @@ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -36,19 +46,28 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; 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.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; 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.anyBoolean; import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anySet; import static org.mockito.Mockito.clearInvocations; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -67,6 +86,7 @@ public class MembershipManagerImplTest { private CommitRequestManager commitRequestManager; private ConsumerTestBuilder testBuilder; + private BlockingQueue backgroundEventQueue; @BeforeEach public void setup() { @@ -74,6 +94,7 @@ public void setup() { metadata = testBuilder.metadata; subscriptionState = testBuilder.subscriptions; commitRequestManager = testBuilder.commitRequestManager.get(); + backgroundEventQueue = testBuilder.backgroundEventQueue; } @AfterEach @@ -791,6 +812,169 @@ public void testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() { verify(membershipManager, never()).transitionToJoining(); } + @Test + public void testListenerCallbacksBasic() { + // Step 1: set up mocks + MembershipManagerImpl membershipManager = createMemberInStableState(); + mockOwnedPartition("topic1", 0); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + + // Step 2: put the state machine into the appropriate... state + receiveEmptyAssignment(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + assertTrue(membershipManager.reconciliationInProgress()); + listener.assertCounts(0, 0, 0); + + // Step 2: revoke partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(new TopicPartition("topic1", 0)) + ); + + // Step 3: assign partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet() + ); + + // Step 4: Receive ack and make sure we're done and our listener was called appropriately + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager.state()); + + listener.assertCounts(1, 1, 0); + } + + @Test + public void testListenerCallbacksNoListeners() { + // Step 1: set up mocks + MembershipManagerImpl membershipManager = createMemberInStableState(); + mockOwnedPartition("topic1", 0); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + + // Step 2: put the state machine into the appropriate... state + receiveEmptyAssignment(membershipManager); + assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + assertFalse(membershipManager.reconciliationInProgress()); + assertEquals(0, backgroundEventQueue.size()); + listener.assertCounts(0, 0, 0); + + // Step 3: Receive ack and make sure we're done and our listener was called appropriately + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager.state()); + + listener.assertCounts(0, 0, 0); + } + + @Test + public void testOnPartitionsLostNoError() { + mockOwnedPartition("topic1", 0); + testOnPartitionsLost(Optional.empty()); + } + + @Test + public void testOnPartitionsLostError() { + mockOwnedPartition("topic1", 0); + testOnPartitionsLost(Optional.of(new KafkaException("Intentional error for test"))); + } + + private void testOnPartitionsLost(Optional lostError) { + // Step 1: set up mocks + MembershipManagerImpl membershipManager = createMemberInStableState(); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( + Optional.empty(), + Optional.empty(), + lostError + ); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + + // Step 2: put the state machine into the appropriate... state + membershipManager.transitionToFenced(); + assertEquals(MemberState.FENCED, membershipManager.state()); + assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + listener.assertCounts(0, 0, 0); + + // Step 3: invoke the callback + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, + topicPartitions(new TopicPartition("topic1", 0)) + ); + + // Step 4: Receive ack and make sure we're done and our listener was called appropriately + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager.state()); + + listener.assertCounts(0, 0, 1); + } + + private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { + ConsumerCoordinatorMetrics coordinatorMetrics = new ConsumerCoordinatorMetrics( + subscriptionState, + new Metrics(), + "test-"); + return new ConsumerRebalanceListenerInvoker( + new LogContext(), + subscriptionState, + new MockTime(1), + coordinatorMetrics + ); + } + + private SortedSet topicPartitions(TopicPartition... topicPartitions) { + SortedSet revokedPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); + revokedPartitions.addAll(Arrays.asList(topicPartitions)); + return revokedPartitions; + } + + private void performCallback(MembershipManagerImpl membershipManager, + ConsumerRebalanceListenerInvoker invoker, + ConsumerRebalanceListenerMethodName methodName, + SortedSet partitions) { + // Set up our mock application event handler & background event processor. + ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); + + doAnswer(a -> { + ConsumerRebalanceListenerCallbackCompletedEvent completedEvent = a.getArgument(0); + membershipManager.consumerRebalanceListenerCallbackCompleted(completedEvent); + return null; + }).when(applicationEventHandler).add(any(ConsumerRebalanceListenerCallbackCompletedEvent.class)); + + BackgroundEventProcessor backgroundEventProcessor = new BackgroundEventProcessor( + new LogContext(), + backgroundEventQueue, + applicationEventHandler, + invoker + ); + + // We expect only our enqueued event in the background queue. + assertEquals(1, backgroundEventQueue.size()); + assertNotNull(backgroundEventQueue.peek()); + assertInstanceOf(ConsumerRebalanceListenerCallbackNeededEvent.class, backgroundEventQueue.peek()); + ConsumerRebalanceListenerCallbackNeededEvent neededEvent = (ConsumerRebalanceListenerCallbackNeededEvent) backgroundEventQueue.poll(); + assertNotNull(neededEvent); + assertEquals(methodName, neededEvent.methodName()); + assertEquals(partitions, neededEvent.partitions()); + + backgroundEventProcessor.process(neededEvent); + } + private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( Uuid topicId, String topicName, List partitions) { mockOwnedPartitionAndAssignmentReceived(topicId, topicName, Collections.emptySet(), true); @@ -1092,4 +1276,63 @@ private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean m .setPartitions(Arrays.asList(3, 4, 5)) )); } + + private static class CounterConsumerRebalanceListener implements ConsumerRebalanceListener { + + private final AtomicInteger revokedCounter = new AtomicInteger(); + private final AtomicInteger assignedCounter = new AtomicInteger(); + private final AtomicInteger lostCounter = new AtomicInteger(); + + private final Optional revokedError; + private final Optional assignedError; + private final Optional lostError; + + public CounterConsumerRebalanceListener() { + this(Optional.empty(), Optional.empty(), Optional.empty()); + } + + public CounterConsumerRebalanceListener(Optional revokedError, + Optional assignedError, + Optional lostError) { + this.revokedError = revokedError; + this.assignedError = assignedError; + this.lostError = lostError; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + try { + if (revokedError.isPresent()) + throw revokedError.get(); + } finally { + revokedCounter.incrementAndGet(); + } + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + try { + if (assignedError.isPresent()) + throw assignedError.get(); + } finally { + assignedCounter.incrementAndGet(); + } + } + + @Override + public void onPartitionsLost(Collection partitions) { + try { + if (lostError.isPresent()) + throw lostError.get(); + } finally { + lostCounter.incrementAndGet(); + } + } + + public void assertCounts(int revokedCount, int assignedCount, int lostCount) { + assertEquals(revokedCount, revokedCounter.get()); + assertEquals(assignedCount, assignedCounter.get()); + assertEquals(lostCount, lostCounter.get()); + } + } } From 2096fcdcc238fd385139c56dc8339f2972b30cbe Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 5 Dec 2023 10:25:41 -0800 Subject: [PATCH 48/68] Fixed issue with BackgroundEventHandlerTest that no longer compiled; switched away from ConsumerTestBuilder --- .../events/BackgroundEventHandlerTest.java | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java index 6ef2c1f64a436..e1a77ca71c569 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java @@ -16,39 +16,39 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder; +import org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerInvoker; import org.apache.kafka.common.KafkaException; -import org.junit.jupiter.api.AfterEach; +import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.concurrent.BlockingQueue; +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.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; public class BackgroundEventHandlerTest { - private ConsumerTestBuilder.ApplicationEventHandlerTestBuilder testBuilder; private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private BackgroundEventProcessor backgroundEventProcessor; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder.ApplicationEventHandlerTestBuilder(); - backgroundEventQueue = testBuilder.backgroundEventQueue; - backgroundEventHandler = testBuilder.backgroundEventHandler; - backgroundEventProcessor = testBuilder.backgroundEventProcessor; - } - - @AfterEach - public void tearDown() { - if (testBuilder != null) - testBuilder.close(); + LogContext logContext = new LogContext(); + backgroundEventQueue = new LinkedBlockingQueue<>(); + backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); + backgroundEventProcessor = new BackgroundEventProcessor( + logContext, + backgroundEventQueue, + mock(ApplicationEventHandler.class), + mock(ConsumerRebalanceListenerInvoker.class) + ); } @Test From 3f7074355b8e593b4216e7ca50b305fee0c4fac7 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 6 Dec 2023 12:30:41 -0800 Subject: [PATCH 49/68] Minor tweak to debug logging for ConsumerRebalanceListener callback method name --- .../consumer/internals/AsyncKafkaConsumer.java | 2 +- .../ConsumerRebalanceListenerMethodName.java | 13 ------------- .../consumer/internals/MembershipManagerImpl.java | 6 +++--- 3 files changed, 4 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cc40681132cdc..ed7eea0a84471 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 @@ -248,7 +248,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { break; default: - throw new IllegalArgumentException("The method " + methodName + " to invoke was not expected"); + throw new IllegalArgumentException("The method " + methodName.fullyQualifiedMethodName() + " to invoke was not expected"); } final Optional error; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java index 1a25ecefa7a7a..151c0e03af1d6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerMethodName.java @@ -28,25 +28,12 @@ public enum ConsumerRebalanceListenerMethodName { ON_PARTITIONS_ASSIGNED("onPartitionsAssigned"), ON_PARTITIONS_LOST("onPartitionsLost"); - private final String methodName; - private final String fullyQualifiedMethodName; ConsumerRebalanceListenerMethodName(String methodName) { - this.methodName = methodName; this.fullyQualifiedMethodName = String.format("%s.%s", ConsumerRebalanceListener.class.getSimpleName(), methodName); } - /** - * Provides the method name, e.g. {@code onPartitionsRevoked}. - * - * @return Method name - */ - - public String methodName() { - return methodName; - } - /** * Provides the fully-qualified method name, e.g. {@code ConsumerRebalanceListener.onPartitionsRevoked}. This * is used for log messages. 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 74ab3e4eb38a8..6507c2c24ec87 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 @@ -1015,7 +1015,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume sortedPartitions.addAll(partitions); CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); - log.debug("The event to trigger the {} method execution was enqueued successfully", methodName); + log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); return event.future(); } @@ -1029,13 +1029,13 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener String message = error.get().getMessage(); log.warn( "The {} method completed with an error ({}); signaling to continue to the next phase of rebalance", - methodName, + methodName.fullyQualifiedMethodName(), message ); } else { log.debug( "The {} method completed successfully; signaling to continue to the next phase of rebalance", - methodName + methodName.fullyQualifiedMethodName() ); } From 5009ba8a260ecfc993a8115d4353b406b47ca485 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Dec 2023 15:34:06 -0800 Subject: [PATCH 50/68] Moved the client telemetry stuff out of the membership manager and to the Consumer layer --- .../internals/AsyncKafkaConsumer.java | 15 ++++++++++++++ .../internals/MembershipManagerImpl.java | 20 +------------------ .../consumer/internals/RequestManagers.java | 3 +-- .../internals/ConsumerTestBuilder.java | 3 +-- .../internals/MembershipManagerImplTest.java | 8 ++++---- 5 files changed, 22 insertions(+), 27 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 907faba076506..a15fd4d980bd8 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 @@ -76,6 +76,7 @@ import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils; import org.apache.kafka.common.utils.AppInfoParser; @@ -220,12 +221,26 @@ private void process(final ErrorBackgroundEvent event) { private void process(final GroupMetadataUpdateEvent event) { if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); + final String currentMemberId = currentGroupMetadata.memberId(); AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( currentGroupMetadata.groupId(), event.memberEpoch(), event.memberId(), currentGroupMetadata.groupInstanceId() )); + + // Update the group member id label in the client telemetry reporter if the member id has + // changed. Initially the member id is empty, and it is updated when the member joins the + // group. This is done here to avoid updating the label on every heartbeat response. Also + // check if the member id is null, as the schema defines it as nullable. + if (!currentMemberId.equals(event.memberId()) && clientTelemetryReporter.isPresent()) { + Map labels = Collections.singletonMap( + ClientTelemetryProvider.GROUP_MEMBER_ID, + event.memberId() + ); + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.updateMetricsLabels(labels); + } } } 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 bab0898a090ac..8ea164ee102f2 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 @@ -255,13 +255,6 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private final BackgroundEventHandler backgroundEventHandler; - /** - * Optional client telemetry reporter which sends client telemetry data to the broker. This - * will be empty if the client telemetry feature is not enabled. This is provided to update - * the group member id label when the member joins the group. - */ - private final Optional clientTelemetryReporter; - public MembershipManagerImpl(String groupId, Optional groupInstanceId, Optional serverAssignor, @@ -269,8 +262,7 @@ public MembershipManagerImpl(String groupId, CommitRequestManager commitRequestManager, ConsumerMetadata metadata, LogContext logContext, - BackgroundEventHandler backgroundEventHandler, - Optional clientTelemetryReporter) { + BackgroundEventHandler backgroundEventHandler) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -284,7 +276,6 @@ public MembershipManagerImpl(String groupId, this.currentAssignment = new HashSet<>(); this.log = logContext.logger(MembershipManagerImpl.class); this.backgroundEventHandler = backgroundEventHandler; - this.clientTelemetryReporter = clientTelemetryReporter; } /** @@ -347,15 +338,6 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo throw new IllegalArgumentException(errorMessage); } - // Update the group member id label in the client telemetry reporter if the member id has - // changed. Initially the member id is empty, and it is updated when the member joins the - // group. This is done here to avoid updating the label on every heartbeat response. Also - // check if the member id is null, as the schema defines it as nullable. - if (response.memberId() != null && !response.memberId().equals(memberId)) { - clientTelemetryReporter.ifPresent(reporter -> reporter.updateMetricsLabels( - Collections.singletonMap(ClientTelemetryProvider.GROUP_MEMBER_ID, response.memberId()))); - } - this.memberId = response.memberId(); this.memberEpoch = response.memberEpoch(); ConsumerGroupHeartbeatResponseData.Assignment assignment = response.assignment(); 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 60afaa6d34eda..7b48a5196d159 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 @@ -176,8 +176,7 @@ protected RequestManagers create() { commit, metadata, logContext, - backgroundEventHandler, - clientTelemetryReporter); + backgroundEventHandler); heartbeatRequestManager = new HeartbeatRequestManager( logContext, time, 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 ade75819379ac..d455a28d07ea6 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 @@ -205,8 +205,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA commit, metadata, logContext, - backgroundEventHandler, - Optional.empty() + backgroundEventHandler ) ); HeartbeatRequestManager.HeartbeatState heartbeatState = spy(new HeartbeatRequestManager.HeartbeatState( 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 01149da823d8a..92ff205b1f656 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 @@ -107,7 +107,7 @@ public void tearDown() { private MembershipManagerImpl createMembershipManagerJoiningGroup() { MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, - metadata, testBuilder.logContext, testBuilder.backgroundEventHandler, Optional.empty())); + metadata, testBuilder.logContext, testBuilder.backgroundEventHandler)); manager.transitionToJoining(); return manager; } @@ -117,7 +117,7 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, metadata, testBuilder.logContext, - testBuilder.backgroundEventHandler, Optional.empty()); + testBuilder.backgroundEventHandler); manager.transitionToJoining(); return manager; } @@ -142,7 +142,7 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() // First join should register to get metadata updates MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, - metadata, testBuilder.logContext, testBuilder.backgroundEventHandler, Optional.empty()); + metadata, testBuilder.logContext, testBuilder.backgroundEventHandler); manager.transitionToJoining(); verify(metadata).addClusterUpdateListener(manager); clearInvocations(metadata); @@ -220,7 +220,7 @@ public void testTransitionToFatal() { public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, metadata, - testBuilder.logContext, testBuilder.backgroundEventHandler, Optional.empty()); + testBuilder.logContext, testBuilder.backgroundEventHandler); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); From 477b5d6b87b35eaf77975b89c570cdbd56de54c0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Dec 2023 16:55:12 -0800 Subject: [PATCH 51/68] Minor tweaks to the event processor APIs --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 3 +-- .../kafka/clients/consumer/internals/ConsumerUtils.java | 4 ++-- .../consumer/internals/events/ApplicationEventProcessor.java | 3 +-- .../clients/consumer/internals/events/EventProcessor.java | 2 -- 4 files changed, 4 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 a15fd4d980bd8..81ab9f371a4cb 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 @@ -173,8 +173,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 void process() { + private void process() { AtomicReference firstError = new AtomicReference<>(); ProcessHandler processHandler = (event, error) -> { 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 d599d41a245ee..2bde565017354 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 @@ -51,8 +51,8 @@ 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; public final class ConsumerUtils { @@ -206,7 +206,7 @@ public static void refreshCommittedOffsets(final Map T getResult(CompletableFuture future, Timer timer) { + public static T getResult(Future future, Timer timer) { try { return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); } catch (ExecutionException e) { 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 89fd8b3049b54..5f5b942725b23 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,13 +61,12 @@ 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. */ - @Override public void process() { process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override - public void process(ApplicationEvent event) { + protected void process(ApplicationEvent event) { switch (event.type()) { case COMMIT: process((CommitApplicationEvent) event); 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 64aa6f2f77b8e..0eb8b5b7a8832 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 @@ -47,8 +47,6 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } - public abstract void process(); - protected abstract void process(T event); @Override From 829af6e349fc149a2e7dee9faae77ef79f19bdc9 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Dec 2023 16:08:03 -0800 Subject: [PATCH 52/68] Updates to only pause for AsyncKafkaConsumer.processBackgroundEvents() if there were no events --- .../internals/AsyncKafkaConsumer.java | 56 ++++++++++----- .../consumer/internals/ConsumerUtils.java | 17 +++++ .../internals/MembershipManagerImpl.java | 2 - .../events/ApplicationEventProcessor.java | 6 +- .../internals/events/EventProcessor.java | 8 ++- .../internals/AsyncKafkaConsumerTest.java | 68 +++++++++++++++++++ 6 files changed, 133 insertions(+), 24 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 81ab9f371a4cb..b2c9c0586357d 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 @@ -101,6 +101,7 @@ import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -173,7 +174,8 @@ 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. */ - private void process() { + @Override + public boolean process() { AtomicReference firstError = new AtomicReference<>(); ProcessHandler processHandler = (event, error) -> { @@ -186,10 +188,12 @@ private void process() { } }; - process(processHandler); + boolean hadEvents = process(processHandler); if (firstError.get() != null) throw firstError.get(); + + return hadEvents; } @Override @@ -1316,10 +1320,12 @@ public void unsubscribe() { log.info("Unsubscribing all topics or patterns and assigned partitions"); Timer timer = time.timer(Long.MAX_VALUE); - if (processBackgroundEvents(unsubscribeApplicationEvent, timer)) + try { + processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); - else + } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); + } } subscriptions.unsubscribe(); } finally { @@ -1654,23 +1660,39 @@ private void subscribeInternal(Collection topics, Optional event, Timer timer) { - log.trace("Enqueuing event {} for processing; will wait up to {} ms to complete", event, timer.remainingMs()); + // Visible for testing + T processBackgroundEvents(EventProcessor eventProcessor, + Future future, + Timer timer) { + log.trace("Will wait up to {} ms for future {} to complete", timer.remainingMs(), future); do { - backgroundEventProcessor.process(); + boolean hadEvents = eventProcessor.process(); try { - Timer pollInterval = time.timer(100L); - log.trace("Waiting {} ms for event {} to complete", event, pollInterval.remainingMs()); - ConsumerUtils.getResult(event.future(), pollInterval); - log.trace("Event {} completed successfully", event); - return true; + if (future.isDone()) { + // 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; + } 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; + } } catch (TimeoutException e) { // Ignore this as we will retry the event until the timeout expires. } finally { @@ -1678,8 +1700,8 @@ private boolean processBackgroundEvents(CompletableApplicationEvent event, Ti } } while (timer.notExpired()); - log.trace("Event {} did not complete within timeout", event); - return false; + log.trace("Future {} did not complete within timeout", future); + throw new TimeoutException("Operation timed out before completion"); } @Override 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 2bde565017354..b198098d9da04 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 @@ -224,4 +224,21 @@ else if (t instanceof KafkaException) throw new TimeoutException(e); } } + + public static T getResult(Future future) { + try { + return future.get(); + } catch (ExecutionException e) { + Throwable t = e.getCause(); + + if (t instanceof WakeupException) + throw new WakeupException(); + else if (t instanceof KafkaException) + throw (KafkaException) t; + else + throw new KafkaException(t); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } } 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 8ea164ee102f2..e902206d0172f 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 @@ -35,8 +35,6 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; -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.Utils; import org.slf4j.Logger; 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 5f5b942725b23..50f90e6e70091 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,12 +61,12 @@ 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 void process() { - process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); + public boolean process() { + return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } @Override - protected void process(ApplicationEvent event) { + public void process(ApplicationEvent event) { switch (event.type()) { case COMMIT: process((CommitApplicationEvent) event); 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 0eb8b5b7a8832..bf23bb844acbc 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 @@ -47,6 +47,8 @@ protected EventProcessor(final LogContext logContext, final BlockingQueue eve this.closer = new IdempotentCloser(); } + public abstract boolean process(); + protected abstract void process(T event); @Override @@ -63,14 +65,14 @@ protected interface ProcessHandler { * 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 void process(ProcessHandler 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; + return false; } try { @@ -96,6 +98,8 @@ protected void process(ProcessHandler processHandler) { } finally { log.trace("Completed processing"); } + + return true; } /** 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 d39b63e979022..ffc19f16a3049 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,6 +33,7 @@ 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.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.EventProcessor; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; @@ -59,6 +60,8 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -91,6 +94,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1082,6 +1086,70 @@ public void testLongPollWaitIsLimited() { assertEquals(singleton(tp), consumer.assignment()); } + @Test + public void testProcessBackgroundEventsWithInitialDelay() throws Exception { + Time time = new MockTime(); + Timer timer = time.timer(1000); + CompletableFuture future = mock(CompletableFuture.class); + CountDownLatch latch = new CountDownLatch(3); + + doAnswer(invocation -> { + latch.countDown(); + + if (latch.getCount() > 0) { + long timeout = invocation.getArgument(0, Long.class); + timer.sleep(timeout); + throw new java.util.concurrent.TimeoutException("Intentional timeout"); + } + + future.complete(null); + return "success"; + }).when(future).get(any(Long.class), any(TimeUnit.class)); + + try (EventProcessor processor = mock(EventProcessor.class)) { + consumer.processBackgroundEvents(processor, future, timer); + assertEquals(800, timer.remainingMs()); + } + } + + @Test + public void testProcessBackgroundEventsWithoutDelay() throws Exception { + CompletableFuture future = mock(CompletableFuture.class); + + doReturn(true).when(future).isDone(); + doAnswer(invocation -> { + future.complete(null); + return null; + }).when(future).get(any(Long.class), any(TimeUnit.class)); + + try (EventProcessor processor = mock(EventProcessor.class)) { + doAnswer(invocation -> true).when(processor).process(); + + Time time = new MockTime(); + Timer timer = time.timer(1000); + consumer.processBackgroundEvents(processor, future, timer); + assertEquals(1000, timer.remainingMs()); + } + } + + @Test + public void testProcessBackgroundEventsTimesOut() throws Exception { + Time time = new MockTime(); + Timer timer = time.timer(1000); + CompletableFuture future = mock(CompletableFuture.class); + + doAnswer(invocation -> { + long timeout = invocation.getArgument(0, Long.class); + timer.sleep(timeout); + 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)); + assertEquals(0, timer.remainingMs()); + } + } + private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) { assertNull(wakeupTrigger.getPendingTask()); } From c4a97b76574b2632f71b5bba3025ffaa37e683ac Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Dec 2023 17:32:24 -0800 Subject: [PATCH 53/68] Updates to tests for rebalance callbacks --- .../internals/AsyncKafkaConsumer.java | 134 ++++++++++-------- .../internals/MembershipManagerImpl.java | 10 +- ...balanceListenerCallbackCompletedEvent.java | 14 +- .../internals/AsyncKafkaConsumerTest.java | 40 ++++-- .../internals/MembershipManagerImplTest.java | 42 +----- 5 files changed, 114 insertions(+), 126 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 b2c9c0586357d..e8c1bd979f086 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 @@ -222,69 +222,16 @@ private void process(final ErrorBackgroundEvent event) { } private void process(final GroupMetadataUpdateEvent event) { - if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { - final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); - final String currentMemberId = currentGroupMetadata.memberId(); - AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - event.memberEpoch(), - event.memberId(), - currentGroupMetadata.groupInstanceId() - )); - - // Update the group member id label in the client telemetry reporter if the member id has - // changed. Initially the member id is empty, and it is updated when the member joins the - // group. This is done here to avoid updating the label on every heartbeat response. Also - // check if the member id is null, as the schema defines it as nullable. - if (!currentMemberId.equals(event.memberId()) && clientTelemetryReporter.isPresent()) { - Map labels = Collections.singletonMap( - ClientTelemetryProvider.GROUP_MEMBER_ID, - event.memberId() - ); - ClientTelemetryReporter reporter = clientTelemetryReporter.get(); - reporter.updateMetricsLabels(labels); - } - } + updateConsumerGroupMetadata(event.memberId(), event.memberEpoch()); } private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { - SortedSet partitions = event.partitions(); - ConsumerRebalanceListenerMethodName methodName = event.methodName(); - final Exception e; - - switch (methodName) { - case ON_PARTITIONS_REVOKED: - e = rebalanceListenerInvoker.invokePartitionsRevoked(partitions); - break; - - case ON_PARTITIONS_ASSIGNED: - e = rebalanceListenerInvoker.invokePartitionsAssigned(partitions); - break; - - case ON_PARTITIONS_LOST: - e = rebalanceListenerInvoker.invokePartitionsLost(partitions); - break; - - default: - throw new IllegalArgumentException("The method " + methodName.fullyQualifiedMethodName() + " to invoke was not expected"); - } - - final Optional error; - - if (e != null) { - if (e instanceof KafkaException) - error = Optional.of((KafkaException) e); - else - error = Optional.of(new KafkaException("User rebalance callback throws an error", e)); - } else { - error = Optional.empty(); - } - - ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent( - methodName, - partitions, - event.future(), - error); + ApplicationEvent invokedEvent = invokeRebalanceCallbacks( + rebalanceListenerInvoker, + event.methodName(), + event.partitions(), + event.future() + ); applicationEventHandler.add(invokedEvent); } } @@ -1704,6 +1651,73 @@ T processBackgroundEvents(EventProcessor eventProcessor, throw new TimeoutException("Operation timed out before completion"); } + void updateConsumerGroupMetadata(String newMemberId, int newMemberEpoch) { + if (groupMetadata.isPresent()) { + final ConsumerGroupMetadata currentGroupMetadata = groupMetadata.get(); + final String oldMemberId = currentGroupMetadata.memberId(); + this.groupMetadata = Optional.of(new ConsumerGroupMetadata( + currentGroupMetadata.groupId(), + newMemberEpoch, + newMemberId, + currentGroupMetadata.groupInstanceId() + )); + + // Update the group member id label in the client telemetry reporter if the member id has + // changed. Initially the member id is empty, and it is updated when the member joins the + // group. This is done here to avoid updating the label on every heartbeat response. Also + // check if the member id is null, as the schema defines it as nullable. + if (!oldMemberId.equals(newMemberId) && clientTelemetryReporter.isPresent()) { + Map labels = Collections.singletonMap( + ClientTelemetryProvider.GROUP_MEMBER_ID, + newMemberId + ); + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.updateMetricsLabels(labels); + } + } + } + + static ConsumerRebalanceListenerCallbackCompletedEvent invokeRebalanceCallbacks(ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, + ConsumerRebalanceListenerMethodName methodName, + SortedSet partitions, + CompletableFuture future) { + final Exception e; + + switch (methodName) { + case ON_PARTITIONS_REVOKED: + e = rebalanceListenerInvoker.invokePartitionsRevoked(partitions); + break; + + case ON_PARTITIONS_ASSIGNED: + e = rebalanceListenerInvoker.invokePartitionsAssigned(partitions); + break; + + case ON_PARTITIONS_LOST: + e = rebalanceListenerInvoker.invokePartitionsLost(partitions); + break; + + default: + throw new IllegalArgumentException("The method " + methodName.fullyQualifiedMethodName() + " to invoke was not expected"); + } + + final Optional error; + + if (e != null) { + if (e instanceof KafkaException) + error = Optional.of((KafkaException) e); + else + error = Optional.of(new KafkaException("User rebalance callback throws an error", e)); + } else { + error = Optional.empty(); + } + + return new ConsumerRebalanceListenerCallbackCompletedEvent( + methodName, + future, + error + ); + } + @Override public String clientId() { return clientId; 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 e902206d0172f..d27e57820999d 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 @@ -1027,20 +1027,22 @@ public void consumerRebalanceListenerCallbackCompleted(ConsumerRebalanceListener CompletableFuture future = event.future(); if (error.isPresent()) { - String message = error.get().getMessage(); + Exception e = error.get(); log.warn( "The {} method completed with an error ({}); signaling to continue to the next phase of rebalance", methodName.fullyQualifiedMethodName(), - message + e.getMessage() ); + + future.completeExceptionally(e); } else { log.debug( "The {} method completed successfully; signaling to continue to the next phase of rebalance", methodName.fullyQualifiedMethodName() ); - } - future.complete(null); + future.complete(null); + } } /** 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 a065b5bec27c2..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 @@ -18,13 +18,10 @@ 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.KafkaException; -import java.util.Collections; import java.util.Objects; import java.util.Optional; -import java.util.SortedSet; import java.util.concurrent.CompletableFuture; /** @@ -34,17 +31,14 @@ public class ConsumerRebalanceListenerCallbackCompletedEvent extends ApplicationEvent { private final ConsumerRebalanceListenerMethodName methodName; - private final SortedSet partitions; private final CompletableFuture future; private final Optional error; public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions, CompletableFuture future, Optional error) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); this.methodName = Objects.requireNonNull(methodName); - this.partitions = Collections.unmodifiableSortedSet(partitions); this.future = Objects.requireNonNull(future); this.error = Objects.requireNonNull(error); } @@ -53,10 +47,6 @@ public ConsumerRebalanceListenerMethodName methodName() { return methodName; } - public SortedSet partitions() { - return partitions; - } - public CompletableFuture future() { return future; } @@ -74,21 +64,19 @@ public boolean equals(Object o) { ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; return methodName == that.methodName && - partitions.equals(that.partitions) && future.equals(that.future) && error.equals(that.error); } @Override public int hashCode() { - return Objects.hash(methodName, partitions, future, error); + return Objects.hash(methodName, future, error); } @Override protected String toStringBase() { return super.toStringBase() + ", methodName=" + methodName + - ", partitions=" + partitions + ", future=" + future + ", error=" + error; } 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 ffc19f16a3049..221c09b9b84b5 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 @@ -1086,6 +1086,10 @@ public void testLongPollWaitIsLimited() { assertEquals(singleton(tp), consumer.assignment()); } + /** + * 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 public void testProcessBackgroundEventsWithInitialDelay() throws Exception { Time time = new MockTime(); @@ -1093,6 +1097,9 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { CompletableFuture future = mock(CompletableFuture.class); CountDownLatch latch = new CountDownLatch(3); + // Mock our call to Future.get(timeout) so that it mimics a delay of 200 milliseconds. Keep in mind that + // the incremental timeout inside processBackgroundEvents is 100 seconds for each pass. Our first two passes + // will exceed the incremental timeout, but the third will return. doAnswer(invocation -> { latch.countDown(); @@ -1103,35 +1110,42 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { } future.complete(null); - return "success"; + return null; }).when(future).get(any(Long.class), any(TimeUnit.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. assertEquals(800, timer.remainingMs()); } } + /** + * 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 - public void testProcessBackgroundEventsWithoutDelay() throws Exception { - CompletableFuture future = mock(CompletableFuture.class); + public void testProcessBackgroundEventsWithoutDelay() { + Time time = new MockTime(); + Timer timer = time.timer(1000); - doReturn(true).when(future).isDone(); - doAnswer(invocation -> { - future.complete(null); - return null; - }).when(future).get(any(Long.class), any(TimeUnit.class)); + // Create a future that is already completed. + CompletableFuture future = CompletableFuture.completedFuture(null); try (EventProcessor processor = mock(EventProcessor.class)) { - doAnswer(invocation -> true).when(processor).process(); - - Time time = new MockTime(); - Timer timer = time.timer(1000); 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()); } } + /** + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * handles the case where the {@link Future} does not complete within the timeout. + */ @Test public void testProcessBackgroundEventsTimesOut() throws Exception { Time time = new MockTime(); @@ -1146,6 +1160,8 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { 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()); } } 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 92ff205b1f656..79c7f8f9e9692 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 @@ -53,6 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -858,7 +859,6 @@ public void testListenerCallbacksNoListeners() { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartition("topic1", 0); - CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); doNothing().when(subscriptionState).markPendingRevocation(anySet()); @@ -869,13 +869,10 @@ public void testListenerCallbacksNoListeners() { assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); assertFalse(membershipManager.reconciliationInProgress()); assertEquals(0, backgroundEventQueue.size()); - listener.assertCounts(0, 0, 0); // Step 3: Receive ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); - - listener.assertCounts(0, 0, 0); } @Test @@ -965,41 +962,12 @@ private void performCallback(MembershipManagerImpl membershipManager, assertEquals(methodName, neededEvent.methodName()); assertEquals(partitions, neededEvent.partitions()); - final Exception e; - - switch (methodName) { - case ON_PARTITIONS_REVOKED: - e = invoker.invokePartitionsRevoked(partitions); - break; - - case ON_PARTITIONS_ASSIGNED: - e = invoker.invokePartitionsAssigned(partitions); - break; - - case ON_PARTITIONS_LOST: - e = invoker.invokePartitionsLost(partitions); - break; - - default: - throw new IllegalArgumentException("The method " + methodName + " to invoke was not expected"); - } - - final Optional error; - - if (e != null) { - if (e instanceof KafkaException) - error = Optional.of((KafkaException) e); - else - error = Optional.of(new KafkaException("User rebalance callback throws an error", e)); - } else { - error = Optional.empty(); - } - - ApplicationEvent invokedEvent = new ConsumerRebalanceListenerCallbackCompletedEvent( + ApplicationEvent invokedEvent = invokeRebalanceCallbacks( + invoker, methodName, partitions, - neededEvent.future(), - error); + neededEvent.future() + ); applicationEventHandler.add(invokedEvent); } From a3dd9e47e6ceb41e043ad96fe5874b845f6f04ef Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Dec 2023 18:37:57 -0800 Subject: [PATCH 54/68] Added more test cases to MembershipManagerImplTest for failing callbacks --- .../internals/MembershipManagerImplTest.java | 169 ++++++++++++++---- 1 file changed, 134 insertions(+), 35 deletions(-) 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 79c7f8f9e9692..83ccf347535da 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 @@ -18,8 +18,6 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -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.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; @@ -61,14 +59,11 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; 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.anyBoolean; import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anySet; import static org.mockito.Mockito.clearInvocations; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -829,9 +824,13 @@ public void testListenerCallbacksBasic() { assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); assertTrue(membershipManager.reconciliationInProgress()); - listener.assertCounts(0, 0, 0); + assertEquals(0, listener.revokedCounter.get()); + assertEquals(0, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); - // Step 2: revoke partitions + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 3: revoke partitions performCallback( membershipManager, invoker, @@ -839,7 +838,9 @@ public void testListenerCallbacksBasic() { topicPartitions(new TopicPartition("topic1", 0)) ); - // Step 3: assign partitions + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 4: assign partitions performCallback( membershipManager, invoker, @@ -847,11 +848,15 @@ public void testListenerCallbacksBasic() { Collections.emptySortedSet() ); - // Step 4: Receive ack and make sure we're done and our listener was called appropriately + assertFalse(membershipManager.reconciliationInProgress()); + + // Step 5: Receive ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); - listener.assertCounts(1, 1, 0); + assertEquals(1, listener.revokedCounter.get()); + assertEquals(1, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); } @Test @@ -873,6 +878,111 @@ public void testListenerCallbacksNoListeners() { // Step 3: Receive ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); + assertFalse(membershipManager.reconciliationInProgress()); + } + + // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked. When that + // happens, the assignment step is skipped + // @Test + public void testListenerCallbacksFailsRevocation() { + // Step 1: set up mocks + MembershipManagerImpl membershipManager = createMemberInStableState(); + mockOwnedPartition("topic1", 0); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( + Optional.of(new IllegalArgumentException("Intentional onPartitionsRevoked() error")), + Optional.empty(), + Optional.empty() + ); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + + // Step 2: put the state machine into the appropriate... state + receiveEmptyAssignment(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + assertTrue(membershipManager.reconciliationInProgress()); + assertEquals(0, listener.revokedCounter.get()); + assertEquals(0, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); + + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 3: revoke partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(new TopicPartition("topic1", 0)) + ); + + assertFalse(membershipManager.reconciliationInProgress()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + // Step 4: Receive ack and make sure we're done and our listener was called appropriately + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + assertEquals(1, listener.revokedCounter.get()); + assertEquals(1, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); + } + + @Test + public void testListenerCallbacksFailsAssignment() { + // Step 1: set up mocks + MembershipManagerImpl membershipManager = createMemberInStableState(); + mockOwnedPartition("topic1", 0); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( + Optional.empty(), + Optional.of(new IllegalArgumentException("Intentional onPartitionsAssigned() error")), + Optional.empty() + ); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + + // Step 2: put the state machine into the appropriate... state + receiveEmptyAssignment(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + assertTrue(membershipManager.reconciliationInProgress()); + assertEquals(0, listener.revokedCounter.get()); + assertEquals(0, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); + + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 3: revoke partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(new TopicPartition("topic1", 0)) + ); + + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 4: assign partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + Collections.emptySortedSet() + ); + + assertFalse(membershipManager.reconciliationInProgress()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + // Step 5: Receive ack and make sure we're done and our listener was called appropriately + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + assertEquals(1, listener.revokedCounter.get()); + assertEquals(1, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); } @Test @@ -904,7 +1014,9 @@ private void testOnPartitionsLost(Optional lostError) { membershipManager.transitionToFenced(); assertEquals(MemberState.FENCED, membershipManager.state()); assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); - listener.assertCounts(0, 0, 0); + assertEquals(0, listener.revokedCounter.get()); + assertEquals(0, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); // Step 3: invoke the callback performCallback( @@ -918,7 +1030,9 @@ private void testOnPartitionsLost(Optional lostError) { membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); - listener.assertCounts(0, 0, 1); + assertEquals(0, listener.revokedCounter.get()); + assertEquals(0, listener.assignedCounter.get()); + assertEquals(1, listener.lostCounter.get()); } private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { @@ -942,33 +1056,24 @@ private SortedSet topicPartitions(TopicPartition... topicPartiti private void performCallback(MembershipManagerImpl membershipManager, ConsumerRebalanceListenerInvoker invoker, - ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions) { - // Set up our mock application event handler & background event processor. - ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); - - doAnswer(a -> { - ConsumerRebalanceListenerCallbackCompletedEvent completedEvent = a.getArgument(0); - membershipManager.consumerRebalanceListenerCallbackCompleted(completedEvent); - return null; - }).when(applicationEventHandler).add(any(ConsumerRebalanceListenerCallbackCompletedEvent.class)); - + ConsumerRebalanceListenerMethodName expectedMethodName, + SortedSet expectedPartitions) { // We expect only our enqueued event in the background queue. assertEquals(1, backgroundEventQueue.size()); assertNotNull(backgroundEventQueue.peek()); assertInstanceOf(ConsumerRebalanceListenerCallbackNeededEvent.class, backgroundEventQueue.peek()); ConsumerRebalanceListenerCallbackNeededEvent neededEvent = (ConsumerRebalanceListenerCallbackNeededEvent) backgroundEventQueue.poll(); assertNotNull(neededEvent); - assertEquals(methodName, neededEvent.methodName()); - assertEquals(partitions, neededEvent.partitions()); + assertEquals(expectedMethodName, neededEvent.methodName()); + assertEquals(expectedPartitions, neededEvent.partitions()); - ApplicationEvent invokedEvent = invokeRebalanceCallbacks( + ConsumerRebalanceListenerCallbackCompletedEvent invokedEvent = invokeRebalanceCallbacks( invoker, - methodName, - partitions, + neededEvent.methodName(), + neededEvent.partitions(), neededEvent.future() ); - applicationEventHandler.add(invokedEvent); + membershipManager.consumerRebalanceListenerCallbackCompleted(invokedEvent); } private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( @@ -1324,11 +1429,5 @@ public void onPartitionsLost(Collection partitions) { lostCounter.incrementAndGet(); } } - - public void assertCounts(int revokedCount, int assignedCount, int lostCount) { - assertEquals(revokedCount, revokedCounter.get()); - assertEquals(assignedCount, assignedCounter.get()); - assertEquals(lostCount, lostCounter.get()); - } } } From 1c7e0ad1f88c19b90ddd69bf4cb99911c5b101ca Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Dec 2023 20:20:16 -0800 Subject: [PATCH 55/68] Updates to make creating sets of TopicPartitions and TopicIdPartitions a little cleaner --- .../internals/MembershipManagerImplTest.java | 138 +++++++++--------- 1 file changed, 68 insertions(+), 70 deletions(-) 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 83ccf347535da..7b6bb6b7b7389 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 @@ -290,9 +290,7 @@ public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { CompletableFuture commitResult = mockNewAssignmentAndRevocationStuckOnCommit(membershipManager, topicId1, topic1, Arrays.asList(1, 2), true); - Set assignment1 = new HashSet<>(); - assignment1.add(new TopicIdPartition(topicId1, new TopicPartition(topic1, 1))); - assignment1.add(new TopicIdPartition(topicId1, new TopicPartition(topic1, 2))); + Set assignment1 = topicIdPartitions(topicId1, topic1, 1, 2); assertEquals(assignment1, membershipManager.assignmentReadyToReconcile()); // Get fenced and rejoin while still reconciling. Get new assignment to reconcile after @@ -307,8 +305,7 @@ public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { mockOwnedPartitionAndAssignmentReceived(topicId3, "topic3", owned, true); receiveAssignmentAfterRejoin(topicId3, Collections.singletonList(5), membershipManager); verifyReconciliationNotTriggered(membershipManager); - Set assignmentAfterRejoin = Collections.singleton( - new TopicIdPartition(topicId3, new TopicPartition("topic3", 5))); + Set assignmentAfterRejoin = topicIdPartitions(topicId3, "topic3", 5); assertEquals(assignmentAfterRejoin, membershipManager.assignmentReadyToReconcile()); // Reconciliation completes when the member has already re-joined the group. Should not @@ -372,9 +369,7 @@ public void testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU // Pending assignment that was discovered in metadata should be ready to reconcile in the // next reconciliation loop. - Set topic2Assignment = new HashSet<>(Arrays.asList( - new TopicIdPartition(topicId2, new TopicPartition(topic2, 1)), - new TopicIdPartition(topicId2, new TopicPartition(topic2, 2)))); + Set topic2Assignment = topicIdPartitions(topicId2, topic2, 1, 2); assertEquals(topic2Assignment, membershipManager.assignmentReadyToReconcile()); } @@ -568,9 +563,7 @@ public void testReconcileNewPartitionsAssignedWhenNoPartitionOwned() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); - Set assignedPartitions = new HashSet<>(Arrays.asList( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), - new TopicIdPartition(topicId, new TopicPartition(topicName, 1)))); + Set assignedPartitions = topicIdPartitions(topicId, topicName, 0, 1); verifyReconciliationTriggeredAndCompleted(membershipManager, assignedPartitions); } @@ -586,10 +579,9 @@ public void testReconcileNewPartitionsAssignedWhenOtherPartitionsOwned() { // New assignment received, adding partitions 1 and 2 to the previously owned partition 0. receiveAssignment(topicId, Arrays.asList(0, 1, 2), membershipManager); - Set assignedPartitions = new HashSet<>(Arrays.asList( - new TopicIdPartition(topicId, ownedPartition), - new TopicIdPartition(topicId, new TopicPartition("topic1", 1)), - new TopicIdPartition(topicId, new TopicPartition("topic1", 2)))); + Set assignedPartitions = new HashSet<>(); + assignedPartitions.add(new TopicIdPartition(topicId, ownedPartition)); + assignedPartitions.addAll(topicIdPartitions(topicId, topicName, 1, 2)); verifyReconciliationTriggeredAndCompleted(membershipManager, assignedPartitions); } @@ -603,11 +595,7 @@ public void testReconciliationSkippedWhenSameAssignmentReceived() { // Receive assignment different from what the member owns - should reconcile Set owned = Collections.emptySet(); mockOwnedPartitionAndAssignmentReceived(topicId, topicName, owned, true); - Set expectedAssignmentReconciled = new HashSet<>(); - expectedAssignmentReconciled.add(new TopicIdPartition(topicId, - new TopicPartition(topicName, 0))); - expectedAssignmentReconciled.add(new TopicIdPartition(topicId, - new TopicPartition(topicName, 1))); + Set expectedAssignmentReconciled = topicIdPartitions(topicId, topicName, 0, 1); receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); verifyReconciliationTriggeredAndCompleted(membershipManager, expectedAssignmentReconciled); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); @@ -691,9 +679,7 @@ public void testReconcileNewPartitionsAssignedAndRevoked() { receiveAssignment(topicId, Arrays.asList(1, 2), membershipManager); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); - Set assignedPartitions = new HashSet<>(Arrays.asList( - new TopicIdPartition(topicId, new TopicPartition("topic1", 1)), - new TopicIdPartition(topicId, new TopicPartition("topic1", 2)))); + Set assignedPartitions = topicIdPartitions(topicId, topicName, 1, 2); assertEquals(assignedPartitions, membershipManager.currentAssignment()); assertFalse(membershipManager.reconciliationInProgress()); @@ -723,9 +709,7 @@ public void testMetadataUpdatesReconcilesUnresolvedAssignments() { // When metadata is updated, the member should re-trigger reconciliation membershipManager.onUpdate(null); - Set expectedAssignmentReconciled = new HashSet<>(Arrays.asList( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), - new TopicIdPartition(topicId, new TopicPartition(topicName, 1)))); + Set expectedAssignmentReconciled = topicIdPartitions(topicId, topicName, 0, 1); verifyReconciliationTriggeredAndCompleted(membershipManager, expectedAssignmentReconciled); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); assertTrue(membershipManager.topicsWaitingForMetadata().isEmpty()); @@ -793,8 +777,7 @@ public void testRevokePartitionsUsesTopicNamesLocalCacheWhenMetadataNotAvailable // Revocation should complete without requesting any metadata update given that the topic // received in target assignment should exist in local topic name cache. verify(metadata, never()).requestUpdate(anyBoolean()); - Set remainingAssignment = Collections.singleton( - new TopicIdPartition(topicId, new TopicPartition(topicName, 1))); + Set remainingAssignment = topicIdPartitions(topicId, topicName, 1); testRevocationCompleted(membershipManager, remainingAssignment); } @@ -812,15 +795,21 @@ public void testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() { public void testListenerCallbacksBasic() { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); - mockOwnedPartition("topic1", 0); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + String topicName = "topic1"; + Uuid topicId = Uuid.randomUuid(); + + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); doNothing().when(subscriptionState).markPendingRevocation(anySet()); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); // Step 2: put the state machine into the appropriate... state - receiveEmptyAssignment(membershipManager); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); assertTrue(membershipManager.reconciliationInProgress()); @@ -830,52 +819,49 @@ public void testListenerCallbacksBasic() { assertTrue(membershipManager.reconciliationInProgress()); - // Step 3: revoke partitions - performCallback( - membershipManager, - invoker, - ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(new TopicPartition("topic1", 0)) - ); - - assertTrue(membershipManager.reconciliationInProgress()); - - // Step 4: assign partitions + // Step 3: assign partitions performCallback( membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - Collections.emptySortedSet() + topicPartitions(topicName, 0, 1) ); assertFalse(membershipManager.reconciliationInProgress()); - // Step 5: Receive ack and make sure we're done and our listener was called appropriately + // Step 4: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); + assertEquals(topicPartitions(topicName, 0, 1), subscriptionState.assignedPartitions()); + assertEquals(topicIdPartitions(topicId, topicName, 0, 1), membershipManager.currentAssignment()); - assertEquals(1, listener.revokedCounter.get()); + assertEquals(0, listener.revokedCounter.get()); assertEquals(1, listener.assignedCounter.get()); assertEquals(0, listener.lostCounter.get()); - } - @Test - public void testListenerCallbacksNoListeners() { - // Step 1: set up mocks - MembershipManagerImpl membershipManager = createMemberInStableState(); - mockOwnedPartition("topic1", 0); + receiveEmptyAssignment(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertTrue(membershipManager.reconciliationInProgress()); - when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); - doNothing().when(subscriptionState).markPendingRevocation(anySet()); + // Step 6: revoke partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions("topic1", 0) + ); - // Step 2: put the state machine into the appropriate... state - receiveEmptyAssignment(membershipManager); - assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); - assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); - assertFalse(membershipManager.reconciliationInProgress()); - assertEquals(0, backgroundEventQueue.size()); + assertTrue(membershipManager.reconciliationInProgress()); + + // Step 5: assign partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions("topic1", 0) + ); - // Step 3: Receive ack and make sure we're done and our listener was called appropriately + // Step 3: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); assertFalse(membershipManager.reconciliationInProgress()); @@ -884,7 +870,7 @@ public void testListenerCallbacksNoListeners() { // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked. When that // happens, the assignment step is skipped // @Test - public void testListenerCallbacksFailsRevocation() { + public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartition("topic1", 0); @@ -914,13 +900,13 @@ public void testListenerCallbacksFailsRevocation() { membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(new TopicPartition("topic1", 0)) + topicPartitions("topic1", 0) ); assertFalse(membershipManager.reconciliationInProgress()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - // Step 4: Receive ack and make sure we're done and our listener was called appropriately + // Step 4: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -930,7 +916,7 @@ public void testListenerCallbacksFailsRevocation() { } @Test - public void testListenerCallbacksFailsAssignment() { + public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); mockOwnedPartition("topic1", 0); @@ -960,7 +946,7 @@ public void testListenerCallbacksFailsAssignment() { membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions(new TopicPartition("topic1", 0)) + topicPartitions("topic1", 0) ); assertTrue(membershipManager.reconciliationInProgress()); @@ -976,7 +962,7 @@ public void testListenerCallbacksFailsAssignment() { assertFalse(membershipManager.reconciliationInProgress()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - // Step 5: Receive ack and make sure we're done and our listener was called appropriately + // Step 5: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -1023,7 +1009,7 @@ private void testOnPartitionsLost(Optional lostError) { membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST, - topicPartitions(new TopicPartition("topic1", 0)) + topicPartitions("topic1", 0) ); // Step 4: Receive ack and make sure we're done and our listener was called appropriately @@ -1048,10 +1034,22 @@ private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { ); } - private SortedSet topicPartitions(TopicPartition... topicPartitions) { - SortedSet revokedPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); - revokedPartitions.addAll(Arrays.asList(topicPartitions)); - return revokedPartitions; + private SortedSet topicPartitions(String topicName, int... partitions) { + SortedSet topicPartitions = new TreeSet<>(new Utils.TopicPartitionComparator()); + + for (int partition : partitions) + topicPartitions.add(new TopicPartition(topicName, partition)); + + return topicPartitions; + } + + private SortedSet topicIdPartitions(Uuid topicId, String topicName, int... partitions) { + SortedSet topicIdPartitions = new TreeSet<>(new Utils.TopicIdPartitionComparator()); + + for (int partition : partitions) + topicIdPartitions.add(new TopicIdPartition(topicId, new TopicPartition(topicName, partition))); + + return topicIdPartitions; } private void performCallback(MembershipManagerImpl membershipManager, From 482fa55c628200a924dc91792c7ffe3a6e8b71d4 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 8 Dec 2023 20:39:18 -0800 Subject: [PATCH 56/68] Added missing mock method to fix testListenerCallbacksBasic --- .../internals/MembershipManagerImplTest.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) 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 7b6bb6b7b7389..0bb6396630452 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 @@ -832,13 +832,14 @@ public void testListenerCallbacksBasic() { // Step 4: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(topicPartitions(topicName, 0, 1), subscriptionState.assignedPartitions()); assertEquals(topicIdPartitions(topicId, topicName, 0, 1), membershipManager.currentAssignment()); assertEquals(0, listener.revokedCounter.get()); assertEquals(1, listener.assignedCounter.get()); assertEquals(0, listener.lostCounter.get()); + // Step 5: receive an empty assignment, which means we should call revoke + when(subscriptionState.assignedPartitions()).thenReturn(topicPartitions(topicName, 0, 1)); receiveEmptyAssignment(membershipManager); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertTrue(membershipManager.reconciliationInProgress()); @@ -848,27 +849,34 @@ public void testListenerCallbacksBasic() { membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, - topicPartitions("topic1", 0) + topicPartitions(topicName, 0, 1) ); - assertTrue(membershipManager.reconciliationInProgress()); - // Step 5: assign partitions + // Step 7: assign partitions should still be called, even though it's empty performCallback( membershipManager, invoker, ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, - topicPartitions("topic1", 0) + Collections.emptySortedSet() ); + assertFalse(membershipManager.reconciliationInProgress()); - // Step 3: Send ack and make sure we're done and our listener was called appropriately + // Step 8: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); assertFalse(membershipManager.reconciliationInProgress()); + + assertEquals(1, listener.revokedCounter.get()); + assertEquals(2, listener.assignedCounter.get()); + assertEquals(0, listener.lostCounter.get()); } - // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked. When that - // happens, the assignment step is skipped + // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked(). When that + // happens, the assignment step is skipped, which means onPartitionsAssigned() is never run. + // The jury is out on whether or not this is a bug or intentional. + // + // See https://github.com/apache/kafka/pull/14640#discussion_r1421253120 for more details. // @Test public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 1: set up mocks From 251fdc9ddc9914db267e2a12dbf09f5853913be0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 11:57:42 -0800 Subject: [PATCH 57/68] Added MembershipManagerImplTest testListenerCallbacksThrowsErrorOnPartitionsRevoked as @Disabled --- .../clients/consumer/internals/MembershipManagerImplTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 5323c555575a1..fb4943f41daff 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 @@ -35,6 +35,7 @@ import org.apache.kafka.common.utils.MockTime; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -1123,7 +1124,8 @@ public void testListenerCallbacksBasic() { // The jury is out on whether or not this is a bug or intentional. // // See https://github.com/apache/kafka/pull/14640#discussion_r1421253120 for more details. - // @Test + @Test + @Disabled public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 1: set up mocks String topicName = "topic1"; From 64bf9decd3330f652c331216ba657cad40838794 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 12:38:40 -0800 Subject: [PATCH 58/68] Updates to revert unncessesary changes --- .../internals/AsyncKafkaConsumer.java | 38 +++++-------------- .../internals/MembershipManagerImpl.java | 20 ++++++++++ .../consumer/internals/RequestManagers.java | 1 + .../internals/ConsumerTestBuilder.java | 1 + .../internals/MembershipManagerImplTest.java | 10 ++--- 5 files changed, 37 insertions(+), 33 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 ef8c142760a4c..1e81e2341b031 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 @@ -76,7 +76,6 @@ import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils; import org.apache.kafka.common.utils.AppInfoParser; @@ -222,7 +221,16 @@ private void process(final ErrorBackgroundEvent event) { } private void process(final GroupMetadataUpdateEvent event) { - updateConsumerGroupMetadata(event.memberId(), event.memberEpoch()); + if (groupMetadata.isPresent()) { + final ConsumerGroupMetadata currentGroupMetadata = groupMetadata.get(); + final String newMemberId = event.memberId(); + AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( + currentGroupMetadata.groupId(), + event.memberEpoch(), + newMemberId, + currentGroupMetadata.groupInstanceId() + )); + } } private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { @@ -1646,32 +1654,6 @@ T processBackgroundEvents(EventProcessor eventProcessor, throw new TimeoutException("Operation timed out before completion"); } - void updateConsumerGroupMetadata(String newMemberId, int newMemberEpoch) { - if (groupMetadata.isPresent()) { - final ConsumerGroupMetadata currentGroupMetadata = groupMetadata.get(); - final String oldMemberId = currentGroupMetadata.memberId(); - this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - newMemberEpoch, - newMemberId, - currentGroupMetadata.groupInstanceId() - )); - - // Update the group member id label in the client telemetry reporter if the member id has - // changed. Initially the member id is empty, and it is updated when the member joins the - // group. This is done here to avoid updating the label on every heartbeat response. Also - // check if the member id is null, as the schema defines it as nullable. - if (!oldMemberId.equals(newMemberId) && clientTelemetryReporter.isPresent()) { - Map labels = Collections.singletonMap( - ClientTelemetryProvider.GROUP_MEMBER_ID, - newMemberId - ); - ClientTelemetryReporter reporter = clientTelemetryReporter.get(); - reporter.updateMetricsLabels(labels); - } - } - } - static ConsumerRebalanceListenerCallbackCompletedEvent invokeRebalanceCallbacks(ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, ConsumerRebalanceListenerMethodName methodName, SortedSet partitions, 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 93fc629e6d1bc..d5d50a9e7dc60 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 @@ -35,6 +35,8 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; +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.Utils; import org.slf4j.Logger; @@ -245,6 +247,13 @@ public class MembershipManagerImpl implements MembershipManager, ClusterResource */ private boolean isRegisteredForMetadataUpdates; + /** + * Optional client telemetry reporter which sends client telemetry data to the broker. This + * will be empty if the client telemetry feature is not enabled. This is provided to update + * the group member id label when the member joins the group. + */ + private final Optional clientTelemetryReporter; + /** * Serves as the conduit by which we can report events to the application thread. This is needed as we send * {@link ConsumerRebalanceListenerCallbackNeededEvent callbacks} and, if needed, @@ -259,6 +268,7 @@ public MembershipManagerImpl(String groupId, CommitRequestManager commitRequestManager, ConsumerMetadata metadata, LogContext logContext, + Optional clientTelemetryReporter, BackgroundEventHandler backgroundEventHandler) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; @@ -272,6 +282,7 @@ public MembershipManagerImpl(String groupId, this.assignmentReadyToReconcile = new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR); this.currentAssignment = new HashMap<>(); this.log = logContext.logger(MembershipManagerImpl.class); + this.clientTelemetryReporter = clientTelemetryReporter; this.backgroundEventHandler = backgroundEventHandler; } @@ -335,6 +346,15 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo throw new IllegalArgumentException(errorMessage); } + // Update the group member id label in the client telemetry reporter if the member id has + // changed. Initially the member id is empty, and it is updated when the member joins the + // group. This is done here to avoid updating the label on every heartbeat response. Also + // check if the member id is null, as the schema defines it as nullable. + if (response.memberId() != null && !response.memberId().equals(memberId)) { + clientTelemetryReporter.ifPresent(reporter -> reporter.updateMetricsLabels( + Collections.singletonMap(ClientTelemetryProvider.GROUP_MEMBER_ID, response.memberId()))); + } + this.memberId = response.memberId(); this.memberEpoch = response.memberEpoch(); commitRequestManager.updateMemberInformation(this.memberId, this.memberEpoch); 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 7b48a5196d159..935402b884d81 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 @@ -176,6 +176,7 @@ protected RequestManagers create() { commit, metadata, logContext, + clientTelemetryReporter, backgroundEventHandler); heartbeatRequestManager = new HeartbeatRequestManager( logContext, 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 d455a28d07ea6..d7690286e15ce 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 @@ -205,6 +205,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA commit, metadata, logContext, + Optional.empty(), 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 fb4943f41daff..877821995fd40 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 @@ -111,7 +111,7 @@ public void tearDown() { private MembershipManagerImpl createMembershipManagerJoiningGroup() { MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, - metadata, logContext, backgroundEventHandler)); + metadata, logContext, Optional.empty(), backgroundEventHandler)); manager.transitionToJoining(); return manager; } @@ -120,7 +120,7 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, - backgroundEventHandler)); + Optional.empty(), backgroundEventHandler)); manager.transitionToJoining(); return manager; } @@ -130,7 +130,7 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, metadata, logContext, - backgroundEventHandler); + Optional.empty(), backgroundEventHandler); manager.transitionToJoining(); return manager; } @@ -155,7 +155,7 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() // First join should register to get metadata updates MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, - metadata, logContext, backgroundEventHandler); + metadata, logContext, Optional.empty(), backgroundEventHandler); manager.transitionToJoining(); verify(metadata).addClusterUpdateListener(manager); clearInvocations(metadata); @@ -233,7 +233,7 @@ public void testTransitionToFatal() { public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), Optional.empty(), subscriptionState, commitRequestManager, metadata, - logContext, backgroundEventHandler); + logContext, Optional.empty(), backgroundEventHandler); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); From 47bdc7cac2ed1d8b4dc9fd7b6d2e98fd43feca6b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 12:53:11 -0800 Subject: [PATCH 59/68] More clean up to revert and streamline PR --- .../internals/AsyncKafkaConsumer.java | 23 +++++-------- .../consumer/internals/ConsumerUtils.java | 33 +++++++++---------- .../consumer/internals/MembershipManager.java | 1 - .../internals/MembershipManagerImpl.java | 2 +- .../internals/events/EventProcessor.java | 9 ++--- 5 files changed, 28 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 1e81e2341b031..d8b5fe74a0edb 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 @@ -221,14 +221,13 @@ private void process(final ErrorBackgroundEvent event) { } private void process(final GroupMetadataUpdateEvent event) { - if (groupMetadata.isPresent()) { - final ConsumerGroupMetadata currentGroupMetadata = groupMetadata.get(); - final String newMemberId = event.memberId(); + if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { + final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - event.memberEpoch(), - newMemberId, - currentGroupMetadata.groupInstanceId() + currentGroupMetadata.groupId(), + event.memberEpoch(), + event.memberId(), + currentGroupMetadata.groupInstanceId() )); } } @@ -1679,14 +1678,10 @@ static ConsumerRebalanceListenerCallbackCompletedEvent invokeRebalanceCallbacks( final Optional error; - if (e != null) { - if (e instanceof KafkaException) - error = Optional.of((KafkaException) e); - else - error = Optional.of(new KafkaException("User rebalance callback throws an error", e)); - } else { + if (e != null) + error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(e, "User rebalance callback throws an error")); + else error = Optional.empty(); - } return new ConsumerRebalanceListenerCallbackCompletedEvent( methodName, 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 b198098d9da04..e84f70f7d9bed 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 @@ -31,7 +31,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.KafkaMetricsContext; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -210,14 +209,7 @@ public static T getResult(Future future, Timer timer) { try { return future.get(timer.remainingMs(), TimeUnit.MILLISECONDS); } catch (ExecutionException e) { - Throwable t = e.getCause(); - - if (t instanceof WakeupException) - throw new WakeupException(); - else if (t instanceof KafkaException) - throw (KafkaException) t; - else - throw new KafkaException(t); + throw maybeWrapAsKafkaException(e.getCause()); } catch (InterruptedException e) { throw new InterruptException(e); } catch (java.util.concurrent.TimeoutException e) { @@ -229,16 +221,23 @@ public static T getResult(Future future) { try { return future.get(); } catch (ExecutionException e) { - Throwable t = e.getCause(); - - if (t instanceof WakeupException) - throw new WakeupException(); - else if (t instanceof KafkaException) - throw (KafkaException) t; - else - throw new KafkaException(t); + throw maybeWrapAsKafkaException(e.getCause()); } catch (InterruptedException e) { throw new InterruptException(e); } } + + public static KafkaException maybeWrapAsKafkaException(Throwable t) { + if (t instanceof KafkaException) + return (KafkaException) t; + else + return new KafkaException(t); + } + + public static KafkaException maybeWrapAsKafkaException(Throwable t, String message) { + if (t instanceof KafkaException) + return (KafkaException) t; + else + return new KafkaException(message, t); + } } 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 5f21c57ece550..b075570003ba0 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 @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; -import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; 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 d5d50a9e7dc60..67d1bbb92722b 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 @@ -352,7 +352,7 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo // check if the member id is null, as the schema defines it as nullable. if (response.memberId() != null && !response.memberId().equals(memberId)) { clientTelemetryReporter.ifPresent(reporter -> reporter.updateMetricsLabels( - Collections.singletonMap(ClientTelemetryProvider.GROUP_MEMBER_ID, response.memberId()))); + Collections.singletonMap(ClientTelemetryProvider.GROUP_MEMBER_ID, response.memberId()))); } this.memberId = response.memberId(); 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 bf23bb844acbc..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 @@ -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.KafkaException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; @@ -85,13 +86,7 @@ protected boolean process(ProcessHandler processHandler) { process(event); processHandler.onProcess(event, Optional.empty()); } catch (Throwable t) { - KafkaException error; - - if (t instanceof KafkaException) - error = (KafkaException) t; - else - error = new KafkaException(t); - + KafkaException error = ConsumerUtils.maybeWrapAsKafkaException(t); processHandler.onProcess(event, Optional.of(error)); } } From 51ae424ce73a69c45c563ca04a63dcb371bae233 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 13:06:03 -0800 Subject: [PATCH 60/68] More clean up --- .../events/CompletableBackgroundEvent.java | 17 ----------------- .../internals/MembershipManagerImplTest.java | 11 ++++++----- 2 files changed, 6 insertions(+), 22 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 61ec322cb3a55..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 @@ -16,9 +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; /** @@ -40,20 +37,6 @@ 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 boolean equals(Object o) { if (this == o) return true; 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 877821995fd40..a38dbba0de289 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 @@ -129,8 +129,8 @@ private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupIn String serverAssignor) { MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), Optional.ofNullable(serverAssignor), - subscriptionState, commitRequestManager, metadata, logContext, - Optional.empty(), backgroundEventHandler); + subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), + backgroundEventHandler); manager.transitionToJoining(); return manager; } @@ -836,8 +836,7 @@ public void testReconciliationSkippedWhenSameAssignmentReceived() { String topicName = "topic1"; // Receive assignment different from what the member owns - should reconcile - List owned = Collections.emptyList(); - mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, owned); + mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, Collections.emptyList()); List expectedAssignmentReconciled = topicIdPartitions(topicId, topicName, 0, 1); receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); verifyReconciliationTriggeredAndCompleted(membershipManager, expectedAssignmentReconciled); @@ -999,6 +998,7 @@ public void testRevokePartitionsUsesTopicNamesLocalCacheWhenMetadataNotAvailable mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, Collections.emptyList()); // Member received assignment to reconcile; + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); // Member should complete reconciliation @@ -1364,7 +1364,8 @@ private void testFenceIsNoOp(MembershipManagerImpl membershipManager, private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( Uuid topicId, String topicName, List partitions) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); - mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, Collections.emptyList()); + mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, topicName, + Collections.emptyList()); receiveAssignment(topicId, partitions, membershipManager); From f5541486905c12edfd2831e85570dccfbfa8a602 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 13:14:45 -0800 Subject: [PATCH 61/68] Minor formatting change --- .../clients/consumer/internals/AsyncKafkaConsumer.java | 6 +----- 1 file changed, 1 insertion(+), 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 d8b5fe74a0edb..a702452fa29d6 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 @@ -1683,11 +1683,7 @@ static ConsumerRebalanceListenerCallbackCompletedEvent invokeRebalanceCallbacks( else error = Optional.empty(); - return new ConsumerRebalanceListenerCallbackCompletedEvent( - methodName, - future, - error - ); + return new ConsumerRebalanceListenerCallbackCompletedEvent(methodName, future, error); } @Override From bc4e2aaf3381297fbe2e947acac3ed692c7e7da0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 11 Dec 2023 15:32:59 -0800 Subject: [PATCH 62/68] Added a parameterized unit test for callbacks to AsyncKafkaConsumerTest --- .../internals/AsyncKafkaConsumerTest.java | 83 +++++++++++++ .../CounterConsumerRebalanceListener.java | 90 ++++++++++++++ .../internals/MembershipManagerImplTest.java | 110 +++++------------- 3 files changed, 200 insertions(+), 83 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/CounterConsumerRebalanceListener.java 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 200180e39cdfe..cad3bc58a41cf 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.ClientResponse; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.GroupProtocol; @@ -31,6 +32,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.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; @@ -69,6 +72,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; @@ -89,6 +93,8 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.SortedSet; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -102,6 +108,9 @@ import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST; +import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -131,6 +140,7 @@ public class AsyncKafkaConsumerTest { private ConsumerTestBuilder.AsyncKafkaConsumerTestBuilder testBuilder; private ApplicationEventHandler applicationEventHandler; private SubscriptionState subscriptions; + private BlockingQueue backgroundEventQueue; @BeforeEach public void setup() { @@ -144,6 +154,7 @@ private void setup(Optional groupInfo, boo consumer = testBuilder.consumer; fetchCollector = testBuilder.fetchCollector; subscriptions = testBuilder.subscriptions; + backgroundEventQueue = testBuilder.backgroundEventQueue; } @AfterEach @@ -860,6 +871,78 @@ public void testGroupMetadataUpdateSingleCall() { } } + /** + * Tests that the consumer correctly invokes the callbacks for {@link ConsumerRebalanceListener} that was + * specified. We don't go through the full effort to emulate heartbeats and correct group management here. We're + * simply exercising the background {@link EventProcessor} does the correct thing when + * {@link AsyncKafkaConsumer#poll(Duration)} is called. + * + * Note that we test {@link ConsumerRebalanceListener} that throws errors in its different callbacks. Failed + * callback execution does not immediately errors. Instead, those errors are forwarded to the + * application event thread for the {@link MembershipManagerImpl} to handle. + */ + @ParameterizedTest + @MethodSource("listenerCallbacksInvokeSource") + public void testListenerCallbacksInvoke(List methodNames, + Optional revokedError, + Optional assignedError, + Optional lostError, + int expectedRevokedCount, + int expectedAssignedCount, + int expectedLostCount) { + CounterConsumerRebalanceListener consumerRebalanceListener = new CounterConsumerRebalanceListener( + revokedError, + assignedError, + lostError + ); + consumer.subscribe(Collections.singletonList("topic"), consumerRebalanceListener); + SortedSet partitions = Collections.emptySortedSet(); + + for (ConsumerRebalanceListenerMethodName methodName : methodNames) { + CompletableBackgroundEvent e = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, partitions); + backgroundEventQueue.add(e); + + // This will trigger the background event queue to process our background event message. + consumer.poll(Duration.ZERO); + } + + assertEquals(expectedRevokedCount, consumerRebalanceListener.revokedCount()); + assertEquals(expectedAssignedCount, consumerRebalanceListener.assignedCount()); + assertEquals(expectedLostCount, consumerRebalanceListener.lostCount()); + } + + private static Stream listenerCallbacksInvokeSource() { + Optional empty = Optional.empty(); + Optional error = Optional.of(new RuntimeException("Intentional error")); + + return Stream.of( + // Tests if we don't have an event, the listener doesn't get called. + Arguments.of(Collections.emptyList(), empty, empty, empty, 0, 0, 0), + + // Tests if we get an event for a revocation, that we invoke our listener. + Arguments.of(Collections.singletonList(ON_PARTITIONS_REVOKED), empty, empty, empty, 1, 0, 0), + + // Tests if we get an event for an assignment, that we invoke our listener. + Arguments.of(Collections.singletonList(ON_PARTITIONS_ASSIGNED), empty, empty, empty, 0, 1, 0), + + // Tests that we invoke our listener even if it encounters an exception. + Arguments.of(Collections.singletonList(ON_PARTITIONS_LOST), empty, empty, empty, 0, 0, 1), + + // Tests that we invoke our listener even if it encounters an exception. + Arguments.of(Collections.singletonList(ON_PARTITIONS_REVOKED), error, empty, empty, 1, 0, 0), + + // Tests that we invoke our listener even if it encounters an exception. + Arguments.of(Collections.singletonList(ON_PARTITIONS_ASSIGNED), empty, error, empty, 0, 1, 0), + + // Tests that we invoke our listener even if it encounters an exception. + Arguments.of(Collections.singletonList(ON_PARTITIONS_LOST), empty, empty, error, 0, 0, 1), + + // Tests if we get separate events for revocation and then assignment--AND our revocation throws an error-- + // we still invoke the listeners correctly without throwing the error at the user. + Arguments.of(Arrays.asList(ON_PARTITIONS_REVOKED, ON_PARTITIONS_ASSIGNED), error, empty, empty, 1, 1, 0) + ); + } + @Test public void testBackgroundError() { final String groupId = "consumerGroupA"; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CounterConsumerRebalanceListener.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CounterConsumerRebalanceListener.java new file mode 100644 index 0000000000000..645920faf4611 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CounterConsumerRebalanceListener.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; + +public class CounterConsumerRebalanceListener implements ConsumerRebalanceListener { + + private final AtomicInteger revokedCounter = new AtomicInteger(); + private final AtomicInteger assignedCounter = new AtomicInteger(); + private final AtomicInteger lostCounter = new AtomicInteger(); + + private final Optional revokedError; + private final Optional assignedError; + private final Optional lostError; + + public CounterConsumerRebalanceListener() { + this(Optional.empty(), Optional.empty(), Optional.empty()); + } + + public CounterConsumerRebalanceListener(Optional revokedError, + Optional assignedError, + Optional lostError) { + this.revokedError = revokedError; + this.assignedError = assignedError; + this.lostError = lostError; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + try { + if (revokedError.isPresent()) + throw revokedError.get(); + } finally { + revokedCounter.incrementAndGet(); + } + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + try { + if (assignedError.isPresent()) + throw assignedError.get(); + } finally { + assignedCounter.incrementAndGet(); + } + } + + @Override + public void onPartitionsLost(Collection partitions) { + try { + if (lostError.isPresent()) + throw lostError.get(); + } finally { + lostCounter.incrementAndGet(); + } + } + + public int revokedCount() { + return revokedCounter.get(); + } + + public int assignedCount() { + return assignedCounter.get(); + } + + public int lostCount() { + return lostCounter.get(); + } +} \ No newline at end of file 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 a38dbba0de289..d670b80012370 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 @@ -17,7 +17,6 @@ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 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; @@ -40,7 +39,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -52,7 +50,6 @@ import java.util.TreeSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks; @@ -1060,9 +1057,9 @@ public void testListenerCallbacksBasic() { receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertTrue(membershipManager.reconciliationInProgress()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(0, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.assignedCount()); + assertEquals(0, listener.lostCount()); assertTrue(membershipManager.reconciliationInProgress()); @@ -1081,9 +1078,9 @@ public void testListenerCallbacksBasic() { assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(topicIdPartitionsMap(topicId, 0, 1), membershipManager.currentAssignment()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(1, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(1, listener.assignedCount()); + assertEquals(0, listener.lostCount()); // Step 5: receive an empty assignment, which means we should call revoke when(subscriptionState.assignedPartitions()).thenReturn(topicPartitions(topicName, 0, 1)); @@ -1114,9 +1111,9 @@ public void testListenerCallbacksBasic() { assertEquals(MemberState.STABLE, membershipManager.state()); assertFalse(membershipManager.reconciliationInProgress()); - assertEquals(1, listener.revokedCounter.get()); - assertEquals(2, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(1, listener.revokedCount()); + assertEquals(2, listener.assignedCount()); + assertEquals(0, listener.lostCount()); } // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked(). When that @@ -1148,9 +1145,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); assertTrue(membershipManager.reconciliationInProgress()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(0, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.assignedCount()); + assertEquals(0, listener.lostCount()); assertTrue(membershipManager.reconciliationInProgress()); @@ -1169,9 +1166,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); - assertEquals(1, listener.revokedCounter.get()); - assertEquals(1, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(1, listener.revokedCount()); + assertEquals(1, listener.assignedCount()); + assertEquals(0, listener.lostCount()); } @Test @@ -1196,9 +1193,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment()); assertTrue(membershipManager.reconciliationInProgress()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(0, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.assignedCount()); + assertEquals(0, listener.lostCount()); assertTrue(membershipManager.reconciliationInProgress()); @@ -1227,9 +1224,9 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); - assertEquals(1, listener.revokedCounter.get()); - assertEquals(1, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(1, listener.revokedCount()); + assertEquals(1, listener.assignedCount()); + assertEquals(0, listener.lostCount()); } @Test @@ -1267,9 +1264,9 @@ private void testOnPartitionsLost(Optional lostError) { membershipManager.transitionToFenced(); assertEquals(MemberState.FENCED, membershipManager.state()); assertEquals(Collections.emptyMap(), membershipManager.currentAssignment()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(0, listener.assignedCounter.get()); - assertEquals(0, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.assignedCount()); + assertEquals(0, listener.lostCount()); // Step 3: invoke the callback performCallback( @@ -1283,9 +1280,9 @@ private void testOnPartitionsLost(Optional lostError) { membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); - assertEquals(0, listener.revokedCounter.get()); - assertEquals(0, listener.assignedCounter.get()); - assertEquals(1, listener.lostCounter.get()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.assignedCount()); + assertEquals(1, listener.lostCount()); } private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { @@ -1697,57 +1694,4 @@ private ConsumerGroupHeartbeatResponseData.Assignment createAssignment(boolean m .setPartitions(Arrays.asList(3, 4, 5)) )); } - - private static class CounterConsumerRebalanceListener implements ConsumerRebalanceListener { - - private final AtomicInteger revokedCounter = new AtomicInteger(); - private final AtomicInteger assignedCounter = new AtomicInteger(); - private final AtomicInteger lostCounter = new AtomicInteger(); - - private final Optional revokedError; - private final Optional assignedError; - private final Optional lostError; - - public CounterConsumerRebalanceListener() { - this(Optional.empty(), Optional.empty(), Optional.empty()); - } - - public CounterConsumerRebalanceListener(Optional revokedError, - Optional assignedError, - Optional lostError) { - this.revokedError = revokedError; - this.assignedError = assignedError; - this.lostError = lostError; - } - - @Override - public void onPartitionsRevoked(Collection partitions) { - try { - if (revokedError.isPresent()) - throw revokedError.get(); - } finally { - revokedCounter.incrementAndGet(); - } - } - - @Override - public void onPartitionsAssigned(Collection partitions) { - try { - if (assignedError.isPresent()) - throw assignedError.get(); - } finally { - assignedCounter.incrementAndGet(); - } - } - - @Override - public void onPartitionsLost(Collection partitions) { - try { - if (lostError.isPresent()) - throw lostError.get(); - } finally { - lostCounter.incrementAndGet(); - } - } - } } From e57d39d0c000f0e35ca1d74c27056fc6ec0f2d74 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 12 Dec 2023 08:47:00 -0800 Subject: [PATCH 63/68] Re-enabled MembershipManagerImplTest.testListenerCallbacksThrowsErrorOnPartitionsRevoked() --- .../consumer/internals/MembershipManagerImplTest.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) 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 888a93ca391c7..15d636c5ac60d 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 @@ -34,7 +34,6 @@ import org.apache.kafka.common.utils.MockTime; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -1117,13 +1116,7 @@ public void testListenerCallbacksBasic() { assertEquals(0, listener.lostCount()); } - // TODO: Reconciliation needs to support when a listener throws an error on onPartitionsRevoked(). When that - // happens, the assignment step is skipped, which means onPartitionsAssigned() is never run. - // The jury is out on whether or not this is a bug or intentional. - // - // See https://github.com/apache/kafka/pull/14640#discussion_r1421253120 for more details. @Test - @Disabled public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 1: set up mocks String topicName = "topic1"; @@ -1144,7 +1137,7 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { // Step 2: put the state machine into the appropriate... state receiveEmptyAssignment(membershipManager); assertEquals(MemberState.RECONCILING, membershipManager.state()); - assertEquals(Collections.emptySet(), membershipManager.currentAssignment()); + assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment()); assertTrue(membershipManager.reconciliationInProgress()); assertEquals(0, listener.revokedCount()); assertEquals(0, listener.assignedCount()); @@ -1168,7 +1161,7 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(1, listener.revokedCount()); - assertEquals(1, listener.assignedCount()); + assertEquals(0, listener.assignedCount()); assertEquals(0, listener.lostCount()); } From 9211f647cc901b44c461fe475081196975361896 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 13 Dec 2023 15:50:31 -0800 Subject: [PATCH 64/68] Update set of PlaintextConsumerTest's disabled/enabled integration tests for the consumer group protocol --- .../kafka/api/PlaintextConsumerTest.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 86a23c6aacba7..35a126f42632e 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -169,7 +169,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { startingTimestamp = startingTimestamp) } - // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. + // TODO: enable this test for the consumer group protocol when KAFKA-16008 has been fixed. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = { @@ -196,7 +196,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(1, listener.callsToRevoked) } - // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. + // TODO: enable this test for the consumer group protocol when KAFKA-16009 has been fixed. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = { @@ -238,9 +238,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertTrue(commitCompleted) } - // TODO: enable this test for the consumer group protocol when max.poll.interval.ms is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testMaxPollIntervalMsDelayInAssignment(quorum: String, groupProtocol: String): Unit = { this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString) this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) @@ -266,7 +265,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { // TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testAutoCommitOnClose(quorum: String, groupProtocol: String): Unit = { this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") val consumer = createConsumer() @@ -548,9 +547,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { awaitAssignment(consumer, expandedAssignment) } - // TODO: enable this test for the consumer group protocol when proper support for assignment change is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { val otherTopic = "other" createTopic(otherTopic, 2, brokerCount) @@ -1266,15 +1264,16 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } - // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. + // TODO: enable this test for the consumer group protocol when KAFKA-16010 has been fixed. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = { runMultiConsumerSessionTimeoutTest(false) } + // TODO: enable this test for the consumer group protocol when KAFKA-16011 has been fixed. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = { runMultiConsumerSessionTimeoutTest(true) } @@ -1596,9 +1595,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(500, consumer.committed(Set(tp2).asJava).get(tp2).offset) } - // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testPerPartitionLeadMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = { val numMessages = 1000 val topic2 = "topic2" From f2de38422028fed054944dbfe98694a1273388e3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 13 Dec 2023 15:52:41 -0800 Subject: [PATCH 65/68] Enable another previously disable test in PlaintextConsumerTest --- .../scala/integration/kafka/api/PlaintextConsumerTest.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 35a126f42632e..db46991c492f7 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -1635,9 +1635,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertNull(consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags2))) } - // TODO: enable this test for the consumer group protocol when fixes for hanging reconciliation is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testPerPartitionLagMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = { val numMessages = 1000 val topic2 = "topic2" From de782354ab5392515eecd36115d90776f9670286 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 13 Dec 2023 16:57:13 -0800 Subject: [PATCH 66/68] Enabled testCoordinatorFailover for consumer group protocol now that partitionsFor is implemented --- .../test/scala/integration/kafka/api/BaseConsumerTest.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 68b331e25af42..b89b674d3cd1a 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -78,9 +78,8 @@ abstract class BaseConsumerTest extends AbstractConsumerTest { assertNotEquals(0, BaseConsumerTest.updateConsumerCount.get()) } - // TODO: enable this test for the consumer group protocol when partitionsFor is implemented. @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testCoordinatorFailover(quorum: String, groupProtocol: String): Unit = { val listener = new TestConsumerReassignmentListener() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "5001") From a184aa3dd05ad5c9931b41adeb4bb6d1639fab9a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 13 Dec 2023 17:05:41 -0800 Subject: [PATCH 67/68] TODO-ified comments for PlaintextConsumerTest for consumer interceptors --- .../scala/integration/kafka/api/PlaintextConsumerTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index db46991c492f7..839f6d67fb16f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -1278,7 +1278,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { runMultiConsumerSessionTimeoutTest(true) } - // Consumer interceptors temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when consumer interceptors are supported @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testInterceptors(quorum: String, groupProtocol: String): Unit = { @@ -1339,7 +1339,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { MockProducerInterceptor.resetCounters() } - // Consumer interceptors temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when consumer interceptors are supported @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testAutoCommitIntercept(quorum: String, groupProtocol: String): Unit = { @@ -1391,7 +1391,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { MockConsumerInterceptor.resetCounters() } - // Consumer interceptors temporarily not supported for consumer group protocol + // TODO: enable this test for the consumer group protocol when consumer interceptors are supported @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly")) def testInterceptorsWithWrongKeyValue(quorum: String, groupProtocol: String): Unit = { From fff4440a381150baa73ea1966e5d6254d9e4377c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 14 Dec 2023 07:04:06 -0800 Subject: [PATCH 68/68] Fix compilation error in MembershipManagerImplTest post-sync of fork --- .../clients/consumer/internals/MembershipManagerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4163c0c7df90e..88058d1d54132 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 @@ -301,7 +301,7 @@ public void testNewAssignmentIgnoredWhenStateIsPrepareLeaving() { // the group, ignoring the new assignment received. Uuid topicId = Uuid.randomUuid(); mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, "topic1", - Collections.emptyList(), true); + Collections.emptyList()); receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); assertEquals(MemberState.PREPARE_LEAVING, membershipManager.state()); assertTrue(membershipManager.assignmentReadyToReconcile().isEmpty());