cadonna commented on code in PR #18551: URL: https://github.com/apache/kafka/pull/18551#discussion_r1935146957
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java: ########## @@ -0,0 +1,1009 @@ +/* + * 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.StreamsOnAllTasksLostCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksAssignedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksRevokedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.metrics.ConsumerRebalanceMetricsManager; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.StreamsGroupHeartbeatRequest; +import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Tracks state the state of a single member in relationship to a group: + * <p/> + * Responsible for: + * <ul> + * <li>Keeping member state</li> + * <li>Keeping assignment for the member</li> + * <li>Computing assignment for the group if the member is required to do so</li> + * <li>Calling the assignment and revocation callbacks on the Streams client</li> + * </ul> + */ +public class StreamsMembershipManager implements RequestManager { + + /** + * A data structure to represent the current task assignment, and current target task assignment of a member in a + * streams group. + * <p/> + * Besides the assigned tasks, it contains a local epoch that is bumped whenever the assignment changes, to ensure + * that two assignments with the same tasks but different local epochs are not considered equal. + */ + private static class LocalAssignment { + public static final long NONE_EPOCH = -1; + public static final LocalAssignment NONE = new LocalAssignment( + NONE_EPOCH, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap() + ); + + public final long localEpoch; + public final Map<String, SortedSet<Integer>> activeTasks; + public final Map<String, SortedSet<Integer>> standbyTasks; + public final Map<String, SortedSet<Integer>> warmupTasks; + + public LocalAssignment(final long localEpoch, + final Map<String, SortedSet<Integer>> activeTasks, + final Map<String, SortedSet<Integer>> standbyTasks, + final Map<String, SortedSet<Integer>> warmupTasks) { + this.localEpoch = localEpoch; + this.activeTasks = activeTasks; + this.standbyTasks = standbyTasks; + this.warmupTasks = warmupTasks; + if (localEpoch == NONE_EPOCH && + (!activeTasks.isEmpty() || !standbyTasks.isEmpty() || !warmupTasks.isEmpty())) { + throw new IllegalArgumentException("Local epoch must be set if tasks are assigned."); + } + } + + Optional<LocalAssignment> updateWith(final Map<String, SortedSet<Integer>> activeTasks, + final Map<String, SortedSet<Integer>> standbyTasks, + final Map<String, SortedSet<Integer>> warmupTasks) { + if (localEpoch != NONE_EPOCH) { + if (activeTasks.equals(this.activeTasks) && + standbyTasks.equals(this.standbyTasks) && + warmupTasks.equals(this.warmupTasks)) { + + return Optional.empty(); + } + } + + long nextLocalEpoch = localEpoch + 1; + return Optional.of(new LocalAssignment(nextLocalEpoch, activeTasks, standbyTasks, warmupTasks)); + } + + @Override + public String toString() { + return "LocalAssignment{" + + "localEpoch=" + localEpoch + + ", activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + LocalAssignment that = (LocalAssignment) o; + return localEpoch == that.localEpoch && + Objects.equals(activeTasks, that.activeTasks) && + Objects.equals(standbyTasks, that.standbyTasks) && + Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(localEpoch, activeTasks, standbyTasks, warmupTasks); + } + } + + static final Utils.TopicPartitionComparator TOPIC_PARTITION_COMPARATOR = new Utils.TopicPartitionComparator(); + + private final Logger log; + + private final StreamsRebalanceEventsProcessor streamsRebalanceEventsProcessor; + + private final StreamsRebalanceData streamsRebalanceData; + + private final SubscriptionState subscriptionState; + + private MemberState state; + + private final String groupId; + + private final String memberId = Uuid.randomUuid().toString(); + + private final Optional<String> groupInstanceId = Optional.empty(); + + private int memberEpoch = 0; + + private Optional<CompletableFuture<Void>> leaveGroupInProgress = Optional.empty(); + + private CompletableFuture<Void> staleMemberAssignmentRelease; + + private boolean reconciliationInProgress; + + private boolean rejoinedWhileReconciliationInProgress; + + private final List<MemberStateListener> stateUpdatesListeners = new ArrayList<>(); + + private LocalAssignment targetAssignment = LocalAssignment.NONE; + + private LocalAssignment currentAssignment = LocalAssignment.NONE; + + private final AtomicBoolean subscriptionUpdated = new AtomicBoolean(false); + + private final RebalanceMetricsManager metricsManager; + + private final Time time; + + private boolean isPollTimerExpired; + + /** + * Constructs the Streams membership manager. + * + * @param groupId The ID of the group. + * @param streamsRebalanceEventsProcessor The processor that handles Streams rebalance events like requests for + * invocation of assignment/revocation callbacks. + * @param streamsRebalanceData Data needed to participate in the Streams rebalance protocol. + * @param subscriptionState The subscription state of the member. + * @param logContext The log context. + * @param time The time. + * @param metrics The metrics. + */ + public StreamsMembershipManager(final String groupId, + final StreamsRebalanceEventsProcessor streamsRebalanceEventsProcessor, + final StreamsRebalanceData streamsRebalanceData, + final SubscriptionState subscriptionState, + final LogContext logContext, + final Time time, + final Metrics metrics) { + log = logContext.logger(StreamsMembershipManager.class); + this.state = MemberState.UNSUBSCRIBED; + this.groupId = groupId; + this.streamsRebalanceEventsProcessor = streamsRebalanceEventsProcessor; + this.streamsRebalanceData = streamsRebalanceData; + this.subscriptionState = subscriptionState; + metricsManager = new ConsumerRebalanceMetricsManager(metrics); + this.time = time; + } + + public String groupId() { + return groupId; + } + + public String memberId() { + return memberId; + } + + public Optional<String> groupInstanceId() { + return groupInstanceId; + } + + public int memberEpoch() { + return memberEpoch; + } + + public MemberState state() { + return state; + } + + public boolean isLeavingGroup() { + MemberState state = state(); + return state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING; + } + + private boolean isNotInGroup() { + MemberState state = state(); + return state == MemberState.UNSUBSCRIBED || + state == MemberState.FENCED || + state == MemberState.FATAL || + state == MemberState.STALE; + } + + public void registerStateListener(MemberStateListener listener) { + stateUpdatesListeners.add(Objects.requireNonNull(listener, "State updates listener cannot be null")); Review Comment: Not allowing duplicate listener makes sense to me. However, I would prefer to check for reference equality instead of using a `Set` since the latter requires `equals()` and `hashCode()` being implemented which is not needed for listener IMO. -- 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