This is an automated email from the ASF dual-hosted git repository.

cadonna pushed a commit to branch kip1071
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit dcb0c923ff7b28285cdb1ce692888de6cf4b47a2
Author: Bruno Cadonna <[email protected]>
AuthorDate: Tue Oct 29 15:29:49 2024 +0100

    Introduce the streams membership manager (#17564)
    
    The streams membership manager manages the state of the member
    and is responsible for reconciling the received task assignment
    with the current task assignment of the member. Additionally,
    it requests the call of the callback that installs the
    reconciled task assignment in the Streams client.
---
 .../internals/StreamsAssignmentInterface.java      |  27 +-
 .../StreamsGroupHeartbeatRequestManager.java       |   4 +-
 .../internals/StreamsMembershipManager.java        | 832 +++++++++++++++++++
 .../internals/events/ApplicationEvent.java         |   1 +
 .../consumer/internals/events/BackgroundEvent.java |   3 +-
 .../StreamsOnAssignmentCallbackCompletedEvent.java |  51 ++
 .../StreamsOnAssignmentCallbackNeededEvent.java    |  42 +
 .../StreamsGroupHeartbeatRequestManagerTest.java   |  12 +-
 .../internals/StreamsMembershipManagerTest.java    | 905 +++++++++++++++++++++
 .../streams/processor/internals/StreamThread.java  |   6 +-
 10 files changed, 1868 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsAssignmentInterface.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsAssignmentInterface.java
index cefe9175b68..650edaa2de2 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsAssignmentInterface.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsAssignmentInterface.java
@@ -200,10 +200,10 @@ public class StreamsAssignmentInterface {
         }
     }
 
