kirktrue commented on code in PR #14638: URL: https://github.com/apache/kafka/pull/14638#discussion_r1372381836
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java: ########## @@ -0,0 +1,128 @@ +/* + * 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 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.Optional; +import java.util.Set; +import java.util.SortedSet; + +/** + * 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. + */ +class ConsumerRebalanceListenerInvoker { + + private final Logger log; + private final SubscriptionState subscriptions; + private final Time time; + private final ConsumerCoordinatorMetrics coordinatorMetrics; + + ConsumerRebalanceListenerInvoker(LogContext logContext, + SubscriptionState subscriptions, + Time time, + ConsumerCoordinatorMetrics coordinatorMetrics) { + this.log = logContext.logger(getClass()); + this.subscriptions = subscriptions; + this.time = time; + this.coordinatorMetrics = coordinatorMetrics; + } + + Exception invokePartitionsAssigned(final SortedSet<TopicPartition> assignedPartitions) { + log.info("Adding newly assigned partitions: {}", Utils.join(assignedPartitions, ", ")); + + Optional<ConsumerRebalanceListener> listener = subscriptions.rebalanceListener(); + + if (listener.isPresent()) { + try { + final long startMs = time.milliseconds(); + 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 {}", + listener.getClass().getName(), assignedPartitions, e); + return e; + } + } + + return null; + } + + Exception invokePartitionsRevoked(final SortedSet<TopicPartition> revokedPartitions) { + log.info("Revoke previously assigned partitions {}", Utils.join(revokedPartitions, ", ")); + Set<TopicPartition> 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, ", ")); + + Optional<ConsumerRebalanceListener> listener = subscriptions.rebalanceListener(); + + if (listener.isPresent()) { + try { + final long startMs = time.milliseconds(); + listener.get().onPartitionsRevoked(revokedPartitions); + coordinatorMetrics.revokeCallbackSensor.record(time.milliseconds() - startMs); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { Review Comment: I can't speak to all the different use cases, but this is what is done in the existing `ConsumerCoordinator`, so this was done to keep the existing behavior equivalent. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -1081,4 +1026,74 @@ boolean updateAssignmentMetadataIfNeeded(Timer timer) { // logic return updateFetchPositions(timer); } + + @Override + public void subscribe(Collection<String> topics) { + subscribeInternal(topics, Optional.empty()); + } + + @Override + public void subscribe(Collection<String> topics, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + subscribeInternal(topics, Optional.of(listener)); + } + + @Override + public void subscribe(Pattern pattern) { + subscribeInternal(pattern, Optional.empty()); + } + + @Override + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + subscribeInternal(pattern, Optional.of(listener)); + } + + private void subscribeInternal(Pattern pattern, Optional<ConsumerRebalanceListener> listener) { + maybeThrowInvalidGroupIdException(); + if (pattern == null || pattern.toString().isEmpty()) + throw new IllegalArgumentException("Topic pattern to subscribe to cannot be " + (pattern == null ? + "null" : "empty")); + + throwIfNoAssignorsConfigured(); + log.info("Subscribed to pattern: '{}'", pattern); + subscriptions.subscribe(pattern, listener); + updatePatternSubscription(metadata.fetch()); + metadata.requestUpdateForNewTopics(); + } + + private void subscribeInternal(Collection<String> topics, Optional<ConsumerRebalanceListener> listener) { + maybeThrowInvalidGroupIdException(); + if (topics == null) + throw new IllegalArgumentException("Topic collection to subscribe to cannot be null"); + if (topics.isEmpty()) { + // treat subscribing to empty topic list as the same as unsubscribing + unsubscribe(); + } else { + for (String topic : topics) { + if (isBlank(topic)) + throw new IllegalArgumentException("Topic collection to subscribe to cannot contain null or empty topic"); + } + + throwIfNoAssignorsConfigured(); + + // Clear the buffered data which are not a part of newly assigned topics + final Set<TopicPartition> currentTopicPartitions = new HashSet<>(); + + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (topics.contains(tp.topic())) Review Comment: For topics, yes. I'm not sure about the regex version, though. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -1081,4 +1026,74 @@ boolean updateAssignmentMetadataIfNeeded(Timer timer) { // logic return updateFetchPositions(timer); } + + @Override + public void subscribe(Collection<String> topics) { + subscribeInternal(topics, Optional.empty()); + } + + @Override + public void subscribe(Collection<String> topics, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + subscribeInternal(topics, Optional.of(listener)); + } + + @Override + public void subscribe(Pattern pattern) { + subscribeInternal(pattern, Optional.empty()); + } + + @Override + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + subscribeInternal(pattern, Optional.of(listener)); + } + + private void subscribeInternal(Pattern pattern, Optional<ConsumerRebalanceListener> listener) { Review Comment: That's an interesting optimization. I am not sure if that will introduce subtle inconsistencies, so I'd prefer to tackle that separately. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ########## @@ -285,10 +285,8 @@ public synchronized void assignFromSubscribed(Collection<TopicPartition> assignm this.assignment.set(assignedPartitionStates); } - private void registerRebalanceListener(ConsumerRebalanceListener listener) { - if (listener == null) - throw new IllegalArgumentException("RebalanceListener cannot be null"); - this.rebalanceListener = listener; + private void registerRebalanceListener(Optional<ConsumerRebalanceListener> listener) { + this.rebalanceListener = Objects.requireNonNull(listener, "RebalanceListener cannot be null"); Review Comment: This is me being very paranoid about the `Optional` itself being `null`. I know, it is a bit confusing 😞 ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ########## @@ -285,10 +285,8 @@ public synchronized void assignFromSubscribed(Collection<TopicPartition> assignm this.assignment.set(assignedPartitionStates); } - private void registerRebalanceListener(ConsumerRebalanceListener listener) { - if (listener == null) - throw new IllegalArgumentException("RebalanceListener cannot be null"); - this.rebalanceListener = listener; + private void registerRebalanceListener(Optional<ConsumerRebalanceListener> listener) { + this.rebalanceListener = Objects.requireNonNull(listener, "RebalanceListener cannot be null"); Review Comment: This is me being very paranoid about the `Optional` itself being `null`. I know, it is a bit confusing 😞 ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ########## @@ -162,13 +162,13 @@ else if (this.subscriptionType != type) throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE); } - public synchronized boolean subscribe(Set<String> topics, ConsumerRebalanceListener listener) { + public synchronized boolean subscribe(Set<String> topics, Optional<ConsumerRebalanceListener> listener) { Review Comment: That's similar to how I had it before, but we _do_ want to throw an exception if the user passes in a `null` to the `KafkaConsumer.subscribe()` methods. So the `null` check is handled in the `KafkaConsumer` and converted to an `Optional` when we call the `SubscriptionState`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org