-    public static class TaskId {
+    public static class TaskId implements Comparable<TaskId> {
 
-        public final String subtopologyId;
-        public final int partitionId;
+        private final String subtopologyId;
+        private final int partitionId;
 
         public int partitionId() {
             return partitionId;
@@ -218,6 +218,27 @@ public class StreamsAssignmentInterface {
             this.partitionId = partitionId;
         }
 
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            TaskId taskId = (TaskId) o;
+            return partitionId == taskId.partitionId && 
Objects.equals(subtopologyId, taskId.subtopologyId);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(subtopologyId, partitionId);
+        }
+
+        @Override
+        public int compareTo(TaskId taskId) {
+            if (subtopologyId.equals(taskId.subtopologyId)) {
+                return partitionId - taskId.partitionId;
+            }
+            return subtopologyId.compareTo(taskId.subtopologyId);
+        }
+
         @Override
         public String toString() {
             return "TaskId{" +
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
index 896800cff3c..5fc15b7e436 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
@@ -114,8 +114,7 @@ public class StreamsGroupHeartbeatRequestManager implements 
RequestManager {
 
     @Override
     public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
-        if (!coordinatorRequestManager.coordinator().isPresent() ||
-            membershipManager.shouldSkipHeartbeat()) {
+        if (!coordinatorRequestManager.coordinator().isPresent() || 
membershipManager.shouldSkipHeartbeat()) {
             membershipManager.onHeartbeatRequestSkipped();
             return NetworkClientDelegate.PollResult.EMPTY;
         }
@@ -221,6 +220,7 @@ public class StreamsGroupHeartbeatRequestManager implements 
RequestManager {
     private void onFailure(final Throwable exception, final long 
responseTimeMs) {
         this.heartbeatRequestState.onFailedAttempt(responseTimeMs);
         this.heartbeatState.reset();
+        membershipManager.onHeartbeatFailure(exception instanceof 
RetriableException);
         if (exception instanceof RetriableException) {
             String message = String.format("StreamsGroupHeartbeatRequest 
failed because of the retriable exception. " +
                     "Will retry in %s ms: %s",
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java
new file mode 100644
index 00000000000..ef39f661f67
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java
@@ -0,0 +1,832 @@
+/*
+ * 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.BackgroundEventHandler;
+import 
org.apache.kafka.clients.consumer.internals.events.StreamsOnAssignmentCallbackCompletedEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.StreamsOnAssignmentCallbackNeededEvent;
+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.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.telemetry.internals.ClientTelemetryProvider;
+import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+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>
+ * </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 StreamsAssignmentInterface streamsAssignmentInterface;
+
+    private final SubscriptionState subscriptionState;
+
+    private final BackgroundEventHandler backgroundEventHandler;
+
+    private MemberState state;
+
+    private final String groupId;
+
+    private String memberId = "";
+
+    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 Optional<ClientTelemetryReporter> clientTelemetryReporter;
+
+    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;
+
+    public StreamsMembershipManager(final String groupId,
+                                    final StreamsAssignmentInterface 
streamsAssignmentInterface,
+                                    final SubscriptionState subscriptionState,
+                                    final LogContext logContext,
+                                    final Optional<ClientTelemetryReporter> 
clientTelemetryReporter,
+                                    final BackgroundEventHandler 
backgroundEventHandler,
+                                    final Time time,
+                                    final Metrics metrics) {
+        log = logContext.logger(StreamsMembershipManager.class);
+        this.state = MemberState.UNSUBSCRIBED;
+        this.groupId = groupId;
+        this.streamsAssignmentInterface = streamsAssignmentInterface;
+        this.subscriptionState = subscriptionState;
+        this.clientTelemetryReporter = clientTelemetryReporter;
+        this.backgroundEventHandler = backgroundEventHandler;
+        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;
+    }
+
+    private void transitionToJoining() {
+        if (state == MemberState.FATAL) {
+            log.warn("No action taken to join the group with the updated 
subscription because " +
+                "the member is in FATAL state");
+            return;
+        }
+        if (reconciliationInProgress) {
+            rejoinedWhileReconciliationInProgress = true;
+        }
+        resetEpoch();
+        transitionTo(MemberState.JOINING);
+        clearPendingTaskAssignment();
+    }
+
+    private void transitionToSendingLeaveGroup(boolean dueToExpiredPollTimer) {
+        if (state == MemberState.FATAL) {
+            log.warn("Member {} with epoch {} won't send leave group request 
because it is in " +
+                "FATAL state", memberIdInfoForLog(), memberEpoch);
+            return;
+        }
+        if (state == MemberState.UNSUBSCRIBED) {
+            log.warn("Member {} won't send leave group request because it is 
already out of the group.",
+                memberIdInfoForLog());
+            return;
+        }
+
+        if (dueToExpiredPollTimer) {
+            isPollTimerExpired = true;
+            // Briefly transition through prepare leaving. The member does not 
have to release
+            // any assignment before sending the leave group given that is 
stale. It will invoke
+            // onTaskAssignment with empty assignment after sending the leave 
group on the STALE state.
+            transitionTo(MemberState.PREPARE_LEAVING);
+        }
+        finalizeLeaving();
+        transitionTo(MemberState.LEAVING);
+    }
+
+    private void finalizeLeaving() {
+        
updateMemberEpoch(StreamsGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH);
+        clearPendingTaskAssignment();
+    }
+
+    private void transitionToStale() {
+        transitionTo(MemberState.STALE);
+
+        CompletableFuture<Void> onAllTasksRevokedDone = 
invokeRevokingAllTasksCallback();
+        staleMemberAssignmentRelease = 
onAllTasksRevokedDone.whenComplete((result, error) -> {
+            if (error != null) {
+                log.error("Task revocation callback invocation failed " +
+                    "after member left group due to expired poll timer.", 
error);
+            }
+            clearTaskAndPartitionAssignment();
+            log.debug("Member {} sent leave group heartbeat and released its 
assignment. It will remain " +
+                    "in {} state until the poll timer is reset, and it will 
then rejoin the group",
+                memberIdInfoForLog(), MemberState.STALE);
+        });
+    }
+
+    public void transitionToFatal() {
+        MemberState previousState = state;
+        transitionTo(MemberState.FATAL);
+        log.error("Member {} with epoch {} transitioned to fatal state", 
memberIdInfoForLog(), memberEpoch);
+
+        if (previousState == MemberState.UNSUBSCRIBED) {
+            log.debug("Member {} with epoch {} got fatal error from the broker 
but it already " +
+                "left the group, so onTaskAssignment callback won't be 
triggered.", memberIdInfoForLog(), memberEpoch);
+            return;
+        }
+
+        if (previousState == MemberState.LEAVING || previousState == 
MemberState.PREPARE_LEAVING) {
+            log.info("Member {} with epoch {} was leaving the group with state 
{} when it got a " +
+                "fatal error from the broker. It will discard the ongoing 
leave and remain in " +
+                "fatal state.", memberIdInfoForLog(), memberEpoch, 
previousState);
+            maybeCompleteLeaveInProgress();
+            return;
+        }
+
+        CompletableFuture<Void> callbackResult = 
invokeRevokingAllTasksCallback();
+        callbackResult.whenComplete((result, error) -> {
+            if (error != null) {
+                log.error("onTaskAssignment callback invocation failed while 
releasing assignment" +
+                    "after member failed with fatal error.", error);
+            }
+            clearTaskAndPartitionAssignment();
+        });
+    }
+
+    public void transitionToUnsubscribeIfLeaving() {
+        if (state == MemberState.LEAVING) {
+            log.warn("Heartbeat to leave group cannot be sent (most probably 
due to coordinator " +
+                    "not known/available). Member {} with epoch {} will 
transition to {}.",
+                memberIdInfoForLog(), memberEpoch, MemberState.UNSUBSCRIBED);
+            transitionTo(MemberState.UNSUBSCRIBED);
+            maybeCompleteLeaveInProgress();
+        }
+    }
+
+    private void transitionTo(MemberState nextState) {
+        if (!state.equals(nextState) && 
!nextState.getPreviousValidStates().contains(state)) {
+            throw new IllegalStateException(String.format("Invalid state 
transition from %s to %s",
+                state, nextState));
+        }
+
+        if (isCompletingRebalance(state, nextState)) {
+            metricsManager.recordRebalanceEnded(time.milliseconds());
+        }
+        if (isStartingRebalance(state, nextState)) {
+            metricsManager.recordRebalanceStarted(time.milliseconds());
+        }
+
+        log.info("Member {} with epoch {} transitioned from {} to {}.", 
memberIdInfoForLog(), memberEpoch, state, nextState);
+        this.state = nextState;
+    }
+
+    private static boolean isCompletingRebalance(MemberState currentState, 
MemberState nextState) {
+        return currentState == MemberState.RECONCILING &&
+            (nextState == MemberState.STABLE || nextState == 
MemberState.ACKNOWLEDGING);
+    }
+
+    private static boolean isStartingRebalance(MemberState currentState, 
MemberState nextState) {
+        return currentState != MemberState.RECONCILING && nextState == 
MemberState.RECONCILING;
+    }
+
+    private void resetEpoch() {
+        
updateMemberEpoch(StreamsGroupHeartbeatRequest.JOIN_GROUP_MEMBER_EPOCH);
+    }
+
+    private void updateMemberEpoch(int newEpoch) {
+        memberEpoch = newEpoch;
+    }
+
+    private void clearPendingTaskAssignment() {
+        currentAssignment = LocalAssignment.NONE;
+    }
+
+    private void clearTaskAndPartitionAssignment() {
+        subscriptionState.assignFromSubscribed(Collections.emptySet());
+        currentAssignment = LocalAssignment.NONE;
+        targetAssignment = LocalAssignment.NONE;
+    }
+
+    public boolean shouldSkipHeartbeat() {
+        return isNotInGroup();
+    }
+
+    public boolean shouldHeartbeatNow() {
+        MemberState state = state();
+        return state == MemberState.ACKNOWLEDGING || state == 
MemberState.LEAVING || state == MemberState.JOINING;
+    }
+
+    public void onSubscriptionUpdated() {
+        subscriptionUpdated.compareAndSet(false, true);
+    }
+
+    public void onConsumerPoll() {
+        if (subscriptionUpdated.compareAndSet(true, false) && state == 
MemberState.UNSUBSCRIBED) {
+            transitionToJoining();
+        }
+    }
+
+    public void onHeartbeatRequestGenerated() {
+        MemberState state = state();
+        if (state == MemberState.ACKNOWLEDGING) {
+            if (targetAssignmentReconciled()) {
+                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 " +
+                    "be reconciled.", memberIdInfoForLog(), memberEpoch, 
MemberState.RECONCILING);
+                transitionTo(MemberState.RECONCILING);
+            }
+        } else if (state == MemberState.LEAVING) {
+            if (isPollTimerExpired) {
+                log.debug("Member {} with epoch {} generated the heartbeat to 
leave due to expired poll timer. It will " +
+                    "remain stale (no heartbeat) until it rejoins the group on 
the next consumer " +
+                    "poll.", memberIdInfoForLog(), memberEpoch);
+                transitionToStale();
+            } else {
+                log.debug("Member {} with epoch {} generated the heartbeat to 
leave the group.", memberIdInfoForLog(), memberEpoch);
+                transitionTo(MemberState.UNSUBSCRIBED);
+            }
+        }
+    }
+
+    public void onHeartbeatSuccess(StreamsGroupHeartbeatResponse response) {
+        StreamsGroupHeartbeatResponseData responseData = response.data();
+        throwIfUnexpectedError(responseData);
+        if (state == MemberState.LEAVING) {
+            log.debug("Ignoring heartbeat response received from broker. 
Member {} with epoch {} is " +
+                "already leaving the group.", memberId, memberEpoch);
+            return;
+        }
+        if (state == MemberState.UNSUBSCRIBED && 
maybeCompleteLeaveInProgress()) {
+            log.debug("Member {} with epoch {} received a successful response 
to the heartbeat " +
+                "to leave the group and completed the leave operation. ", 
memberId, memberEpoch);
+            return;
+        }
+        if (isNotInGroup()) {
+            log.debug("Ignoring heartbeat response received from broker. 
Member {} is in {} state" +
+                " so it's not a member of the group. ", memberId, state);
+            return;
+        }
+
+        // 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 (responseData.memberId() != null && 
!responseData.memberId().equals(memberId)) {
+            clientTelemetryReporter.ifPresent(reporter -> 
reporter.updateMetricsLabels(
+                
Collections.singletonMap(ClientTelemetryProvider.GROUP_MEMBER_ID, 
responseData.memberId())));
+        }
+
+        memberId = responseData.memberId();
+        updateMemberEpoch(responseData.memberEpoch());
+
+        final List<StreamsGroupHeartbeatResponseData.TaskIds> activeTasks = 
responseData.activeTasks();
+        final List<StreamsGroupHeartbeatResponseData.TaskIds> standbyTasks = 
responseData.standbyTasks();
+        final List<StreamsGroupHeartbeatResponseData.TaskIds> warmupTasks = 
responseData.warmupTasks();
+
+        if (activeTasks != null && standbyTasks != null && warmupTasks != 
null) {
+
+            if (!state.canHandleNewAssignment()) {
+                log.debug("Ignoring new assignment: active tasks {}, standby 
tasks {}, and warm-up tasks {} received " +
+                        "from server because member is in {} state.",
+                    activeTasks, standbyTasks, warmupTasks, state);
+                return;
+            }
+
+            processAssignmentReceived(
+                toTasksAssignment(activeTasks),
+                toTasksAssignment(standbyTasks),
+                toTasksAssignment(warmupTasks)
+            );
+        } else {
+            if (responseData.activeTasks() != null ||
+                responseData.standbyTasks() != null ||
+                responseData.warmupTasks() != null) {
+
+                throw new IllegalStateException("Invalid response data, task 
collections must be all null or all non-null: "
+                    + responseData);
+            }
+        }
+    }
+
+    public void onHeartbeatFailure(boolean retriable) {
+        if (!retriable) {
+            metricsManager.maybeRecordRebalanceFailed();
+        }
+        // The leave group request is sent out once (not retried), so we 
should complete the leave
+        // operation once the request completes, regardless of the response.
+        if (state == MemberState.UNSUBSCRIBED && 
maybeCompleteLeaveInProgress()) {
+            log.warn("Member {} with epoch {} received a failed response to 
the heartbeat to " +
+                "leave the group and completed the leave operation. ", 
memberIdInfoForLog(), memberEpoch);
+        }
+    }
+
+    public void onPollTimerExpired() {
+        transitionToSendingLeaveGroup(true);
+    }
+
+    public void onFenced() {
+        if (state == MemberState.PREPARE_LEAVING) {
+            log.info("Member {} with epoch {} got fenced but it is already 
preparing to leave " +
+                "the group, so it will stop sending heartbeat and won't 
attempt to send the " +
+                "leave request or rejoin.", memberIdInfoForLog(), memberEpoch);
+            finalizeLeaving();
+            transitionTo(MemberState.UNSUBSCRIBED);
+            maybeCompleteLeaveInProgress();
+            return;
+        }
+
+        if (state == MemberState.LEAVING) {
+            log.debug("Member {} with epoch {} got fenced before sending leave 
group heartbeat. " +
+                "It will not send the leave request and won't attempt to 
rejoin.", memberIdInfoForLog(), memberEpoch);
+            transitionTo(MemberState.UNSUBSCRIBED);
+            maybeCompleteLeaveInProgress();
+            return;
+        }
+        if (state == MemberState.UNSUBSCRIBED) {
+            log.debug("Member {} with epoch {} got fenced but it already left 
the group, so it " +
+                "won't attempt to rejoin.", memberIdInfoForLog(), memberEpoch);
+            return;
+        }
+        transitionTo(MemberState.FENCED);
+        resetEpoch();
+        log.debug("Member {} with epoch {} transitioned to {} state. It will 
release its " +
+            "assignment and rejoin the group.", memberIdInfoForLog(), 
memberEpoch, MemberState.FENCED);
+
+        CompletableFuture<Void> callbackResult = 
invokeRevokingAllTasksCallback();
+        callbackResult.whenComplete((result, error) -> {
+            if (error != null) {
+                log.error("onTaskAssignment callback invocation failed while 
releasing assignment" +
+                    " after member got fenced. Member will rejoin the group 
anyways.", error);
+            }
+            clearTaskAndPartitionAssignment();
+            if (state == MemberState.FENCED) {
+                transitionToJoining();
+            } else {
+                log.debug("Fenced member onTaskAssignment callback completed 
but the state has " +
+                    "already changed to {}, so the member won't rejoin the 
group", state);
+            }
+        });
+    }
+
+    private void throwIfUnexpectedError(StreamsGroupHeartbeatResponseData 
responseData) {
+        if (responseData.errorCode() != Errors.NONE.code()) {
+            String errorMessage = String.format(
+                "Unexpected error in Heartbeat response. Expected no error, 
but received: %s with message: '%s'",
+                Errors.forCode(responseData.errorCode()), 
responseData.errorMessage()
+            );
+            throw new IllegalArgumentException(errorMessage);
+        }
+    }
+
+    public void maybeRejoinStaleMember() {
+        isPollTimerExpired = false;
+        if (state == MemberState.STALE) {
+            log.debug("Expired poll timer has been reset so stale member {} 
will rejoin the group " +
+                "when it completes releasing its previous assignment.", 
memberIdInfoForLog());
+            staleMemberAssignmentRelease.whenComplete((__, error) -> 
transitionToJoining());
+        }
+    }
+
+    private boolean maybeCompleteLeaveInProgress() {
+        if (leaveGroupInProgress.isPresent()) {
+            leaveGroupInProgress.get().complete(null);
+            leaveGroupInProgress = Optional.empty();
+            return true;
+        }
+        return false;
+    }
+
+    private static SortedSet<StreamsAssignmentInterface.TaskId> 
toTaskIdSet(final Map<String, SortedSet<Integer>> tasks) {
+        SortedSet<StreamsAssignmentInterface.TaskId> taskIdSet = new 
TreeSet<>();
+        for (final Map.Entry<String, SortedSet<Integer>> task : 
tasks.entrySet()) {
+            final String subtopologyId = task.getKey();
+            final SortedSet<Integer> partitions = task.getValue();
+            for (final int partition : partitions) {
+                taskIdSet.add(new 
StreamsAssignmentInterface.TaskId(subtopologyId, partition));
+            }
+        }
+        return taskIdSet;
+    }
+
+    private static Map<String, SortedSet<Integer>> toTasksAssignment(final 
List<StreamsGroupHeartbeatResponseData.TaskIds> taskIds) {
+        return taskIds.stream()
+            
.collect(Collectors.toMap(StreamsGroupHeartbeatResponseData.TaskIds::subtopologyId,
 taskId -> new TreeSet<>(taskId.partitions())));
+    }
+
+    /**
+     * Leaves the group.
+     *
+     * <p>
+     * This method does the following:
+     * <ol>
+     *     <li>Transitions member state to {@link 
MemberState#PREPARE_LEAVING}.</li>
+     *     <li>Requests the invocation of the revocation callback.</li>
+     *     <li>Once the revocation callback completes, it clears the current 
and target assignment, unsubscribes from
+     *     all topics and transitions the member state to {@link 
MemberState#LEAVING}.</li>
+     * </ol>
+     * States {@link MemberState#PREPARE_LEAVING} and {@link 
MemberState#LEAVING} cause the heartbeat request manager
+     * to send a leave group heartbeat.
+     * </p>
+     *
+     * @return future that will complete when the revocation callback 
execution completes and the heartbeat
+     *         to leave the group has been sent out.
+     */
+    public CompletableFuture<Void> leaveGroup() {
+        if (isNotInGroup()) {
+            if (state == MemberState.FENCED) {
+                clearTaskAndPartitionAssignment();
+                transitionTo(MemberState.UNSUBSCRIBED);
+            }
+            subscriptionState.unsubscribe();
+            return CompletableFuture.completedFuture(null);
+        }
+
+        if (state == MemberState.PREPARE_LEAVING || state == 
MemberState.LEAVING) {
+            log.debug("Leave group operation already in progress for member 
{}", memberIdInfoForLog());
+            return leaveGroupInProgress.get();
+        }
+
+        CompletableFuture<Void> onGroupLeft = new CompletableFuture<>();
+        leaveGroupInProgress = Optional.of(onGroupLeft);
+        CompletableFuture<Void> onAllTasksRevokedDone = prepareLeaving();
+        onAllTasksRevokedDone.whenComplete((__, callbackError) -> 
leaving(callbackError));
+
+        return onGroupLeft;
+    }
+
+    private CompletableFuture<Void> prepareLeaving() {
+        transitionTo(MemberState.PREPARE_LEAVING);
+        return invokeRevokingAllTasksCallback();
+    }
+
+    private void leaving(Throwable callbackError) {
+        if (callbackError != null) {
+            log.error("Member {} callback to revoke task assignment failed. It 
will proceed " +
+                    "to clear its assignment and send a leave group heartbeat",
+                memberIdInfoForLog(), callbackError);
+        } else {
+            log.info("Member {} completed callback to revoke task assignment. 
It will proceed " +
+                    "to clear its assignment and send a leave group heartbeat",
+                memberIdInfoForLog());
+        }
+        subscriptionState.unsubscribe();
+        clearTaskAndPartitionAssignment();
+        transitionToSendingLeaveGroup(false);
+    }
+
+    /**
+     * This will process the assignment received if it is different from the 
member's current
+     * assignment. If a new assignment is received, this will make sure 
reconciliation is attempted
+     * on the next call of `poll`. If another reconciliation is currently in 
process, the first `poll`
+     * after that reconciliation will trigger the new reconciliation.
+     *
+     * @param activeTasks Target active tasks assignment received from the 
broker.
+     * @param standbyTasks Target standby tasks assignment received from the 
broker.
+     * @param warmupTasks Target warm-up tasks assignment received from the 
broker.
+     */
+    private void processAssignmentReceived(Map<String, SortedSet<Integer>> 
activeTasks,
+                                           Map<String, SortedSet<Integer>> 
standbyTasks,
+                                           Map<String, SortedSet<Integer>> 
warmupTasks) {
+        replaceTargetAssignmentWithNewAssignment(activeTasks, standbyTasks, 
warmupTasks);
+        if (!targetAssignmentReconciled()) {
+            transitionTo(MemberState.RECONCILING);
+        } else {
+            log.debug("Target assignment {} received from the broker is equals 
to the member " +
+                    "current assignment {}. Nothing to reconcile.",
+                targetAssignment, currentAssignment);
+            // Make sure we transition the member back to STABLE if it was 
RECONCILING (ex.
+            // member was RECONCILING unresolved assignments that were just 
removed by the
+            // broker), or JOINING (member joining received empty assignment).
+            if (state == MemberState.RECONCILING || state == 
MemberState.JOINING) {
+                transitionTo(MemberState.STABLE);
+            }
+        }
+    }
+
+    private boolean targetAssignmentReconciled() {
+        return currentAssignment.equals(targetAssignment);
+    }
+
+    private void replaceTargetAssignmentWithNewAssignment(Map<String, 
SortedSet<Integer>> activeTasks,
+                                                          Map<String, 
SortedSet<Integer>> standbyTasks,
+                                                          Map<String, 
SortedSet<Integer>> warmupTasks) {
+        targetAssignment.updateWith(activeTasks, standbyTasks, warmupTasks)
+            .ifPresent(updatedAssignment -> {
+                log.debug("Target assignment updated from {} to {}. Member 
will reconcile it on the next poll.",
+                    targetAssignment, updatedAssignment);
+                targetAssignment = updatedAssignment;
+            });
+    }
+
+    @Override
+    public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
+        if (state == MemberState.RECONCILING) {
+            maybeReconcile();
+        }
+        return NetworkClientDelegate.PollResult.EMPTY;
+    }
+
+    private void maybeReconcile() {
+        if (targetAssignmentReconciled()) {
+            log.trace("Ignoring reconciliation attempt. Target assignment is 
equal to the " +
+                "current assignment.");
+            return;
+        }
+        if (reconciliationInProgress) {
+            log.trace("Ignoring reconciliation attempt. Another reconciliation 
is already in progress. Assignment " +
+                targetAssignment + " will be handled in the next 
reconciliation loop.");
+            return;
+        }
+
+        markReconciliationInProgress();
+
+        // ToDo: add standby and warmup tasks
+        SortedSet<StreamsAssignmentInterface.TaskId> assignedActiveTasks = 
toTaskIdSet(targetAssignment.activeTasks);
+
+        log.info("Assigned tasks with local epoch {}\n" +
+                "\tMember:                        {}\n" +
+                "\tActive tasks:                  {}\n",
+            targetAssignment.localEpoch,
+            memberIdInfoForLog(),
+            assignedActiveTasks
+        );
+
+        SortedSet<TopicPartition> ownedTopicPartitions = new 
TreeSet<>(TOPIC_PARTITION_COMPARATOR);
+        ownedTopicPartitions.addAll(subscriptionState.assignedPartitions());
+        SortedSet<TopicPartition> assignedTopicPartitions = 
topicPartitionsForActiveTasks(targetAssignment.activeTasks);
+        SortedSet<TopicPartition> assignedTopicPartitionsNotPreviouslyOwned =
+            assignedTopicPartitionsNotPreviouslyOwned(assignedTopicPartitions, 
ownedTopicPartitions);
+
+        subscriptionState.assignFromSubscribedAwaitingCallback(
+            assignedTopicPartitions,
+            assignedTopicPartitionsNotPreviouslyOwned
+        );
+
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            new StreamsOnAssignmentCallbackNeededEvent(new 
StreamsAssignmentInterface.Assignment(
+                assignedActiveTasks,
+                Collections.emptySet(),
+                Collections.emptySet()
+            ));
+        CompletableFuture<Void> onTasksAssignmentDone = 
onAssignmentCallbackNeededEvent.future();
+        backgroundEventHandler.add(onAssignmentCallbackNeededEvent);
+        // The current target assignment is captured to ensure that 
acknowledging the current assignment is done with
+        // the same target assignment that was used when this reconciliation 
was initiated.
+        LocalAssignment currentTargetAssignment = targetAssignment;
+        onTasksAssignmentDone.whenComplete((__, callbackError) -> {
+            if (callbackError != null) {
+                log.error("Reconciliation failed: onTasksAssignment callback 
invocation failed for tasks {}",
+                    targetAssignment, callbackError);
+                markReconciliationCompleted();
+            } else {
+                if (reconciliationInProgress && !maybeAbortReconciliation()) {
+                    
subscriptionState.enablePartitionsAwaitingCallback(assignedTopicPartitionsNotPreviouslyOwned);
+                    currentAssignment = currentTargetAssignment;
+                    transitionTo(MemberState.ACKNOWLEDGING);
+                    markReconciliationCompleted();
+                }
+            }
+        });
+    }
+
+    private SortedSet<TopicPartition> 
assignedTopicPartitionsNotPreviouslyOwned(final SortedSet<TopicPartition> 
assignedTopicPartitions,
+                                                                               
 final SortedSet<TopicPartition> ownedTopicPartitions) {
+        SortedSet<TopicPartition> assignedPartitionsNotPreviouslyOwned = new 
TreeSet<>(TOPIC_PARTITION_COMPARATOR);
+        assignedPartitionsNotPreviouslyOwned.addAll(assignedTopicPartitions);
+        assignedPartitionsNotPreviouslyOwned.removeAll(ownedTopicPartitions);
+        return assignedPartitionsNotPreviouslyOwned;
+    }
+
+    private SortedSet<TopicPartition> topicPartitionsForActiveTasks(final 
Map<String, SortedSet<Integer>> activeTasks) {
+        final SortedSet<TopicPartition> topicPartitions = new 
TreeSet<>(TOPIC_PARTITION_COMPARATOR);
+        activeTasks.forEach((subtopologyId, partitionIds) ->
+            Stream.concat(
+                
streamsAssignmentInterface.subtopologyMap().get(subtopologyId).sourceTopics.stream(),
+                
streamsAssignmentInterface.subtopologyMap().get(subtopologyId).repartitionSourceTopics.keySet().stream()
+            ).forEach(topic -> {
+                for (final int partitionId : partitionIds) {
+                    topicPartitions.add(new TopicPartition(topic, 
partitionId));
+                }
+            })
+        );
+        return topicPartitions;
+    }
+
+    private void markReconciliationCompleted() {
+        reconciliationInProgress = false;
+        rejoinedWhileReconciliationInProgress = false;
+    }
+
+    private boolean maybeAbortReconciliation() {
+        boolean shouldAbort = state != MemberState.RECONCILING || 
rejoinedWhileReconciliationInProgress;
+        if (shouldAbort) {
+            String reason = rejoinedWhileReconciliationInProgress ?
+                "the member has re-joined the group" :
+                "the member already transitioned out of the reconciling state 
into " + state;
+            log.info("Interrupting reconciliation that is not relevant anymore 
because " + reason);
+            markReconciliationCompleted();
+        }
+        return shouldAbort;
+    }
+
+    private void markReconciliationInProgress() {
+        reconciliationInProgress = true;
+        rejoinedWhileReconciliationInProgress = false;
+    }
+
+    private CompletableFuture<Void> invokeRevokingAllTasksCallback() {
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            new StreamsOnAssignmentCallbackNeededEvent(new 
StreamsAssignmentInterface.Assignment(
+                Collections.emptySet(),
+                Collections.emptySet(),
+                Collections.emptySet()
+            ));
+        backgroundEventHandler.add(onAssignmentCallbackNeededEvent);
+        return onAssignmentCallbackNeededEvent.future();
+    }
+
+    public void 
onTaskAssignmentCallbackCompleted(StreamsOnAssignmentCallbackCompletedEvent 
event) {
+        Optional<KafkaException> error = event.error();
+        CompletableFuture<Void> future = event.future();
+
+        if (error.isPresent()) {
+            Exception e = error.get();
+            log.warn("The onTaskAssignment callback completed with an error 
({}); " +
+                "signaling to continue to the next phase of rebalance", 
e.getMessage());
+            future.completeExceptionally(e);
+        } else {
+            log.debug("The onTaskAssignment callback completed successfully; 
signaling to continue to the next phase of rebalance");
+            future.complete(null);
+        }
+    }
+
+    private String memberIdInfoForLog() {
+        return (memberId == null || memberId.isEmpty()) ? "<no ID>" : memberId;
+    }
+}
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 406ca55b79b..dc0c393e279 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
@@ -40,6 +40,7 @@ public abstract class ApplicationEvent {
         SHARE_ACKNOWLEDGE_ON_CLOSE,
         SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK_REGISTRATION,
         SEEK_UNVALIDATED,
+        STREAMS_ON_ASSIGNMENT_CALLBACK_COMPLETED,
     }
 
     private final Type type;
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 7e9fdaed2d8..cffcca42f48 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
@@ -27,7 +27,8 @@ import java.util.Objects;
 public abstract class BackgroundEvent {
 
     public enum Type {
-        ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, 
SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK
+        ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, 
SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK,
+        STREAMS_ON_ASSIGNMENT_CALLBACK_NEEDED
     }
 
     private final Type type;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackCompletedEvent.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackCompletedEvent.java
new file mode 100644
index 00000000000..5b702efb62c
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackCompletedEvent.java
@@ -0,0 +1,51 @@
+/*
+ * 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.KafkaException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+public class StreamsOnAssignmentCallbackCompletedEvent extends 
ApplicationEvent {
+
+    private final CompletableFuture<Void> future;
+    private final Optional<KafkaException> error;
+
+    public StreamsOnAssignmentCallbackCompletedEvent(final 
CompletableFuture<Void> future,
+                                                     final 
Optional<KafkaException> error) {
+        super(Type.STREAMS_ON_ASSIGNMENT_CALLBACK_COMPLETED);
+        this.future = Objects.requireNonNull(future);
+        this.error = Objects.requireNonNull(error);
+    }
+
+    public CompletableFuture<Void> future() {
+        return future;
+    }
+
+    public Optional<KafkaException> error() {
+        return error;
+    }
+
+    @Override
+    protected String toStringBase() {
+        return super.toStringBase() +
+            ", future=" + future +
+            ", error=" + error;
+    }
+}
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackNeededEvent.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackNeededEvent.java
new file mode 100644
index 00000000000..8c2f5a3bc81
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/StreamsOnAssignmentCallbackNeededEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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.StreamsAssignmentInterface;
+
+import java.util.Objects;
+
+public class StreamsOnAssignmentCallbackNeededEvent extends 
CompletableBackgroundEvent<Void> {
+
+    private final StreamsAssignmentInterface.Assignment assignment;
+
+    public 
StreamsOnAssignmentCallbackNeededEvent(StreamsAssignmentInterface.Assignment 
assignment) {
+        super(Type.STREAMS_ON_ASSIGNMENT_CALLBACK_NEEDED, Long.MAX_VALUE);
+        this.assignment = Objects.requireNonNull(assignment);
+    }
+
+    public StreamsAssignmentInterface.Assignment assignment() {
+        return assignment;
+    }
+
+    @Override
+    protected String toStringBase() {
+        return super.toStringBase() +
+            ", assignment=" + assignment;
+    }
+}
+
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java
index 193c901e3b7..bec93f98f22 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java
@@ -303,18 +303,18 @@ class StreamsGroupHeartbeatRequestManagerTest {
         final Assignment targetAssignment = 
streamsAssignmentInterface.targetAssignment.get();
         assertEquals(1, targetAssignment.activeTasks.size());
         final TaskId activeTaskId = 
targetAssignment.activeTasks.stream().findFirst().get();
-        assertEquals(activeTaskId.subtopologyId, "0");
-        assertEquals(activeTaskId.partitionId, 0);
+        assertEquals(activeTaskId.subtopologyId(), "0");
+        assertEquals(activeTaskId.partitionId(), 0);
 
         assertEquals(1, targetAssignment.standbyTasks.size());
         final TaskId standbyTaskId = 
targetAssignment.standbyTasks.stream().findFirst().get();
-        assertEquals(standbyTaskId.subtopologyId, "1");
-        assertEquals(standbyTaskId.partitionId, 1);
+        assertEquals(standbyTaskId.subtopologyId(), "1");
+        assertEquals(standbyTaskId.partitionId(), 1);
 
         assertEquals(1, targetAssignment.warmupTasks.size());
         final TaskId warmupTaskId = 
targetAssignment.warmupTasks.stream().findFirst().get();
-        assertEquals(warmupTaskId.subtopologyId, "2");
-        assertEquals(warmupTaskId.partitionId, 2);
+        assertEquals(warmupTaskId.subtopologyId(), "2");
+        assertEquals(warmupTaskId.partitionId(), 2);
 
     }
 
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java
new file mode 100644
index 00000000000..4965f05198b
--- /dev/null
+++ 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java
@@ -0,0 +1,905 @@
+/*
+ * 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.BackgroundEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
+import 
org.apache.kafka.clients.consumer.internals.events.StreamsOnAssignmentCallbackCompletedEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.StreamsOnAssignmentCallbackNeededEvent;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+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.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+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.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+public class StreamsMembershipManagerTest {
+
+    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 static final String SUB_TOPOLOGY_ID_0 = "subtopology-0";
+    private static final String SUB_TOPOLOGY_ID_1 = "subtopology-1";
+
+    private static final String TOPIC_0 = "topic-0";
+    private static final String TOPIC_1 = "topic-1";
+
+    private static final int PARTITION_0 = 0;
+    private static final int PARTITION_1 = 1;
+
+    private Time time = new MockTime(0);
+    private Metrics metrics = new Metrics(time);
+
+    private StreamsMembershipManager membershipManager;
+
+    @Mock
+    private SubscriptionState subscriptionState;
+
+    @Mock
+    private StreamsAssignmentInterface streamsAssignmentInterface;
+
+    private Queue<BackgroundEvent> backgroundEventQueue = new LinkedList<>();
+    private BackgroundEventHandler backgroundEventHandler = new 
BackgroundEventHandler(backgroundEventQueue);
+
+    @BeforeEach
+    public void setup() {
+        membershipManager = new StreamsMembershipManager(
+            GROUP_ID,
+            streamsAssignmentInterface,
+            subscriptionState,
+            new LogContext("test"),
+            Optional.empty(),
+            backgroundEventHandler,
+            time,
+            metrics
+        );
+        verifyInStateUnsubscribed(membershipManager);
+    }
+
+    @Test
+    public void testUnexpectedErrorInHeartbeatResponse() {
+        final String errorMessage = "Nobody expects the Spanish Inquisition!";
+        final StreamsGroupHeartbeatResponseData responseData = new 
StreamsGroupHeartbeatResponseData()
+            .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code())
+            .setErrorMessage(errorMessage);
+        final StreamsGroupHeartbeatResponse response = new 
StreamsGroupHeartbeatResponse(responseData);
+
+        final IllegalArgumentException exception = assertThrows(
+            IllegalArgumentException.class,
+            () -> membershipManager.onHeartbeatSuccess(response)
+        );
+
+        assertEquals(
+            "Unexpected error in Heartbeat response. Expected no error, but 
received: "
+                + Errors.GROUP_AUTHORIZATION_FAILED.name()
+                + " with message: '" + errorMessage + "'",
+            exception.getMessage()
+        );
+    }
+
+    @Test
+    public void testActiveTasksAreNullInHeartbeatResponse() {
+        joining();
+        final StreamsGroupHeartbeatResponse response = 
makeHeartbeatResponse(null);
+
+        final IllegalStateException exception = assertThrows(
+            IllegalStateException.class,
+            () -> membershipManager.onHeartbeatSuccess(response)
+        );
+
+        assertEquals(
+            "Invalid response data, task collections must be all null or all 
non-null: " + response.data(),
+            exception.getMessage()
+        );
+    }
+
+    @Test
+    public void testJoining() {
+        joining();
+
+        verifyInStateJoining(membershipManager);
+        assertEquals(StreamsGroupHeartbeatRequest.JOIN_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
+    }
+
+    @Test
+    public void testReconcilingEmptyToSingleActiveTask() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 TOPIC_0);
+        joining();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = 
Set.of(new TopicPartition(TOPIC_0, PARTITION_0));
+        
verify(subscriptionState).assignFromSubscribedAwaitingCallback(expectedPartitionAssignment,
 expectedPartitionAssignment);
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks =
+            Set.of(new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_0));
+        final StreamsAssignmentInterface.Assignment expectedTaskAssignment = 
makeTaskAssignment(activeTasks);
+        assertEquals(expectedTaskAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void testReconcilingActiveTaskToDifferentActiveTask() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 TOPIC_0);
+        when(subscriptionState.assignedPartitions())
+            .thenReturn(Collections.emptySet())
+            .thenReturn(Set.of(new TopicPartition(TOPIC_0, PARTITION_0)));
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_1)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = 
Set.of(new TopicPartition(TOPIC_0, PARTITION_1));
+        
verify(subscriptionState).assignFromSubscribedAwaitingCallback(expectedPartitionAssignment,
 expectedPartitionAssignment);
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks = Set.of(
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_1)
+        );
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(activeTasks);
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void testReconcilingSingleActiveTaskToAdditionalActiveTask() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 TOPIC_0);
+        when(subscriptionState.assignedPartitions())
+            .thenReturn(Collections.emptySet())
+            .thenReturn(Set.of(new TopicPartition(TOPIC_0, PARTITION_0)));
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0, PARTITION_1)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = Set.of(
+            new TopicPartition(TOPIC_0, PARTITION_0),
+            new TopicPartition(TOPIC_0, PARTITION_1)
+        );
+        final Collection<TopicPartition> expectedAdditionalPartitionAssignment 
= Set.of(new TopicPartition(TOPIC_0, PARTITION_1));
+        verify(subscriptionState).assignFromSubscribedAwaitingCallback(
+            expectedPartitionAssignment,
+            expectedAdditionalPartitionAssignment
+        );
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks = Set.of(
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_0),
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_1)
+        );
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(activeTasks);
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedAdditionalPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedAdditionalPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void testReconcilingMultipleActiveTaskToSingleActiveTask() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 TOPIC_0);
+        when(subscriptionState.assignedPartitions())
+            .thenReturn(Collections.emptySet())
+            .thenReturn(Set.of(new TopicPartition(TOPIC_0, PARTITION_0), new 
TopicPartition(TOPIC_0, PARTITION_1)));
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0, PARTITION_1)));
+        acknowledging();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_1)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = Set.of(
+            new TopicPartition(TOPIC_0, PARTITION_1)
+        );
+        final Collection<TopicPartition> expectedAdditionalPartitionAssignment 
= Collections.emptySet();
+        verify(subscriptionState).assignFromSubscribedAwaitingCallback(
+            expectedPartitionAssignment,
+            expectedAdditionalPartitionAssignment
+        );
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks = Set.of(
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_1)
+        );
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(activeTasks);
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedAdditionalPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedAdditionalPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void 
testReconcilingEmptyToMultipleActiveTaskOfDifferentSubtopologies() {
+        setupStreamsAssignmentInterfaceWithTwoSubtopologies(
+            SUB_TOPOLOGY_ID_0, TOPIC_0,
+            SUB_TOPOLOGY_ID_1, TOPIC_1
+        );
+        joining();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0), SUB_TOPOLOGY_ID_1, List.of(PARTITION_0)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = Set.of(
+            new TopicPartition(TOPIC_0, PARTITION_0),
+            new TopicPartition(TOPIC_1, PARTITION_0)
+        );
+        
verify(subscriptionState).assignFromSubscribedAwaitingCallback(expectedPartitionAssignment,
 expectedPartitionAssignment);
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks = Set.of(
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_0),
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_1, 
PARTITION_0)
+        );
+        final StreamsAssignmentInterface.Assignment expectedTaskAssignment = 
makeTaskAssignment(activeTasks);
+        assertEquals(expectedTaskAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void 
testReconcilingEmptyToMultipleActiveTaskOfConcatenatedSubtopologies() {
+        setupStreamsAssignmentInterfaceWithTwoConcatenedSubtopologies(
+            SUB_TOPOLOGY_ID_0, TOPIC_0,
+            SUB_TOPOLOGY_ID_1, TOPIC_1
+        );
+        joining();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0), SUB_TOPOLOGY_ID_1, List.of(PARTITION_0)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = Set.of(
+            new TopicPartition(TOPIC_0, PARTITION_0),
+            new TopicPartition(TOPIC_1, PARTITION_0)
+        );
+        
verify(subscriptionState).assignFromSubscribedAwaitingCallback(expectedPartitionAssignment,
 expectedPartitionAssignment);
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks = Set.of(
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_0),
+            new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_1, 
PARTITION_0)
+        );
+        final StreamsAssignmentInterface.Assignment expectedTaskAssignment = 
makeTaskAssignment(activeTasks);
+        assertEquals(expectedTaskAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        
verify(subscriptionState).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    @Test
+    public void testReconcilingAndAssignmentCallbackFails() {
+        final String topicName = "test_topic";
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 topicName);
+        final Set<StreamsAssignmentInterface.TaskId> activeTasks =
+            Set.of(new StreamsAssignmentInterface.TaskId(SUB_TOPOLOGY_ID_0, 
PARTITION_0));
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(activeTasks);
+        joining();
+
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+
+        final Collection<TopicPartition> expectedPartitionAssignment = 
Set.of(new TopicPartition(topicName, PARTITION_0));
+        
verify(subscriptionState).assignFromSubscribedAwaitingCallback(expectedPartitionAssignment,
 expectedPartitionAssignment);
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+        verifyInStateReconciling(membershipManager);
+        onAssignmentCallbackNeededEvent.future().completeExceptionally(new 
RuntimeException("KABOOM!"));
+        verifyInStateReconciling(membershipManager);
+        verify(subscriptionState, 
never()).enablePartitionsAwaitingCallback(expectedPartitionAssignment);
+    }
+
+    @Test
+    public void testLeaveGroupWhenNotInGroup() {
+        final CompletableFuture<Void> future = membershipManager.leaveGroup();
+
+        assertFalse(membershipManager.isLeavingGroup());
+        assertTrue(future.isDone());
+        assertFalse(future.isCancelled());
+        assertFalse(future.isCompletedExceptionally());
+        verify(subscriptionState).unsubscribe();
+        verifyInStateUnsubscribed(membershipManager);
+    }
+
+    @Test
+    public void testLeaveGroupWhenNotInGroupAndFenced() {
+        joining();
+        fenced();
+        final CompletableFuture<Void> future = membershipManager.leaveGroup();
+
+        assertFalse(membershipManager.isLeavingGroup());
+        assertTrue(future.isDone());
+        assertFalse(future.isCancelled());
+        assertFalse(future.isCompletedExceptionally());
+        verify(subscriptionState).unsubscribe();
+        verify(subscriptionState).assignFromSubscribed(Collections.emptySet());
+        verifyInStateUnsubscribed(membershipManager);
+    }
+
+    @Test
+    public void testLeaveGroupWhenInGroupWithAssignment() {
+        final StreamsAssignmentInterface.Assignment emptyStreamsAssignment = 
makeTaskAssignment(Collections.emptySet());
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        final CompletableFuture<Void> future = membershipManager.leaveGroup();
+
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        assertEquals(emptyStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        verify(subscriptionState, never()).unsubscribe();
+        verifyInStatePrepareLeaving(membershipManager);
+        final CompletableFuture<Void> futureBeforeRevocationCallback = 
membershipManager.leaveGroup();
+        assertEquals(future, futureBeforeRevocationCallback);
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        verify(subscriptionState).unsubscribe();
+        assertFalse(future.isDone());
+        verifyInStateLeaving(membershipManager);
+        final CompletableFuture<Void> futureAfterRevocationCallback = 
membershipManager.leaveGroup();
+        assertEquals(future, futureAfterRevocationCallback);
+        membershipManager.transitionToUnsubscribeIfLeaving();
+        verifyInStateUnsubscribed(membershipManager);
+    }
+
+    @Test
+    public void testTransitionToUnsubscribeWhenInLeaving() {
+        final StreamsGroupHeartbeatResponse response = 
makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, List.of(PARTITION_0));
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic");
+        joining();
+        reconcile(response);
+        acknowledging();
+        CompletableFuture<Void> future = leaving();
+
+        membershipManager.transitionToUnsubscribeIfLeaving();
+
+        verifyInStateUnsubscribed(membershipManager);
+        assertTrue(future.isDone());
+        assertFalse(future.isCancelled());
+        assertFalse(future.isCompletedExceptionally());
+    }
+
+    @Test
+    public void testOnPollTimerExpired() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        membershipManager.onPollTimerExpired();
+
+        verifyInStateLeaving(membershipManager);
+        assertEquals(StreamsGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
+    }
+
+    @Test
+    public void testOnHeartbeatRequestGeneratedWhenInAcknowleding() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        membershipManager.onHeartbeatRequestGenerated();
+
+        verifyInStateStable(membershipManager);
+    }
+
+    @Test
+    public void 
testOnHeartbeatRequestGeneratedWhenInAcknowledgingAndNewTargetAssignment() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_1)));
+        acknowledging();
+
+        membershipManager.onHeartbeatRequestGenerated();
+
+        verifyInStateReconciling(membershipManager);
+    }
+
+    @Test
+    public void testOnHeartbeatRequestGeneratedWhenInLeaving() {
+        joining();
+        leaving();
+
+        membershipManager.onHeartbeatRequestGenerated();
+
+        verifyInStateUnsubscribed(membershipManager);
+    }
+
+    @Test
+    public void 
testOnHeartbeatRequestGeneratedWhenInLeavingAndPollTimerExpired() {
+        joining();
+        membershipManager.onPollTimerExpired();
+
+        membershipManager.onHeartbeatRequestGenerated();
+
+        verifyInStateStale(membershipManager);
+    }
+
+    @Test
+    public void testOnFencedWhenInJoining() {
+        joining();
+
+        testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testOnFencedWhenInReconciling() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        backgroundEventQueue.poll();
+
+        testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testOnFencedWhenInAcknowledging() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testOnFencedWhenInStable() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+        stable();
+
+        testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    private void 
testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable() {
+        membershipManager.onFenced();
+
+        verifyInStateFenced(membershipManager);
+        assertEquals(StreamsGroupHeartbeatRequest.JOIN_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(Collections.emptySet());
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        verify(subscriptionState).assignFromSubscribed(Collections.emptySet());
+        verifyInStateJoining(membershipManager);
+    }
+
+    @Test
+    public void testOnFencedWhenInPrepareLeaving() {
+        joining();
+
+        testOnFencedWhenInPrepareLeavingOrLeaving(prepareLeaving());
+    }
+
+    @Test
+    public void testOnFencedWhenInLeaving() {
+        joining();
+
+        testOnFencedWhenInPrepareLeavingOrLeaving(leaving());
+    }
+
+    private void testOnFencedWhenInPrepareLeavingOrLeaving(final 
CompletableFuture<Void> onGroupLeft) {
+        membershipManager.onFenced();
+
+        verifyInStateUnsubscribed(membershipManager);
+        assertEquals(StreamsGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
+        assertTrue(onGroupLeft.isDone());
+        assertFalse(onGroupLeft.isCancelled());
+        assertFalse(onGroupLeft.isCompletedExceptionally());
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInPrepareLeaving() {
+        joining();
+
+        testTransitionToFatalWhenInPrepareLeavingOrLeaving(prepareLeaving());
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInLeaving() {
+        joining();
+
+        testTransitionToFatalWhenInPrepareLeavingOrLeaving(leaving());
+    }
+
+    private void testTransitionToFatalWhenInPrepareLeavingOrLeaving(final 
CompletableFuture<Void> onGroupLeft) {
+        membershipManager.transitionToFatal();
+
+        verifyInStateFatal(membershipManager);
+        assertTrue(onGroupLeft.isDone());
+        assertFalse(onGroupLeft.isCancelled());
+        assertFalse(onGroupLeft.isCompletedExceptionally());
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInJoining() {
+        joining();
+
+        
testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInReconciling() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        backgroundEventQueue.poll();
+
+        
testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInAcknowledging() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+
+        
testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    @Test
+    public void testTransitionToFatalWhenInStable() {
+        
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(SUB_TOPOLOGY_ID_0,
 "topic0");
+        joining();
+        reconcile(makeHeartbeatResponse(SUB_TOPOLOGY_ID_0, 
List.of(PARTITION_0)));
+        acknowledging();
+        stable();
+
+        
testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrStable();
+    }
+
+    private void 
testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrStable() {
+        membershipManager.transitionToFatal();
+
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        final StreamsAssignmentInterface.Assignment expectedStreamsAssignment 
= makeTaskAssignment(Collections.emptySet());
+        assertEquals(expectedStreamsAssignment, 
onAssignmentCallbackNeededEvent.assignment());
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        verify(subscriptionState).assignFromSubscribed(Collections.emptySet());
+        verifyInStateFatal(membershipManager);
+    }
+
+    @Test
+    public void testOnTaskAssignmentCallbackCompleted() {
+        final CompletableFuture<Void> future = new CompletableFuture<>();
+        final StreamsOnAssignmentCallbackCompletedEvent event = new 
StreamsOnAssignmentCallbackCompletedEvent(
+            future,
+            Optional.empty()
+        );
+
+        membershipManager.onTaskAssignmentCallbackCompleted(event);
+
+        assertTrue(future.isDone());
+        assertFalse(future.isCancelled());
+        assertFalse(future.isCompletedExceptionally());
+    }
+
+    @Test
+    public void testOnTaskAssignmentCallbackCompletedWhenCallbackFails() {
+        final String errorMessage = "KABOOM!";
+        final CompletableFuture<Void> future = new CompletableFuture<>();
+        final StreamsOnAssignmentCallbackCompletedEvent event = new 
StreamsOnAssignmentCallbackCompletedEvent(
+            future,
+            Optional.of(new KafkaException(errorMessage))
+        );
+
+        membershipManager.onTaskAssignmentCallbackCompleted(event);
+
+        assertTrue(future.isDone());
+        assertFalse(future.isCancelled());
+        assertTrue(future.isCompletedExceptionally());
+        final ExecutionException executionException = 
assertThrows(ExecutionException.class, future::get);
+        assertInstanceOf(KafkaException.class, executionException.getCause());
+        assertEquals(errorMessage, executionException.getCause().getMessage());
+    }
+
+    private static void verifyInStateReconciling(final 
StreamsMembershipManager membershipManager) {
+        assertEquals(MemberState.RECONCILING, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateAcknowledging(final 
StreamsMembershipManager membershipManager) {
+        assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state());
+        assertTrue(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateLeaving(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.LEAVING, membershipManager.state());
+        assertTrue(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertTrue(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStatePrepareLeaving(final 
StreamsMembershipManager membershipManager) {
+        assertEquals(MemberState.PREPARE_LEAVING, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertTrue(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateUnsubscribed(final 
StreamsMembershipManager membershipManager) {
+        assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertTrue(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateJoining(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.JOINING, membershipManager.state());
+        assertTrue(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateStable(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.STABLE, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertFalse(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateFenced(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.FENCED, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertTrue(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateFatal(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.FATAL, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertTrue(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private static void verifyInStateStale(final StreamsMembershipManager 
membershipManager) {
+        assertEquals(MemberState.STALE, membershipManager.state());
+        assertFalse(membershipManager.shouldHeartbeatNow());
+        assertTrue(membershipManager.shouldSkipHeartbeat());
+        assertFalse(membershipManager.isLeavingGroup());
+    }
+
+    private void 
setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic(final String 
subtopologyId,
+                                                                               
  final String topicName) {
+        when(streamsAssignmentInterface.subtopologyMap()).thenReturn(
+            mkMap(
+                mkEntry(
+                    subtopologyId,
+                    new StreamsAssignmentInterface.Subtopology(
+                        Set.of(topicName),
+                        Collections.emptySet(),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        Collections.emptyList()
+                    )
+                )
+            )
+        );
+    }
+
+    private void setupStreamsAssignmentInterfaceWithTwoSubtopologies(final 
String subtopologyId1,
+                                                                     final 
String topicName1,
+                                                                     final 
String subtopologyId2,
+                                                                     final 
String topicName2) {
+        when(streamsAssignmentInterface.subtopologyMap()).thenReturn(
+            mkMap(
+                mkEntry(
+                    subtopologyId1,
+                    new StreamsAssignmentInterface.Subtopology(
+                        Set.of(topicName1),
+                        Collections.emptySet(),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        Collections.emptyList()
+                    )
+                ),
+                mkEntry(
+                    subtopologyId2,
+                    new StreamsAssignmentInterface.Subtopology(
+                        Set.of(topicName2),
+                        Collections.emptySet(),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        Collections.emptyList()
+                    )
+                )
+            )
+        );
+    }
+
+    private void 
setupStreamsAssignmentInterfaceWithTwoConcatenedSubtopologies(final String 
subtopologyId1,
+                                                                               
final String topicName1,
+                                                                               
final String subtopologyId2,
+                                                                               
final String topicName2) {
+        when(streamsAssignmentInterface.subtopologyMap()).thenReturn(
+            mkMap(
+                mkEntry(
+                    subtopologyId1,
+                    new StreamsAssignmentInterface.Subtopology(
+                        Set.of(topicName1),
+                        Collections.emptySet(),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        Collections.emptyList()
+                    )
+                ),
+                mkEntry(
+                    subtopologyId2,
+                    new StreamsAssignmentInterface.Subtopology(
+                        Set.of(topicName2),
+                        Collections.emptySet(),
+                        mkMap(mkEntry(
+                            topicName2,
+                            new StreamsAssignmentInterface.TopicInfo(
+                                Optional.empty(),
+                                Optional.empty(),
+                                Collections.emptyMap()
+                            )
+                        )),
+                        Collections.emptyMap(),
+                        Collections.emptyList()
+                    )
+                )
+            )
+        );
+    }
+
+    private StreamsGroupHeartbeatResponse makeHeartbeatResponse(final String 
subtopologyId,
+                                                                final 
List<Integer> partitions) {
+        return makeHeartbeatResponse(Collections.singletonList(
+            new StreamsGroupHeartbeatResponseData.TaskIds()
+                .setSubtopologyId(subtopologyId)
+                .setPartitions(partitions)
+        ));
+    }
+
+    private StreamsGroupHeartbeatResponse makeHeartbeatResponse(final String 
subtopologyId0,
+                                                                final 
List<Integer> partitions0,
+                                                                final String 
subtopologyId1,
+                                                                final 
List<Integer> partitions1) {
+        return makeHeartbeatResponse(List.of(
+            new StreamsGroupHeartbeatResponseData.TaskIds()
+                .setSubtopologyId(subtopologyId0)
+                .setPartitions(partitions0),
+            new StreamsGroupHeartbeatResponseData.TaskIds()
+                .setSubtopologyId(subtopologyId1)
+                .setPartitions(partitions1)
+        ));
+    }
+
+    private StreamsGroupHeartbeatResponse makeHeartbeatResponse(final 
List<StreamsGroupHeartbeatResponseData.TaskIds> activeTasks) {
+        final StreamsGroupHeartbeatResponseData responseData = new 
StreamsGroupHeartbeatResponseData()
+            .setErrorCode(Errors.NONE.code())
+            .setMemberId(MEMBER_ID)
+            .setMemberEpoch(MEMBER_EPOCH)
+            .setActiveTasks(activeTasks)
+            .setStandbyTasks(Collections.emptyList())
+            .setWarmupTasks(Collections.emptyList());
+        return new StreamsGroupHeartbeatResponse(responseData);
+    }
+
+    private StreamsAssignmentInterface.Assignment makeTaskAssignment(final 
Set<StreamsAssignmentInterface.TaskId> activeTasks) {
+        return new StreamsAssignmentInterface.Assignment(
+            activeTasks,
+            Collections.emptySet(),
+            Collections.emptySet()
+        );
+    }
+
+    private void joining() {
+        membershipManager.onSubscriptionUpdated();
+        membershipManager.onConsumerPoll();
+        verifyInStateJoining(membershipManager);
+    }
+
+    private void reconcile(final StreamsGroupHeartbeatResponse response) {
+        membershipManager.onHeartbeatSuccess(response);
+        membershipManager.poll(time.milliseconds());
+        verifyInStateReconciling(membershipManager);
+
+    }
+
+    private void acknowledging() {
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        verifyInStateAcknowledging(membershipManager);
+    }
+
+    private CompletableFuture<Void> prepareLeaving() {
+        final CompletableFuture<Void> onGroupLeft = 
membershipManager.leaveGroup();
+        verifyInStatePrepareLeaving(membershipManager);
+        return onGroupLeft;
+    }
+
+    private CompletableFuture<Void> leaving() {
+        final CompletableFuture<Void> future = prepareLeaving();
+        final StreamsOnAssignmentCallbackNeededEvent 
onAssignmentCallbackNeededEvent =
+            (StreamsOnAssignmentCallbackNeededEvent) 
backgroundEventQueue.poll();
+        onAssignmentCallbackNeededEvent.future().complete(null);
+        verifyInStateLeaving(membershipManager);
+        return future;
+    }
+
+    private void stable() {
+        membershipManager.onHeartbeatRequestGenerated();
+    }
+
+    private void fenced() {
+        membershipManager.onFenced();
+    }
+}
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 27e709490be..90e17194b6f 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -1447,12 +1447,12 @@ public class StreamThread extends Thread implements 
ProcessingThread {
         return taskIdStream
             .collect(Collectors.toMap(
                 this::toTaskId,
-                task -> toTopicPartitions(task, 
streamsAssignmentInterface.subtopologyMap().get(task.subtopologyId))
+                task -> toTopicPartitions(task, 
streamsAssignmentInterface.subtopologyMap().get(task.subtopologyId()))
             ));
     }
 
     private TaskId toTaskId(final StreamsAssignmentInterface.TaskId task) {
-        return new TaskId(Integer.parseInt(task.subtopologyId), 
task.partitionId);
+        return new TaskId(Integer.parseInt(task.subtopologyId()), 
task.partitionId());
     }
 
     private Set<TopicPartition> toTopicPartitions(final 
StreamsAssignmentInterface.TaskId task,
@@ -1462,7 +1462,7 @@ public class StreamThread extends Thread implements 
ProcessingThread {
                 subTopology.sourceTopics.stream(),
                 subTopology.repartitionSourceTopics.keySet().stream()
             )
-            .map(t -> new TopicPartition(t, task.partitionId))
+            .map(t -> new TopicPartition(t, task.partitionId()))
             .collect(Collectors.toSet());
     }
 


Reply via email to