diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 09b459a27f797..a4d37677e3f84 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -155,6 +155,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; import org.apache.kafka.coordinator.group.streams.StreamsTopology; import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.TasksTupleWithEpochs; import org.apache.kafka.coordinator.group.streams.assignor.StickyTaskAssignor; import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException; @@ -1451,6 +1452,8 @@ private static boolean isSubset( * it owns any other tasks. * * @param ownedTasks The tasks provided by the streams group member in the request. + * If this is null, it indicates that we do not know which + * tasks are owned by the member, so we return false. * @param assignedTasks The tasks that the member should have. * * @return A boolean indicating whether the owned partitions are a subset or not. @@ -1472,6 +1475,32 @@ private static boolean areOwnedTasksContainedInAssignedTasks( return true; } + /** + * Checks whether all the tasks contained in the list are included in the provided assignment with epochs. + * + * @param ownedTasks The tasks provided by the streams group member in the request. + * If this is null, it indicates that we do not know which + * tasks are owned by the member, so we return false. + * @param assignedTasksWithEpochs The tasks that the member should have (with epochs). + * @return A boolean indicating whether the owned partitions are a subset or not. + */ + private static boolean areOwnedTasksContainedInAssignedTasksWithEpochs( + List ownedTasks, + Map> assignedTasksWithEpochs + ) { + if (ownedTasks == null) return false; + + for (StreamsGroupHeartbeatRequestData.TaskIds ownedTasksOfSubtopology : ownedTasks) { + Map partitionsWithEpochs = assignedTasksWithEpochs.get(ownedTasksOfSubtopology.subtopologyId()); + if (partitionsWithEpochs == null) return false; + for (Integer partitionId : ownedTasksOfSubtopology.partitions()) { + if (!partitionsWithEpochs.containsKey(partitionId)) return false; + } + } + + return true; + } + /** * Checks whether the consumer group can accept a new member or not based on the * max group size defined. @@ -1686,7 +1715,7 @@ private static void throwIfStreamsGroupMemberEpochIsInvalid( // If the member comes with the previous epoch and has a subset of the current assignment partitions, // we accept it because the response with the bumped epoch may have been lost. if (receivedMemberEpoch != member.previousMemberEpoch() - || !areOwnedTasksContainedInAssignedTasks(ownedActiveTasks, member.assignedTasks().activeTasks()) + || !areOwnedTasksContainedInAssignedTasksWithEpochs(ownedActiveTasks, member.assignedTasks().activeTasksWithEpochs()) || !areOwnedTasksContainedInAssignedTasks(ownedStandbyTasks, member.assignedTasks().standbyTasks()) || !areOwnedTasksContainedInAssignedTasks(ownedWarmupTasks, member.assignedTasks().warmupTasks())) { throw new FencedMemberEpochException("The streams group member has a smaller member " @@ -2060,7 +2089,7 @@ private CoordinatorResult stream // 1. The member is joining. // 2. The member's assignment has been updated. if (memberEpoch == 0 || hasAssignedTasksChanged(member, updatedMember)) { - response.setActiveTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().activeTasks())); + response.setActiveTasks(createStreamsGroupHeartbeatResponseTaskIdsFromEpochs(updatedMember.assignedTasks().activeTasksWithEpochs())); response.setStandbyTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().standbyTasks())); response.setWarmupTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().warmupTasks())); if (memberEpoch != 0 || !updatedMember.assignedTasks().isEmpty()) { @@ -2165,7 +2194,7 @@ private StreamsTopology maybeUpdateTopology(final String groupId, } } - private List createStreamsGroupHeartbeatResponseTaskIds(final Map> taskIds) { + private static List createStreamsGroupHeartbeatResponseTaskIds(final Map> taskIds) { return taskIds.entrySet().stream() .map(entry -> new StreamsGroupHeartbeatResponseData.TaskIds() .setSubtopologyId(entry.getKey()) @@ -2173,6 +2202,16 @@ private List createStreamsGroupHeartb .collect(Collectors.toList()); } + private static List createStreamsGroupHeartbeatResponseTaskIdsFromEpochs( + final Map> taskIdsWithEpochs + ) { + return taskIdsWithEpochs.entrySet().stream() + .map(entry -> new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(entry.getKey()) + .setPartitions(entry.getValue().keySet().stream().sorted().toList())) + .collect(Collectors.toList()); + } + private List maybeBuildEndpointToPartitions(StreamsGroup group, StreamsGroupMember updatedMember) { List endpointToPartitionsList = new ArrayList<>(); @@ -5726,8 +5765,8 @@ public void replay( StreamsGroupMember newMember = new StreamsGroupMember.Builder(oldMember) .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); streamsGroup.updateMember(newMember); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java index 2330497a7be39..67c76616e730d 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java @@ -304,6 +304,101 @@ private static boolean computeAssignmentDifferenceForOneSubtopology(final String return hasUnreleasedTasks; } + /** + * Takes the current currentAssignment and the targetAssignment, and generates three + * collections: + * + * - the resultAssignedTasks: the tasks that are assigned in both the current and target + * assignments. + * - the resultTasksPendingRevocation: the tasks that are assigned in the current + * assignment but not in the target assignment. + * - the resultTasksPendingAssignment: the tasks that are assigned in the target assignment but + * not in the current assignment, and can be assigned currently (i.e., they are not owned by + * another member, as defined by the `isUnreleasedTask` predicate). + * + * Epoch Handling: + * - For tasks in resultAssignedTasks and resultTasksPendingRevocation, the epoch from currentAssignment is preserved. + * - For tasks in resultTasksPendingAssignment, the targetAssignmentEpoch is used. + */ + private boolean computeAssignmentDifferenceWithEpoch(Map> currentAssignment, + Map> targetAssignment, + int targetAssignmentEpoch, + Map> resultAssignedTasks, + Map> resultTasksPendingRevocation, + Map> resultTasksPendingAssignment, + BiPredicate isUnreleasedTask) { + boolean hasUnreleasedTasks = false; + + Set allSubtopologyIds = new HashSet<>(targetAssignment.keySet()); + allSubtopologyIds.addAll(currentAssignment.keySet()); + + for (String subtopologyId : allSubtopologyIds) { + hasUnreleasedTasks |= computeAssignmentDifferenceForOneSubtopologyWithEpoch( + subtopologyId, + currentAssignment.getOrDefault(subtopologyId, Map.of()), + targetAssignment.getOrDefault(subtopologyId, Set.of()), + targetAssignmentEpoch, + resultAssignedTasks, + resultTasksPendingRevocation, + resultTasksPendingAssignment, + isUnreleasedTask + ); + } + return hasUnreleasedTasks; + } + + private static boolean computeAssignmentDifferenceForOneSubtopologyWithEpoch(final String subtopologyId, + final Map currentTasksForThisSubtopology, + final Set targetTasksForThisSubtopology, + final int targetAssignmentEpoch, + final Map> resultAssignedTasks, + final Map> resultTasksPendingRevocation, + final Map> resultTasksPendingAssignment, + final BiPredicate isUnreleasedTask) { + // Result Assigned Tasks = Current Tasks ∩ Target Tasks + // i.e. we remove all tasks from the current assignment that are not in the target + // assignment + Map resultAssignedTasksForThisSubtopology = new HashMap<>(); + for (Map.Entry entry : currentTasksForThisSubtopology.entrySet()) { + if (targetTasksForThisSubtopology.contains(entry.getKey())) { + resultAssignedTasksForThisSubtopology.put(entry.getKey(), entry.getValue()); + } + } + + // Result Tasks Pending Revocation = Current Tasks - Result Assigned Tasks + // i.e. we will ask the member to revoke all tasks in its current assignment that + // are not in the target assignment + Map resultTasksPendingRevocationForThisSubtopology = new HashMap<>(currentTasksForThisSubtopology); + resultTasksPendingRevocationForThisSubtopology.keySet().removeAll(resultAssignedTasksForThisSubtopology.keySet()); + + // Result Tasks Pending Assignment = Target Tasks - Result Assigned Tasks - Unreleased Tasks + // i.e. we will ask the member to assign all tasks in its target assignment, + // except those that are already assigned, and those that are unreleased + Map resultTasksPendingAssignmentForThisSubtopology = new HashMap<>(); + for (Integer taskId : targetTasksForThisSubtopology) { + if (!resultAssignedTasksForThisSubtopology.containsKey(taskId)) { + resultTasksPendingAssignmentForThisSubtopology.put(taskId, targetAssignmentEpoch); + } + } + boolean hasUnreleasedTasks = resultTasksPendingAssignmentForThisSubtopology.keySet().removeIf(taskId -> + isUnreleasedTask.test(subtopologyId, taskId) + ); + + if (!resultAssignedTasksForThisSubtopology.isEmpty()) { + resultAssignedTasks.put(subtopologyId, resultAssignedTasksForThisSubtopology); + } + + if (!resultTasksPendingRevocationForThisSubtopology.isEmpty()) { + resultTasksPendingRevocation.put(subtopologyId, resultTasksPendingRevocationForThisSubtopology); + } + + if (!resultTasksPendingAssignmentForThisSubtopology.isEmpty()) { + resultTasksPendingAssignment.put(subtopologyId, resultTasksPendingAssignmentForThisSubtopology); + } + + return hasUnreleasedTasks; + } + /** * Computes the next assignment. * @@ -313,10 +408,10 @@ private static boolean computeAssignmentDifferenceForOneSubtopology(final String * @return A new StreamsGroupMember. */ private StreamsGroupMember computeNextAssignment(int memberEpoch, - TasksTuple memberAssignedTasks) { - Map> newActiveAssignedTasks = new HashMap<>(); - Map> newActiveTasksPendingRevocation = new HashMap<>(); - Map> newActiveTasksPendingAssignment = new HashMap<>(); + TasksTupleWithEpochs memberAssignedTasks) { + Map> newActiveAssignedTasks = new HashMap<>(); + Map> newActiveTasksPendingRevocation = new HashMap<>(); + Map> newActiveTasksPendingAssignment = new HashMap<>(); Map> newStandbyAssignedTasks = new HashMap<>(); Map> newStandbyTasksPendingRevocation = new HashMap<>(); Map> newStandbyTasksPendingAssignment = new HashMap<>(); @@ -324,9 +419,10 @@ private StreamsGroupMember computeNextAssignment(int memberEpoch, Map> newWarmupTasksPendingRevocation = new HashMap<>(); Map> newWarmupTasksPendingAssignment = new HashMap<>(); - boolean hasUnreleasedActiveTasks = computeAssignmentDifference( - memberAssignedTasks.activeTasks(), + boolean hasUnreleasedActiveTasks = computeAssignmentDifferenceWithEpoch( + memberAssignedTasks.activeTasksWithEpochs(), targetAssignment.activeTasks(), + targetAssignmentEpoch, newActiveAssignedTasks, newActiveTasksPendingRevocation, newActiveTasksPendingAssignment, @@ -370,17 +466,17 @@ private StreamsGroupMember computeNextAssignment(int memberEpoch, return buildNewMember( memberEpoch, - new TasksTuple( + new TasksTupleWithEpochs( newActiveTasksPendingRevocation, newStandbyTasksPendingRevocation, newWarmupTasksPendingRevocation ), - new TasksTuple( + new TasksTupleWithEpochs( newActiveAssignedTasks, newStandbyAssignedTasks, newWarmupAssignedTasks ), - new TasksTuple( + new TasksTupleWithEpochs( newActiveTasksPendingAssignment, newStandbyTasksPendingAssignment, newWarmupTasksPendingAssignment @@ -390,9 +486,9 @@ private StreamsGroupMember computeNextAssignment(int memberEpoch, } private StreamsGroupMember buildNewMember(final int memberEpoch, - final TasksTuple newTasksPendingRevocation, - final TasksTuple newAssignedTasks, - final TasksTuple newTasksPendingAssignment, + final TasksTupleWithEpochs newTasksPendingRevocation, + final TasksTupleWithEpochs newAssignedTasks, + final TasksTupleWithEpochs newTasksPendingAssignment, final boolean hasUnreleasedTasks) { final boolean hasTasksToBeRevoked = @@ -424,7 +520,7 @@ private StreamsGroupMember buildNewMember(final int memberEpoch, .setState(newState) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedTasks(newAssignedTasks.merge(newTasksPendingAssignment)) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); } else if (hasUnreleasedTasks) { // If there are no tasks to be revoked nor to be assigned but some @@ -434,7 +530,7 @@ private StreamsGroupMember buildNewMember(final int memberEpoch, .setState(MemberState.UNRELEASED_TASKS) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedTasks(newAssignedTasks) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); } else { // Otherwise, the member transitions to the target epoch and to the @@ -443,7 +539,7 @@ private StreamsGroupMember buildNewMember(final int memberEpoch, .setState(MemberState.STABLE) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedTasks(newAssignedTasks) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java index dc0f4dd7d2fa9..516dc44b0d2da 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java @@ -266,10 +266,10 @@ public static CoordinatorRecord newStreamsGroupCurrentAssignmentRecord( .setMemberEpoch(member.memberEpoch()) .setPreviousMemberEpoch(member.previousMemberEpoch()) .setState(member.state().value()) - .setActiveTasks(toTaskIds(member.assignedTasks().activeTasks())) + .setActiveTasks(toTaskIdsWithEpochs(member.assignedTasks().activeTasksWithEpochs())) .setStandbyTasks(toTaskIds(member.assignedTasks().standbyTasks())) .setWarmupTasks(toTaskIds(member.assignedTasks().warmupTasks())) - .setActiveTasksPendingRevocation(toTaskIds(member.tasksPendingRevocation().activeTasks())) + .setActiveTasksPendingRevocation(toTaskIdsWithEpochs(member.tasksPendingRevocation().activeTasksWithEpochs())) .setStandbyTasksPendingRevocation(toTaskIds(member.tasksPendingRevocation().standbyTasks())) .setWarmupTasksPendingRevocation(toTaskIds(member.tasksPendingRevocation().warmupTasks())), (short) 0 @@ -311,6 +311,33 @@ private static List toTaskIds( return taskIds; } + private static List toTaskIdsWithEpochs( + Map> tasksWithEpochs + ) { + List taskIds = new ArrayList<>(tasksWithEpochs.size()); + tasksWithEpochs.forEach((subtopologyId, partitionEpochMap) -> { + // Sort by partition for consistent ordering + List> sortedEntries = partitionEpochMap.entrySet().stream() + .sorted(Comparator.comparingInt(Map.Entry::getKey)) + .toList(); + + List partitions = new ArrayList<>(sortedEntries.size()); + List epochs = new ArrayList<>(sortedEntries.size()); + + for (Map.Entry entry : sortedEntries) { + partitions.add(entry.getKey()); + epochs.add(entry.getValue()); + } + + taskIds.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(subtopologyId) + .setPartitions(partitions) + .setAssignmentEpochs(epochs)); + }); + taskIds.sort(Comparator.comparing(StreamsGroupCurrentMemberAssignmentValue.TaskIds::subtopologyId)); + return taskIds; + } + /** * Creates a StreamsTopology record. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index ed8e71e79b9dd..192ddada4deae 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -32,6 +32,7 @@ import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.Utils; +import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; @@ -726,8 +727,17 @@ public CommitPartitionValidator validateOffsetCommit( "by members using the streams group protocol"); } - validateMemberEpoch(memberEpoch, member.memberEpoch()); - return CommitPartitionValidator.NO_OP; + if (memberEpoch == member.memberEpoch()) { + return CommitPartitionValidator.NO_OP; + } + + if (memberEpoch > member.memberEpoch()) { + throw new StaleMemberEpochException(String.format("Received member epoch %d is newer than " + + "current member epoch %d.", memberEpoch, member.memberEpoch())); + } + + // Member epoch is older; validate against per-partition assignment epochs. + return createAssignmentEpochValidator(member, memberEpoch); } /** @@ -906,11 +916,11 @@ private void maybeRemoveTaskProcessId( } void removeTaskProcessIds( - TasksTuple tasks, + TasksTupleWithEpochs tasks, String processId ) { if (tasks != null) { - removeTaskProcessIds(tasks.activeTasks(), currentActiveTaskToProcessId, processId); + removeTaskProcessIds(tasks.activeTasksWithEpochs(), currentActiveTaskToProcessId, processId); removeTaskProcessIdsFromSet(tasks.standbyTasks(), currentStandbyTaskToProcessIds, processId); removeTaskProcessIdsFromSet(tasks.warmupTasks(), currentWarmupTaskToProcessIds, processId); } @@ -924,14 +934,14 @@ void removeTaskProcessIds( * @throws IllegalStateException if the process ID does not match the expected one. package-private for testing. */ private void removeTaskProcessIds( - Map> assignment, + Map> assignment, TimelineHashMap> currentTasksProcessId, String expectedProcessId ) { assignment.forEach((subtopologyId, assignedPartitions) -> { currentTasksProcessId.compute(subtopologyId, (__, partitionsOrNull) -> { if (partitionsOrNull != null) { - assignedPartitions.forEach(partitionId -> { + assignedPartitions.keySet().forEach(partitionId -> { String prevValue = partitionsOrNull.remove(partitionId); if (!Objects.equals(prevValue, expectedProcessId)) { throw new IllegalStateException( @@ -997,27 +1007,27 @@ private void removeTaskProcessIdsFromSet( * @throws IllegalStateException if the partition already has an epoch assigned. package-private for testing. */ void addTaskProcessId( - TasksTuple tasks, + TasksTupleWithEpochs tasks, String processId ) { if (tasks != null && processId != null) { - addTaskProcessId(tasks.activeTasks(), processId, currentActiveTaskToProcessId); + addTaskProcessIdFromActiveTasksWithEpochs(tasks.activeTasksWithEpochs(), processId, currentActiveTaskToProcessId); addTaskProcessIdToSet(tasks.standbyTasks(), processId, currentStandbyTaskToProcessIds); addTaskProcessIdToSet(tasks.warmupTasks(), processId, currentWarmupTaskToProcessIds); } } - private void addTaskProcessId( - Map> tasks, + private void addTaskProcessIdFromActiveTasksWithEpochs( + Map> tasksWithEpochs, String processId, TimelineHashMap> currentTaskProcessId ) { - tasks.forEach((subtopologyId, assignedTaskPartitions) -> { + tasksWithEpochs.forEach((subtopologyId, assignedTaskPartitionsWithEpochs) -> { currentTaskProcessId.compute(subtopologyId, (__, partitionsOrNull) -> { if (partitionsOrNull == null) { - partitionsOrNull = new TimelineHashMap<>(snapshotRegistry, assignedTaskPartitions.size()); + partitionsOrNull = new TimelineHashMap<>(snapshotRegistry, assignedTaskPartitionsWithEpochs.size()); } - for (Integer partitionId : assignedTaskPartitions) { + for (Integer partitionId : assignedTaskPartitionsWithEpochs.keySet()) { String prevValue = partitionsOrNull.put(partitionId, processId); if (prevValue != null) { throw new IllegalStateException( @@ -1120,4 +1130,54 @@ public void setLastAssignmentConfigs(Map lastAssignmentConfigs) this.lastAssignmentConfigs.putAll(lastAssignmentConfigs); } } + + /** + * Creates a validator that checks if the received member epoch is valid for each partition's assignment epoch. + * + * @param member The member whose assignments are being validated. + * @param receivedMemberEpoch The received member epoch. + * @return A validator for per-partition validation. + */ + private CommitPartitionValidator createAssignmentEpochValidator( + final StreamsGroupMember member, + int receivedMemberEpoch + ) { + // Retrieve topology once for all partitions - not per partition! + final StreamsTopology streamsTopology = topology.get().orElseThrow(() -> + new StaleMemberEpochException("Topology is not available for offset commit validation.")); + + final TasksTupleWithEpochs assignedTasks = member.assignedTasks(); + final TasksTupleWithEpochs tasksPendingRevocation = member.tasksPendingRevocation(); + + return (topicName, topicId, partitionId) -> { + final StreamsGroupTopologyValue.Subtopology subtopology = streamsTopology.sourceTopicMap().get(topicName); + if (subtopology == null) { + throw new StaleMemberEpochException("Topic " + topicName + " is not in the topology."); + } + + final String subtopologyId = subtopology.subtopologyId(); + + // Search for the partition in assigned tasks, then in tasks pending revocation + Integer assignmentEpoch = assignedTasks.activeTasksWithEpochs() + .getOrDefault(subtopologyId, Collections.emptyMap()) + .get(partitionId); + if (assignmentEpoch == null) { + assignmentEpoch = tasksPendingRevocation.activeTasksWithEpochs() + .getOrDefault(subtopologyId, Collections.emptyMap()) + .get(partitionId); + } + + if (assignmentEpoch == null) { + throw new StaleMemberEpochException(String.format( + "Task %s-%d is not assigned or pending revocation for member.", + subtopologyId, partitionId)); + } + + if (receivedMemberEpoch < assignmentEpoch) { + throw new StaleMemberEpochException(String.format( + "Received member epoch %d is older than assignment epoch %d for task %s-%d.", + receivedMemberEpoch, assignmentEpoch, subtopologyId, partitionId)); + } + }; + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java index c440deaa1a966..ba368cc15b781 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java @@ -49,8 +49,8 @@ * @param userEndpoint The user endpoint exposed for Interactive Queries by the Streams client that * contains the member. * @param clientTags Tags of the client of the member used for rack-aware assignment. - * @param assignedTasks Tasks assigned to the member. - * @param tasksPendingRevocation Tasks owned by the member pending revocation. + * @param assignedTasks Tasks assigned to the member, including assignment epochs for active tasks. + * @param tasksPendingRevocation Tasks owned by the member pending revocation, including assignment epochs for active tasks. */ @SuppressWarnings("checkstyle:JavaNCSS") public record StreamsGroupMember(String memberId, @@ -66,8 +66,8 @@ public record StreamsGroupMember(String memberId, String processId, Optional userEndpoint, Map clientTags, - TasksTuple assignedTasks, - TasksTuple tasksPendingRevocation) { + TasksTupleWithEpochs assignedTasks, + TasksTupleWithEpochs tasksPendingRevocation) { public StreamsGroupMember { Objects.requireNonNull(memberId, "memberId cannot be null"); @@ -94,8 +94,8 @@ public static class Builder { private String processId = null; private Optional userEndpoint = null; private Map clientTags = null; - private TasksTuple assignedTasks = null; - private TasksTuple tasksPendingRevocation = null; + private TasksTupleWithEpochs assignedTasks = null; + private TasksTupleWithEpochs tasksPendingRevocation = null; public Builder(String memberId) { this.memberId = Objects.requireNonNull(memberId, "memberId cannot be null"); @@ -223,12 +223,12 @@ public Builder maybeUpdateClientTags(Optional> clientTags) { return this; } - public Builder setAssignedTasks(TasksTuple assignedTasks) { + public Builder setAssignedTasks(TasksTupleWithEpochs assignedTasks) { this.assignedTasks = assignedTasks; return this; } - public Builder setTasksPendingRevocation(TasksTuple tasksPendingRevocation) { + public Builder setTasksPendingRevocation(TasksTupleWithEpochs tasksPendingRevocation) { this.tasksPendingRevocation = tasksPendingRevocation; return this; } @@ -254,42 +254,38 @@ public Builder updateWith(StreamsGroupCurrentMemberAssignmentValue record) { setPreviousMemberEpoch(record.previousMemberEpoch()); setState(MemberState.fromValue(record.state())); setAssignedTasks( - new TasksTuple( - assignmentFromTaskIds(record.activeTasks()), - assignmentFromTaskIds(record.standbyTasks()), - assignmentFromTaskIds(record.warmupTasks()) + TasksTupleWithEpochs.fromCurrentAssignmentRecord( + record.activeTasks(), + record.standbyTasks(), + record.warmupTasks(), + record.memberEpoch() ) ); setTasksPendingRevocation( - new TasksTuple( - assignmentFromTaskIds(record.activeTasksPendingRevocation()), - assignmentFromTaskIds(record.standbyTasksPendingRevocation()), - assignmentFromTaskIds(record.warmupTasksPendingRevocation()) + TasksTupleWithEpochs.fromCurrentAssignmentRecord( + record.activeTasksPendingRevocation(), + record.standbyTasksPendingRevocation(), + record.warmupTasksPendingRevocation(), + record.memberEpoch() ) ); return this; } - private static Map> assignmentFromTaskIds( - List topicPartitionsList - ) { - return topicPartitionsList.stream().collect(Collectors.toMap( - StreamsGroupCurrentMemberAssignmentValue.TaskIds::subtopologyId, - taskIds -> Set.copyOf(taskIds.partitions()))); - } - public static Builder withDefaults(String memberId) { return new Builder(memberId) .setRebalanceTimeoutMs(-1) .setTopologyEpoch(-1) .setInstanceId(null) .setRackId(null) + .setClientId("") + .setClientHost("") .setProcessId("") .setClientTags(Collections.emptyMap()) .setState(MemberState.STABLE) .setMemberEpoch(0) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .setUserEndpoint(null); } @@ -344,7 +340,7 @@ public StreamsGroupDescribeResponseData.Member asStreamsGroupDescribeMember(Task .setMemberId(memberId) .setAssignment( new StreamsGroupDescribeResponseData.Assignment() - .setActiveTasks(taskIdsFromMap(assignedTasks.activeTasks())) + .setActiveTasks(taskIdsFromMapWithEpochs(assignedTasks.activeTasksWithEpochs())) .setStandbyTasks(taskIdsFromMap(assignedTasks.standbyTasks())) .setWarmupTasks(taskIdsFromMap(assignedTasks.warmupTasks()))) .setTargetAssignment(describedTargetAssignment) @@ -378,6 +374,16 @@ private static List taskIdsFromMap(Map return taskIds; } + private static List taskIdsFromMapWithEpochs(Map> tasksWithEpochs) { + List taskIds = new ArrayList<>(); + tasksWithEpochs.keySet().stream().sorted().forEach(subtopologyId -> { + taskIds.add(new StreamsGroupDescribeResponseData.TaskIds() + .setSubtopologyId(subtopologyId) + .setPartitions(tasksWithEpochs.get(subtopologyId).keySet().stream().sorted().toList())); + }); + return taskIds; + } + /** * @return True if the two provided members have different assigned tasks. */ diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java index aab08da812ae0..5e2e6e04d70fd 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java @@ -20,12 +20,9 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMemberValue; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -67,41 +64,18 @@ public boolean isEmpty() { } /** - * Merges this task tuple with another task tuple. + * Checks if this task tuple contains any of the tasks in another task tuple with epochs. * - * @param other The other task tuple. - * @return A new task tuple, containing all active tasks, standby tasks and warm-up tasks from both tuples. - */ - public TasksTuple merge(TasksTuple other) { - Map> mergedActiveTasks = merge(activeTasks, other.activeTasks); - Map> mergedStandbyTasks = merge(standbyTasks, other.standbyTasks); - Map> mergedWarmupTasks = merge(warmupTasks, other.warmupTasks); - return new TasksTuple(mergedActiveTasks, mergedStandbyTasks, mergedWarmupTasks); - } - - private static Map> merge(final Map> tasks1, final Map> tasks2) { - HashMap> result = new HashMap<>(); - tasks1.forEach((subtopologyId, tasks) -> - result.put(subtopologyId, new HashSet<>(tasks))); - tasks2.forEach((subtopologyId, tasks) -> result - .computeIfAbsent(subtopologyId, __ -> new HashSet<>()) - .addAll(tasks)); - return result; - } - - /** - * Checks if this task tuple contains any of the tasks in another task tuple. - * - * @param other Another task tuple. + * @param other Another task tuple with epochs. * @return true if there is at least one active, standby or warm-up task that is present in both tuples. */ - public boolean containsAny(TasksTuple other) { + public boolean containsAny(TasksTupleWithEpochs other) { return activeTasks.entrySet().stream().anyMatch( - entry -> other.activeTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.activeTasks.get(entry.getKey())) + entry -> other.activeTasksWithEpochs().containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.activeTasksWithEpochs().get(entry.getKey()).keySet()) ) || standbyTasks.entrySet().stream().anyMatch( - entry -> other.standbyTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.standbyTasks.get(entry.getKey())) + entry -> other.standbyTasks().containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.standbyTasks().get(entry.getKey())) ) || warmupTasks.entrySet().stream().anyMatch( - entry -> other.warmupTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.warmupTasks.get(entry.getKey())) + entry -> other.warmupTasks().containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.warmupTasks().get(entry.getKey())) ); } @@ -172,22 +146,34 @@ public static TasksTuple fromHeartbeatRequest(final List> assignment ) { StringBuilder builder = new StringBuilder("["); - Iterator>> subtopologyIterator = assignment.entrySet().iterator(); - while (subtopologyIterator.hasNext()) { - Map.Entry> entry = subtopologyIterator.next(); - Iterator partitionsIterator = entry.getValue().iterator(); - while (partitionsIterator.hasNext()) { - builder.append(entry.getKey()); - builder.append("-"); - builder.append(partitionsIterator.next()); - if (partitionsIterator.hasNext() || subtopologyIterator.hasNext()) { + + // Sort subtopology IDs for deterministic output + String[] subtopologyIds = assignment.keySet().toArray(new String[0]); + java.util.Arrays.sort(subtopologyIds); + + boolean first = true; + for (String subtopologyId : subtopologyIds) { + Set partitions = assignment.get(subtopologyId); + + // Sort partition IDs for deterministic output + Integer[] partitionIds = partitions.toArray(new Integer[0]); + java.util.Arrays.sort(partitionIds); + + for (Integer partitionId : partitionIds) { + if (!first) { builder.append(", "); } + builder.append(subtopologyId); + builder.append("-"); + builder.append(partitionId); + first = false; } } builder.append("]"); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochs.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochs.java new file mode 100644 index 0000000000000..e5d726b0e646b --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochs.java @@ -0,0 +1,215 @@ +/* + * 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.coordinator.group.streams; + +import org.apache.kafka.coordinator.group.generated.StreamsGroupCurrentMemberAssignmentValue; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * An immutable tuple containing active, standby and warm-up tasks with assignment epochs. + *

+ * Active tasks include epoch information to support fencing of zombie commits. + * Standby and warmup tasks do not have epochs as they don't commit offsets. + * + * @param activeTasksWithEpochs Active tasks with their assignment epochs. + * The outer map key is the subtopology ID, the inner map key is the partition ID, + * and the inner map value is the assignment epoch. + * @param standbyTasks Standby tasks. + * The key of the map is the subtopology ID, and the value is the set of partition IDs. + * @param warmupTasks Warm-up tasks. + * The key of the map is the subtopology ID, and the value is the set of partition IDs. + */ +public record TasksTupleWithEpochs(Map> activeTasksWithEpochs, + Map> standbyTasks, + Map> warmupTasks) { + + public TasksTupleWithEpochs { + activeTasksWithEpochs = Collections.unmodifiableMap(Objects.requireNonNull(activeTasksWithEpochs)); + standbyTasks = Collections.unmodifiableMap(Objects.requireNonNull(standbyTasks)); + warmupTasks = Collections.unmodifiableMap(Objects.requireNonNull(warmupTasks)); + } + + /** + * An empty task tuple. + */ + public static final TasksTupleWithEpochs EMPTY = new TasksTupleWithEpochs( + Map.of(), + Map.of(), + Map.of() + ); + + /** + * @return true if all collections in the tuple are empty. + */ + public boolean isEmpty() { + return activeTasksWithEpochs.isEmpty() && standbyTasks.isEmpty() && warmupTasks.isEmpty(); + } + + /** + * Merges this task tuple with another task tuple. + * For overlapping active tasks, epochs from the other tuple take precedence. + * + * @param other The other task tuple. + * @return A new task tuple, containing all active tasks, standby tasks and warm-up tasks from both tuples. + */ + public TasksTupleWithEpochs merge(TasksTupleWithEpochs other) { + Map> mergedActive = new HashMap<>(); + + // Add all tasks from this tuple + this.activeTasksWithEpochs.forEach((subtopologyId, partitionsWithEpochs) -> { + mergedActive.put(subtopologyId, new HashMap<>(partitionsWithEpochs)); + }); + + // Add tasks from other tuple, overwriting epochs for overlapping partitions + other.activeTasksWithEpochs.forEach((subtopologyId, partitionsWithEpochs) -> { + mergedActive.computeIfAbsent(subtopologyId, k -> new HashMap<>()) + .putAll(partitionsWithEpochs); + }); + + Map> mergedStandby = mergeTasks(this.standbyTasks, other.standbyTasks); + Map> mergedWarmup = mergeTasks(this.warmupTasks, other.warmupTasks); + return new TasksTupleWithEpochs(mergedActive, mergedStandby, mergedWarmup); + } + + /** + * Creates a TasksTupleWithEpochs from a current assignment record. + * + * @param activeTasks The active tasks from the record. + * @param standbyTasks The standby tasks from the record. + * @param warmupTasks The warmup tasks from the record. + * @param memberEpoch The member epoch to use as default for tasks without explicit epochs. + * @return The TasksTupleWithEpochs + */ + public static TasksTupleWithEpochs fromCurrentAssignmentRecord( + List activeTasks, + List standbyTasks, + List warmupTasks, + int memberEpoch + ) { + return new TasksTupleWithEpochs( + parseActiveTasksWithEpochs(activeTasks, memberEpoch), + parseSimpleTasks(standbyTasks), + parseSimpleTasks(warmupTasks) + ); + } + + private static Map> mergeTasks(final Map> tasks1, final Map> tasks2) { + HashMap> result = new HashMap<>(); + tasks1.forEach((subtopologyId, tasks) -> + result.put(subtopologyId, new HashSet<>(tasks))); + tasks2.forEach((subtopologyId, tasks) -> result + .computeIfAbsent(subtopologyId, __ -> new HashSet<>()) + .addAll(tasks)); + return result; + } + + private static Map> parseActiveTasksWithEpochs( + List taskIdsList, + int memberEpoch + ) { + Map> result = new HashMap<>(); + + for (StreamsGroupCurrentMemberAssignmentValue.TaskIds taskIds : taskIdsList) { + String subtopologyId = taskIds.subtopologyId(); + List partitions = taskIds.partitions(); + List epochs = taskIds.assignmentEpochs(); + + Map partitionsWithEpochs = new HashMap<>(); + + if (epochs != null && !epochs.isEmpty()) { + if (epochs.size() != partitions.size()) { + throw new IllegalStateException( + "Assignment epochs must be provided for all partitions. " + + "Subtopology " + subtopologyId + " has " + partitions.size() + + " partitions but " + epochs.size() + " epochs" + ); + } + + for (int i = 0; i < partitions.size(); i++) { + partitionsWithEpochs.put(partitions.get(i), epochs.get(i)); + } + } else { + // Legacy record without epochs: use member epoch as default + for (Integer partition : partitions) { + partitionsWithEpochs.put(partition, memberEpoch); + } + } + + result.put(subtopologyId, partitionsWithEpochs); + } + + return result; + } + + private static Map> parseSimpleTasks( + List taskIdsList + ) { + Map> result = new HashMap<>(); + + for (StreamsGroupCurrentMemberAssignmentValue.TaskIds taskIds : taskIdsList) { + result.put(taskIds.subtopologyId(), new HashSet<>(taskIds.partitions())); + } + + return result; + } + + @Override + public String toString() { + return "(active=" + taskAssignmentToString(activeTasksWithEpochs) + + ", standby=" + TasksTuple.taskAssignmentToString(standbyTasks) + + ", warmup=" + TasksTuple.taskAssignmentToString(warmupTasks) + + ')'; + } + + private static String taskAssignmentToString(Map> assignment) { + StringBuilder builder = new StringBuilder("["); + + // Sort subtopology IDs + String[] subtopologyIds = assignment.keySet().toArray(new String[0]); + java.util.Arrays.sort(subtopologyIds); + + boolean first = true; + for (String subtopologyId : subtopologyIds) { + Map partitions = assignment.get(subtopologyId); + + // Sort partition IDs + Integer[] partitionIds = partitions.keySet().toArray(new Integer[0]); + java.util.Arrays.sort(partitionIds); + + for (Integer partitionId : partitionIds) { + if (!first) { + builder.append(", "); + } + builder.append(subtopologyId); + builder.append("-"); + builder.append(partitionId); + builder.append("@"); + builder.append(partitions.get(partitionId)); + first = false; + } + } + builder.append("]"); + return builder.toString(); + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java index 316ac4040d7c4..7b55f346f0983 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java @@ -40,7 +40,11 @@ public static StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToP final StreamsGroup streamsGroup, final CoordinatorMetadataImage metadataImage) { StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToPartitions = new StreamsGroupHeartbeatResponseData.EndpointToPartitions(); - Map> activeTasks = streamsGroupMember.assignedTasks().activeTasks(); + Map> activeTasks = streamsGroupMember.assignedTasks().activeTasksWithEpochs().entrySet().stream() + .collect(java.util.stream.Collectors.toUnmodifiableMap( + Map.Entry::getKey, + entry -> entry.getValue().keySet() + )); Map> standbyTasks = streamsGroupMember.assignedTasks().standbyTasks(); endpointToPartitions.setUserEndpoint(responseEndpoint); Map configuredSubtopologies = streamsGroup.configuredTopology().flatMap(ConfiguredTopology::subtopologies).get(); diff --git a/group-coordinator/src/main/resources/common/message/StreamsGroupCurrentMemberAssignmentValue.json b/group-coordinator/src/main/resources/common/message/StreamsGroupCurrentMemberAssignmentValue.json index 463c1e84e1702..6de638a3af765 100644 --- a/group-coordinator/src/main/resources/common/message/StreamsGroupCurrentMemberAssignmentValue.json +++ b/group-coordinator/src/main/resources/common/message/StreamsGroupCurrentMemberAssignmentValue.json @@ -45,7 +45,9 @@ { "name": "SubtopologyId", "type": "string", "versions": "0+", "about": "The subtopology ID." }, { "name": "Partitions", "type": "[]int32", "versions": "0+", - "about": "The partitions of the input topics processed by this member." } + "about": "The partitions of the input topics processed by this member." }, + { "name": "AssignmentEpochs", "versions": "0+", "nullableVersions": "0+", "taggedVersions": "0+", "tag": 0, "type": "[]int32", "default": null, + "about": "The epoch at which the partition was assigned to the member. Used to fence zombie commits requests. Of the same length as Partitions. Only defined for active tasks." } ]} ] } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 02fc987e10182..cf80e19c4ff1b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -139,6 +139,7 @@ import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil; import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole; import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.TasksTupleWithEpochs; import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException; import org.apache.kafka.image.MetadataDelta; @@ -219,6 +220,9 @@ import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.SHARE_GROUP_REBALANCES_SENSOR_NAME; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTupleWithEpochs; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksWithEpochs; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -4267,15 +4271,13 @@ public void testOnLoadedWithStreamsGroup() { .setInstanceId(null) .setRebalanceTimeoutMs(100) .setClientTags(new HashMap<>()) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) .setTopologyEpoch(1) .setUserEndpoint(new Endpoint().setHost("localhost").setPort(1500)) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 9, TaskAssignmentTestUtil.mkTasks(fooTopicName, 0, 1, 2))) - .setTasksPendingRevocation(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 9, TaskAssignmentTestUtil.mkTasks(fooTopicName, 3, 4, 5))) .build()) .withMember(new StreamsGroupMember.Builder("foo-2") @@ -4285,8 +4287,8 @@ public void testOnLoadedWithStreamsGroup() { .setProcessId("process-id") .setRackId(null) .setInstanceId(null) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .setRebalanceTimeoutMs(100) .setClientTags(new HashMap<>()) .setTopologyEpoch(1) @@ -4491,8 +4493,8 @@ private StreamsGroupMember.Builder streamsGroupMemberBuilderWithDefaults(String .setRackId(null) .setInstanceId(null) .setRebalanceTimeoutMs(1500) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .setTopologyEpoch(0) .setClientTags(Map.of()) .setClientId(DEFAULT_CLIENT_ID) @@ -16035,7 +16037,7 @@ public void testStreamsGroupMemberEpochValidation() { StreamsGroupMember member = streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(100) .setPreviousMemberEpoch(99) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 100, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) .build(); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, member)); @@ -16125,7 +16127,7 @@ public void testStreamsOwnedTasksValidation() { .withMember(streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) @@ -16280,9 +16282,10 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) .setRebalanceTimeoutMs(1500) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 1, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .build(); List expectedRecords = List.of( @@ -16363,7 +16366,7 @@ public void testJoinEmptyStreamsGroupAndDescribe() { .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) .setRebalanceTimeoutMs(1500) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 1, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .build(); @@ -16765,14 +16768,14 @@ public void testStreamsGroupMemberRequestingShutdownApplication() { .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5))) .build()) .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, @@ -16961,7 +16964,7 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .build()) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, @@ -17009,9 +17012,22 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), - TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(subtopology1, + Map.of( + 0, 10, + 1, 10, + 2, 10, + 3, 10, + 4, 10, + 5, 10 + )), + mkTasksWithEpochs(subtopology2, + Map.of( + 0, 11, + 1, 11, + 2, 11 + )))) .setProcessId("process-id2") .build(); @@ -17069,7 +17085,7 @@ barTopicName, computeTopicHash(barTopicName, oldMetadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .build()) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, @@ -17116,9 +17132,22 @@ barTopicName, computeTopicHash(barTopicName, oldMetadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), - TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(subtopology1, + Map.of( + 0, 10, + 1, 10, + 2, 10, + 3, 10, + 4, 10, + 5, 10 + )), + mkTasksWithEpochs(subtopology2, + Map.of( + 0, 11, + 1, 11, + 2, 11 + )))) .setProcessId("process-id2") .build(); @@ -17173,14 +17202,14 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) .build()) .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .build()) @@ -17265,14 +17294,14 @@ public void testStreamsLeavingMemberRemovesMemberAndBumpsGroupEpoch() { .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) .build()) .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .build()) @@ -17415,14 +17444,14 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) .build()) .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .build()) @@ -17536,10 +17565,10 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), TaskAssignmentTestUtil.mkTasks(subtopology2, 0))) - .setTasksPendingRevocation(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) .build())), @@ -17582,10 +17611,10 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 3), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) - .setTasksPendingRevocation(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5))) .build())), result.records() @@ -17655,7 +17684,8 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + // Assignment epoch not bumped + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), TaskAssignmentTestUtil.mkTasks(subtopology2, 0))) .build())), @@ -17712,7 +17742,7 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 11, TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) .build())), result.records() @@ -17785,9 +17815,11 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId2) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), - TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + // Assignment epoch of previous tasks is preserved, new tasks gets new assignment epoch + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(subtopology1, Map.of(2, 11, 3, 10)), + mkTasksWithEpochs(subtopology2, Map.of(2, 10)) + )) .build())), result.records() ); @@ -17830,9 +17862,10 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), - TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) + // All tasks were assigned in epoch 11 + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(subtopology1, Map.of(4, 11, 5, 11)), + mkTasksWithEpochs(subtopology2, Map.of(1, 11)))) .build())), result.records() ); @@ -17900,7 +17933,8 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setMemberEpoch(11) .setPreviousMemberEpoch(10) .setAssignedTasks( - TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 11, + TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) .build())); assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); @@ -17910,7 +17944,8 @@ barTopicName, computeTopicHash(barTopicName, metadataImage) .setMemberEpoch(11) .setPreviousMemberEpoch(10) .setAssignedTasks( - TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 11, + TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) .build())); assertEquals(StreamsGroup.StreamsGroupState.STABLE, context.streamsGroupState(groupId)); @@ -17981,7 +18016,7 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() { .withMember(streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) @@ -18035,8 +18070,17 @@ fooTopicName, computeTopicHash( StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(subtopology1, + Map.of( + 0, 10, // 0, 1, 2 were already assigned in epoch 10 + 1, 10, + 2, 10, + 3, 11, // 3, 4, 5 were added in epoch 11 + 4, 11, + 5, 11 + )) + )) .build(); List expectedRecords = List.of( @@ -18082,7 +18126,7 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { .withMember(streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 11, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) @@ -18156,9 +18200,9 @@ fooTopicName, computeTopicHash( StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 11, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); List expectedRecords = List.of( @@ -18845,7 +18889,7 @@ public void testStreamsGroupEpochIncreaseWithAssignmentConfigChanges() { .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, 10, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .build()) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, @@ -18996,8 +19040,8 @@ public void testStreamsGroupHeartbeatWithEmptyClassicGroup() { .setRebalanceTimeoutMs(5000) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .setRebalanceTimeoutMs(12000) .setTopologyEpoch(0) .build(); @@ -19820,12 +19864,14 @@ public void testReplayStreamsGroupCurrentMemberAssignment() { .setMemberEpoch(10) .setPreviousMemberEpoch(9) .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) - .setAssignedTasks(new TasksTuple( - TaskAssignmentTestUtil.mkTasksPerSubtopology(TaskAssignmentTestUtil.mkTasks("subtopology-1", 0, 1, 2)), + .setAssignedTasks(new TasksTupleWithEpochs( + TaskAssignmentTestUtil.mkTasksPerSubtopologyWithCommonEpoch(10, + TaskAssignmentTestUtil.mkTasks("subtopology-1", 0, 1, 2) + ), TaskAssignmentTestUtil.mkTasksPerSubtopology(TaskAssignmentTestUtil.mkTasks("subtopology-1", 3, 4, 5)), TaskAssignmentTestUtil.mkTasksPerSubtopology(TaskAssignmentTestUtil.mkTasks("subtopology-1", 6, 7, 8)) )) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); // The group and the member are created if they do not exist. @@ -19852,9 +19898,9 @@ public void testReplayStreamsGroupCurrentMemberAssignmentTombstoneNotExisting() @Test public void testReplayStreamsGroupCurrentMemberAssignmentTombstoneExisting() { - final TasksTuple tasks = - new TasksTuple( - TaskAssignmentTestUtil.mkTasksPerSubtopology( + final TasksTupleWithEpochs tasks = + new TasksTupleWithEpochs( + TaskAssignmentTestUtil.mkTasksPerSubtopologyWithCommonEpoch(1, TaskAssignmentTestUtil.mkTasks("subtopology-1", 0, 1, 2)), TaskAssignmentTestUtil.mkTasksPerSubtopology( TaskAssignmentTestUtil.mkTasks("subtopology-1", 3, 4, 5)), diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index b2a3463224324..f1379e2404a5a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -113,7 +113,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupBuilder; import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; -import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.TasksTupleWithEpochs; import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager; import org.apache.kafka.server.authorizer.Authorizer; @@ -1790,8 +1790,8 @@ public void updateGroupConfig(String groupId, Properties newGroupConfig) { public static StreamsGroupMember.Builder streamsGroupMemberBuilderWithDefaults(String memberId) { return new StreamsGroupMember.Builder(memberId) .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) - .setAssignedTasks(TasksTuple.EMPTY) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) .setRackId(null) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index 672cc37c41411..275a3eeccc470 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -66,6 +66,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroup; import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; import org.apache.kafka.coordinator.group.streams.StreamsTopology; +import org.apache.kafka.coordinator.group.streams.TasksTupleWithEpochs; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; @@ -3659,4 +3660,77 @@ private ClassicGroupMember mkGenericMember( ) ); } + + @Test + public void testStreamsGroupOffsetCommitWithAssignmentEpochValid() { + OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); + StreamsGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedStreamsGroup("foo", true); + + // Setup: topology with topic "bar" in subtopology "0" + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("bar"))))); + + // Member at epoch 10, with partitions assigned at epoch 4 and 5 respsectively. + group.updateMember(StreamsGroupMember.Builder.withDefaults("member") + .setMemberEpoch(10) + .setAssignedTasks(new TasksTupleWithEpochs( + Map.of("0", Map.of(0, 4, 1, 5)), + Map.of(), Map.of())) + .build()); + + // Commit with member epoch 5 should succeed (5 >= assignment epoch 5) + CoordinatorResult result = context.commitOffset( + new OffsetCommitRequestData() + .setGroupId("foo") + .setMemberId("member") + .setGenerationIdOrMemberEpoch(5) + .setTopics(List.of(new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setName("bar") + .setPartitions(List.of( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(100L), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(200L)))))); + + assertEquals(Errors.NONE.code(), result.response().topics().get(0).partitions().get(0).errorCode()); + assertEquals(Errors.NONE.code(), result.response().topics().get(0).partitions().get(1).errorCode()); + assertEquals(2, result.records().size()); + } + + @Test + public void testStreamsGroupOffsetCommitWithAssignmentEpochStale() { + OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); + StreamsGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedStreamsGroup("foo", true); + + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("bar"))))); + + // Member at epoch 10, with partitions assigned at different epochs + group.updateMember(StreamsGroupMember.Builder.withDefaults("member") + .setMemberEpoch(10) + .setAssignedTasks(new TasksTupleWithEpochs( + Map.of("0", Map.of(0, 5, 1, 8)), + Map.of(), Map.of())) + .build()); + + // Commit with member epoch 7 should fail (3 < assignment epochs 8) + assertThrows(StaleMemberEpochException.class, () -> context.commitOffset( + new OffsetCommitRequestData() + .setGroupId("foo") + .setMemberId("member") + .setGenerationIdOrMemberEpoch(3) + .setTopics(List.of(new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setName("bar") + .setPartitions(List.of( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(100L), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(200L))))))); + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java index 075795f6e11e5..b383a35f786f3 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java @@ -24,10 +24,14 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import java.util.Map; import java.util.Set; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTupleWithEpochs; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksWithEpochs; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -50,11 +54,12 @@ public void testStableToStable(TaskRole taskRole) { .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) .setAssignedTasks( - mkTasksTuple( + mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -73,11 +78,12 @@ public void testStableToStable(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple( + .setAssignedTasks(mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -95,11 +101,12 @@ public void testStableToStableAtTargetEpoch(TaskRole taskRole) { .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) .setAssignedTasks( - mkTasksTuple( + mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -118,11 +125,12 @@ public void testStableToStableAtTargetEpoch(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple( + .setAssignedTasks(mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -138,10 +146,10 @@ public void testStableToStableWithNewTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, - mkTasks(SUBTOPOLOGY_ID1, 1, 2), - mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(mkTasksTupleWithEpochs(taskRole, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 9, 2, 8)), + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, 9, 4, 8)))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -160,10 +168,10 @@ public void testStableToStableWithNewTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, - mkTasks(SUBTOPOLOGY_ID1, 1, 2, 4), - mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setAssignedTasks(mkTasksTupleWithEpochs(taskRole, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 9, 2, 8, 4, memberEpoch + 1)), + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, 9, 4, 8, 7, memberEpoch + 1)))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -179,10 +187,10 @@ public void testStableToUnrevokedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -201,10 +209,10 @@ public void testStableToUnrevokedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2), mkTasks(SUBTOPOLOGY_ID2, 4))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1), mkTasks(SUBTOPOLOGY_ID2, 3))) .build(), @@ -224,11 +232,12 @@ public void testStableToUnrevokedWithEmptyAssignment(TaskRole taskRole) { .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) .setAssignedTasks( - mkTasksTuple( + mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -245,10 +254,11 @@ public void testStableToUnrevokedWithEmptyAssignment(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(TasksTuple.EMPTY) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) .setTasksPendingRevocation( - mkTasksTuple( + mkTasksTupleWithCommonEpoch( taskRole, + memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) .build(), @@ -266,10 +276,10 @@ public void testStableToUnreleasedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -288,10 +298,10 @@ public void testStableToUnreleasedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -307,10 +317,10 @@ public void testStableToUnreleasedTasksWithOwnedTasksNotHavingRevokedTasks(TaskR .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3, 4))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -332,10 +342,10 @@ public void testStableToUnreleasedTasksWithOwnedTasksNotHavingRevokedTasks(TaskR .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1, 2), mkTasks(SUBTOPOLOGY_ID2, 3))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -351,10 +361,10 @@ public void testUnrevokedTasksToStable(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1), mkTasks(SUBTOPOLOGY_ID2, 4))) .build(); @@ -378,10 +388,10 @@ public void testUnrevokedTasksToStable(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -397,10 +407,10 @@ public void testRemainsInUnrevokedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1), mkTasks(SUBTOPOLOGY_ID2, 4))) .build(); @@ -452,10 +462,10 @@ public void testUnrevokedTasksToUnrevokedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1), mkTasks(SUBTOPOLOGY_ID2, 4))) .build(); @@ -476,10 +486,10 @@ public void testUnrevokedTasksToUnrevokedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 3), mkTasks(SUBTOPOLOGY_ID2, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2), mkTasks(SUBTOPOLOGY_ID2, 5))) .build(), @@ -497,10 +507,10 @@ public void testUnrevokedTasksToUnreleasedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch - 1) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 1), mkTasks(SUBTOPOLOGY_ID2, 4))) .build(); @@ -525,10 +535,10 @@ public void testUnrevokedTasksToUnreleasedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -544,10 +554,10 @@ public void testUnreleasedTasksToStable(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -567,10 +577,10 @@ public void testUnreleasedTasksToStable(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -586,10 +596,10 @@ public void testUnreleasedTasksToStableWithNewTasks(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -608,10 +618,10 @@ public void testUnreleasedTasksToStableWithNewTasks(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); @@ -627,10 +637,10 @@ public void testUnreleasedTasksToUnreleasedTasks(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -658,10 +668,10 @@ public void testUnreleasedTasksToUnreleasedTasksOtherUnreleasedTaskRole(TaskRole .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -692,9 +702,10 @@ public void testUnreleasedTasksToUnreleasedTasksAnyActiveOwner() { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember expectedMember = new StreamsGroupMember.Builder(MEMBER_NAME) @@ -702,10 +713,10 @@ public void testUnreleasedTasksToUnreleasedTasksAnyActiveOwner() { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(TaskRole.ACTIVE, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) @@ -734,10 +745,10 @@ public void testUnreleasedTasksToUnrevokedTasks(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) - .setTasksPendingRevocation(mkTasksTuple(TaskRole.ACTIVE, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 4), mkTasks(SUBTOPOLOGY_ID2, 7))) .build(); @@ -758,10 +769,10 @@ public void testUnreleasedTasksToUnrevokedTasks(TaskRole taskRole) { .setProcessId("process1") .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 3), mkTasks(SUBTOPOLOGY_ID2, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2), mkTasks(SUBTOPOLOGY_ID2, 5))) .build(), @@ -779,10 +790,10 @@ public void testUnknownState(TaskRole taskRole) { .setMemberEpoch(memberEpoch) .setPreviousMemberEpoch(memberEpoch) .setProcessId(PROCESS_ID) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 3), mkTasks(SUBTOPOLOGY_ID2, 6))) - .setTasksPendingRevocation(mkTasksTuple(taskRole, + .setTasksPendingRevocation(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 2), mkTasks(SUBTOPOLOGY_ID2, 5))) .build(); @@ -814,10 +825,151 @@ public void testUnknownState(TaskRole taskRole) { .setProcessId(PROCESS_ID) .setMemberEpoch(memberEpoch + 1) .setPreviousMemberEpoch(memberEpoch) - .setAssignedTasks(mkTasksTuple(taskRole, + .setAssignedTasks(mkTasksTupleWithCommonEpoch(taskRole, memberEpoch, mkTasks(SUBTOPOLOGY_ID1, 3), mkTasks(SUBTOPOLOGY_ID2, 6))) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build(), + updatedMember + ); + } + + @Test + public void testAssignmentEpochsShouldBePreservedFromPreviousAssignment() { + final int memberEpoch = 10; + + // Create a member with tasks that have specific epochs in assigned tasks + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 5, 2, 6)), + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, 7, 4, 8)))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build(); + + // Same tasks in target assignment should retain their epochs from assigned tasks + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .build(); + + // Verify that epochs are preserved from assigned tasks + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 5, 2, 6)), + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, 7, 4, 8)))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build(), + updatedMember + ); + } + + @Test + public void testNewlyAssignedTasksGetTargetAssignmentEpoch() { + final int memberEpoch = 10; + final int targetAssignmentEpoch = 11; + + // Create a member with empty assignments + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build(); + + // New tasks are assigned + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(targetAssignmentEpoch, mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .build(); + + // Verify that all tasks use the target assignment epoch + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(targetAssignmentEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTupleWithCommonEpoch(TaskRole.ACTIVE, targetAssignmentEpoch, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build(), + updatedMember + ); + } + + /** + * Tests mixed epoch assignment scenarios. + * - Some epochs from previously assigned tasks (Tasks 1, 2). + * This happens regardless of whether the assigned task is reconciled (owned) by the client (Task 1) or not (Task 2) + * - Some newly assigned task (Task 5) which should get the target assignment epoch. + * - Some tasks are revoked by the member (Task 3, 4). One is immediately reassigned, which also gets + * the target assignment epoch (Task 3). + */ + @Test + public void testMixedPreservedAndNewAssignmentEpochs() { + final int memberEpoch = 10; + final int targetAssignmentEpoch = 11; + + // Create a member with: + // - Tasks 1, 2 in assigned with epochs 5, 6 + // - Tasks 3, 4 in pending revocation with epochs 7, 8 + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 5, 2, 6)))) + .setTasksPendingRevocation(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, 7, 4, 8)))) + .build(); + + // The member revokes tasks 3, 4 (not in owned), transitions to next epoch + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(targetAssignmentEpoch, mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 5))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) + .withOwnedAssignment(mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 1))) // Only owns task 1 (task 2 is not yet reconciled, tasks 3,4 already revoked) + .build(); + + // Verify mixed epoch assignment: + // - Task 1 from SUBTOPOLOGY_ID1 should have epoch 5 (previous assignment epoch) + // - Task 3 from SUBTOPOLOGY_ID2 should have epoch 11 (target assignment epoch) + // - Task 5 from SUBTOPOLOGY_ID2 should have epoch 11 (target assignment epoch) + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(targetAssignmentEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTupleWithEpochs(TaskRole.ACTIVE, + mkTasksWithEpochs(SUBTOPOLOGY_ID1, Map.of(1, 5, 2, 6)), + mkTasksWithEpochs(SUBTOPOLOGY_ID2, Map.of(3, targetAssignmentEpoch, 5, targetAssignmentEpoch)))) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(), updatedMember ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java index c57f8cbd60d43..4a5bd69c25b8f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java @@ -44,7 +44,10 @@ import java.util.Set; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksWithEpochs; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksWithEpochsPerSubtopology; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; @@ -411,27 +414,19 @@ public void testNewStreamsGroupCurrentAssignmentRecord() { .setProcessId(PROCESS_ID) .setUserEndpoint(new Endpoint().setHost(USER_ENDPOINT).setPort(USER_ENDPOINT_PORT)) .setClientTags(Map.of(TAG_1, VALUE_1, TAG_2, VALUE_2)) - .setAssignedTasks(new TasksTuple( - Map.of( - SUBTOPOLOGY_1, Set.of(1, 2, 3) + .setAssignedTasks(new TasksTupleWithEpochs( + mkTasksWithEpochsPerSubtopology( + mkTasksWithEpochs(SUBTOPOLOGY_1, Map.of(1, 10, 2, 11, 3, 12)) ), - Map.of( - SUBTOPOLOGY_2, Set.of(4, 5, 6) - ), - Map.of( - SUBTOPOLOGY_3, Set.of(7, 8, 9) - ) + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY_2, 4, 5, 6)), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY_3, 7, 8, 9)) )) - .setTasksPendingRevocation(new TasksTuple( - Map.of( - SUBTOPOLOGY_1, Set.of(1, 2, 3) - ), - Map.of( - SUBTOPOLOGY_2, Set.of(4, 5, 6) + .setTasksPendingRevocation(new TasksTupleWithEpochs( + mkTasksWithEpochsPerSubtopology( + mkTasksWithEpochs(SUBTOPOLOGY_1, Map.of(1, 5, 2, 6, 3, 7)) ), - Map.of( - SUBTOPOLOGY_3, Set.of(7, 8, 9) - ) + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY_2, 4, 5, 6)), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY_3, 7, 8, 9)) )) .build(); @@ -448,6 +443,7 @@ public void testNewStreamsGroupCurrentAssignmentRecord() { new StreamsGroupCurrentMemberAssignmentValue.TaskIds() .setSubtopologyId(SUBTOPOLOGY_1) .setPartitions(List.of(1, 2, 3)) + .setAssignmentEpochs(List.of(10, 11, 12)) )) .setStandbyTasks(List.of( new StreamsGroupCurrentMemberAssignmentValue.TaskIds() @@ -463,6 +459,7 @@ public void testNewStreamsGroupCurrentAssignmentRecord() { new StreamsGroupCurrentMemberAssignmentValue.TaskIds() .setSubtopologyId(SUBTOPOLOGY_1) .setPartitions(List.of(1, 2, 3)) + .setAssignmentEpochs(List.of(5, 6, 7)) )) .setStandbyTasksPendingRevocation(List.of( new StreamsGroupCurrentMemberAssignmentValue.TaskIds() @@ -496,8 +493,8 @@ public void testNewStreamsGroupCurrentAssignmentRecordWithEmptyAssignment() { .setProcessId(PROCESS_ID) .setUserEndpoint(new Endpoint().setHost(USER_ENDPOINT).setPort(USER_ENDPOINT_PORT)) .setClientTags(Map.of(TAG_1, VALUE_1, TAG_2, VALUE_2)) - .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) - .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); CoordinatorRecord expectedRecord = CoordinatorRecord.record( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java index 023e491d6469e..5fd33272857db 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java @@ -37,6 +37,7 @@ import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopologyWithCommonEpoch; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -70,15 +71,15 @@ public class StreamsGroupMemberTest { private static final List TASKS4 = List.of(3, 2, 1); private static final List TASKS5 = List.of(6, 5, 4); private static final List TASKS6 = List.of(9, 7); - private static final TasksTuple ASSIGNED_TASKS = - new TasksTuple( - mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS1.toArray(Integer[]::new))), + private static final TasksTupleWithEpochs ASSIGNED_TASKS = + new TasksTupleWithEpochs( + mkMap(mkEntry(SUBTOPOLOGY1, mkMap(mkEntry(1, 5), mkEntry(2, 6), mkEntry(3, 7)))), mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS2.toArray(Integer[]::new))), mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS3.toArray(Integer[]::new))) ); - private static final TasksTuple TASKS_PENDING_REVOCATION = - new TasksTuple( - mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS4.toArray(Integer[]::new))), + private static final TasksTupleWithEpochs TASKS_PENDING_REVOCATION = + new TasksTupleWithEpochs( + mkMap(mkEntry(SUBTOPOLOGY2, mkMap(mkEntry(3, 4), mkEntry(2, 3), mkEntry(1, 2)))), mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS5.toArray(Integer[]::new))), mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS6.toArray(Integer[]::new))) ); @@ -102,7 +103,7 @@ public void testBuilderWithMemberIsNull() { } @Test - public void testBuilderWithDefaults() { + public void testBuilderWithoutDefaults() { StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_ID).build(); assertEquals(MEMBER_ID, member.memberId()); @@ -122,6 +123,27 @@ public void testBuilderWithDefaults() { assertNull(member.tasksPendingRevocation()); } + @Test + public void testBuilderWithDefaults() { + StreamsGroupMember member = StreamsGroupMember.Builder.withDefaults(MEMBER_ID) + .build(); + + assertEquals(MEMBER_ID, member.memberId()); + assertEquals(0, member.memberEpoch()); + assertEquals(MemberState.STABLE, member.state()); + assertEquals(Optional.empty(), member.instanceId()); + assertEquals(Optional.empty(), member.rackId()); + assertEquals("", member.clientId()); + assertEquals("", member.clientHost()); + assertEquals(-1, member.rebalanceTimeoutMs()); + assertEquals(-1, member.topologyEpoch()); + assertEquals("", member.processId()); + assertEquals(Optional.empty(), member.userEndpoint()); + assertEquals(Map.of(), member.clientTags()); + assertEquals(TasksTupleWithEpochs.EMPTY, member.assignedTasks()); + assertEquals(TasksTupleWithEpochs.EMPTY, member.tasksPendingRevocation()); + } + @Test public void testBuilderNewMember() { StreamsGroupMember member = createStreamsGroupMember(); @@ -183,14 +205,23 @@ public void testBuilderUpdateWithStreamsGroupMemberMetadataValue() { @Test public void testBuilderUpdateWithConsumerGroupCurrentMemberAssignmentValue() { + List assignmentEpochsForTasks1 = List.of(5, 6, 7); + List assignmentEpochsForTasks4 = List.of(4, 3, 2); + StreamsGroupCurrentMemberAssignmentValue record = new StreamsGroupCurrentMemberAssignmentValue() .setMemberEpoch(MEMBER_EPOCH) .setPreviousMemberEpoch(PREVIOUS_MEMBER_EPOCH) .setState(STATE.value()) - .setActiveTasks(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY1).setPartitions(TASKS1))) + .setActiveTasks(List.of(new TaskIds() + .setSubtopologyId(SUBTOPOLOGY1) + .setPartitions(TASKS1) + .setAssignmentEpochs(assignmentEpochsForTasks1))) .setStandbyTasks(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY2).setPartitions(TASKS2))) .setWarmupTasks(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY1).setPartitions(TASKS3))) - .setActiveTasksPendingRevocation(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY2).setPartitions(TASKS4))) + .setActiveTasksPendingRevocation(List.of(new TaskIds() + .setSubtopologyId(SUBTOPOLOGY2) + .setPartitions(TASKS4) + .setAssignmentEpochs(assignmentEpochsForTasks4))) .setStandbyTasksPendingRevocation(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY1).setPartitions(TASKS5))) .setWarmupTasksPendingRevocation(List.of(new TaskIds().setSubtopologyId(SUBTOPOLOGY2).setPartitions(TASKS6))); @@ -262,6 +293,7 @@ public void testBuilderMaybeUpdateMember() { assertEquals(member.memberId(), updatedMember.memberId()); assertEquals(member.memberEpoch(), updatedMember.memberEpoch()); assertEquals(member.previousMemberEpoch(), updatedMember.previousMemberEpoch()); + assertEquals(member.assignedTasks(), updatedMember.assignedTasks()); assertEquals(member.state(), updatedMember.state()); assertEquals(member.clientId(), updatedMember.clientId()); assertEquals(member.clientHost(), updatedMember.clientHost()); @@ -280,8 +312,8 @@ public void testBuilderUpdateMemberEpoch() { assertEquals(member.memberId(), updatedMember.memberId()); assertEquals(newMemberEpoch, updatedMember.memberEpoch()); - // The previous member epoch becomes the old current member epoch. assertEquals(member.memberEpoch(), updatedMember.previousMemberEpoch()); + assertEquals(member.assignedTasks(), updatedMember.assignedTasks()); assertEquals(member.state(), updatedMember.state()); assertEquals(member.instanceId(), updatedMember.instanceId()); assertEquals(member.rackId(), updatedMember.rackId()); @@ -375,38 +407,46 @@ public void testAsStreamsGroupDescribeWithTargetAssignmentNull() { @Test public void testHasAssignedTasksChanged() { + TasksTupleWithEpochs assignedTasks1 = new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(MEMBER_EPOCH, mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + ); StreamsGroupMember member1 = new StreamsGroupMember.Builder(MEMBER_ID) - .setAssignedTasks(new TasksTuple( - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), - mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) - )) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignedTasks(assignedTasks1) .build(); + TasksTupleWithEpochs assignedTasks2 = new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(MEMBER_EPOCH, mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS4))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS5))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS6))) + ); StreamsGroupMember member2 = new StreamsGroupMember.Builder(MEMBER_ID) - .setAssignedTasks(new TasksTuple( - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS4))), - mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS5))), - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS6))) - )) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignedTasks(assignedTasks2) .build(); assertTrue(StreamsGroupMember.hasAssignedTasksChanged(member1, member2)); + TasksTupleWithEpochs assignedTasks3 = new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(MEMBER_EPOCH, mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + ); StreamsGroupMember member3 = new StreamsGroupMember.Builder(MEMBER_ID) - .setAssignedTasks(new TasksTuple( - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), - mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) - )) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignedTasks(assignedTasks3) .build(); + TasksTupleWithEpochs assignedTasks4 = new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(MEMBER_EPOCH, mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkTasksPerSubtopology(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + ); StreamsGroupMember member4 = new StreamsGroupMember.Builder(MEMBER_ID) - .setAssignedTasks(new TasksTuple( - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), - mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), - mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) - )) + .setMemberEpoch(MEMBER_EPOCH) + .setAssignedTasks(assignedTasks4) .build(); assertFalse(StreamsGroupMember.hasAssignedTasksChanged(member3, member4)); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java index 1324c0679c8c9..de1bf2d82c87f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage; import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder; +import org.apache.kafka.coordinator.group.CommitPartitionValidator; import org.apache.kafka.coordinator.group.OffsetAndMetadata; import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; @@ -66,7 +67,7 @@ import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; -import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopologyWithCommonEpoch; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -223,15 +224,15 @@ public void testUpdatingMemberUpdatesProcessId() { member = new StreamsGroupMember.Builder("member") .setProcessId("process") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopology, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopology, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 3)) ) ) .setTasksPendingRevocation( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(barSubtopology, 4)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(barSubtopology, 4)), mkTasksPerSubtopology(mkTasks(barSubtopology, 5)), mkTasksPerSubtopology(mkTasks(barSubtopology, 6)) ) @@ -259,15 +260,15 @@ public void testUpdatingMemberUpdatesProcessId() { member = new StreamsGroupMember.Builder(member) .setProcessId("process1") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopology, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopology, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 3)) ) ) .setTasksPendingRevocation( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(barSubtopology, 4)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(barSubtopology, 4)), mkTasksPerSubtopology(mkTasks(barSubtopology, 5)), mkTasksPerSubtopology(mkTasks(barSubtopology, 6)) ) @@ -302,16 +303,10 @@ public void testUpdatingMemberUpdatesTaskProcessIdWhenPartitionIsReassignedBefor member = new StreamsGroupMember.Builder("member") .setProcessId("process") - .setAssignedTasks( - new TasksTuple( - Map.of(), - Map.of(), - Map.of() - ) - ) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) .setTasksPendingRevocation( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 3)) ) @@ -325,13 +320,13 @@ public void testUpdatingMemberUpdatesTaskProcessIdWhenPartitionIsReassignedBefor member = new StreamsGroupMember.Builder(member) .setProcessId("process1") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 3)) ) ) - .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build(); streamsGroup.updateMember(member); @@ -347,8 +342,8 @@ public void testUpdatingMemberUpdatesTaskProcessIdWhenPartitionIsNotReleased() { StreamsGroupMember m1 = new StreamsGroupMember.Builder("m1") .setProcessId("process") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), Map.of(), Map.of() ) @@ -360,8 +355,8 @@ public void testUpdatingMemberUpdatesTaskProcessIdWhenPartitionIsNotReleased() { StreamsGroupMember m2 = new StreamsGroupMember.Builder("m2") .setProcessId("process") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), Map.of(), Map.of() ) @@ -382,20 +377,20 @@ public void testRemoveTaskProcessIds(TaskRole taskRole) { // Removing should fail because there is no epoch set. assertThrows(IllegalStateException.class, () -> streamsGroup.removeTaskProcessIds( - mkTasksTuple(taskRole, mkTasks(fooSubtopologyId, 1)), + TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch(taskRole, 10, mkTasks(fooSubtopologyId, 1)), "process" )); StreamsGroupMember m1 = new StreamsGroupMember.Builder("m1") .setProcessId("process") - .setAssignedTasks(mkTasksTuple(taskRole, mkTasks(fooSubtopologyId, 1))) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch(taskRole, 10, mkTasks(fooSubtopologyId, 1))) .build(); streamsGroup.updateMember(m1); // Removing should fail because the expected epoch is incorrect. assertThrows(IllegalStateException.class, () -> streamsGroup.removeTaskProcessIds( - mkTasksTuple(taskRole, mkTasks(fooSubtopologyId, 1)), + TaskAssignmentTestUtil.mkTasksTupleWithCommonEpoch(taskRole, 10, mkTasks(fooSubtopologyId, 1)), "process1" )); } @@ -406,8 +401,8 @@ public void testAddTaskProcessIds() { StreamsGroup streamsGroup = createStreamsGroup("foo"); streamsGroup.addTaskProcessId( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 3)) ), @@ -417,8 +412,8 @@ public void testAddTaskProcessIds() { // Changing the epoch should fail because the owner of the partition // should remove it first. assertThrows(IllegalStateException.class, () -> streamsGroup.addTaskProcessId( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopologyId, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 3)) ), @@ -438,15 +433,15 @@ public void testDeletingMemberRemovesProcessId() { member = new StreamsGroupMember.Builder("member") .setProcessId("process") .setAssignedTasks( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(fooSubtopology, 1)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(fooSubtopology, 1)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 2)), mkTasksPerSubtopology(mkTasks(fooSubtopology, 3)) ) ) .setTasksPendingRevocation( - new TasksTuple( - mkTasksPerSubtopology(mkTasks(barSubtopology, 4)), + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(10, mkTasks(barSubtopology, 4)), mkTasksPerSubtopology(mkTasks(barSubtopology, 5)), mkTasksPerSubtopology(mkTasks(barSubtopology, 6)) ) @@ -666,7 +661,7 @@ public void testValidateOffsetCommit(short version) { assertThrows(UnknownMemberIdException.class, () -> group.validateOffsetCommit("", null, -1, isTransactional, version)); - // The member epoch is stale. + // The member epoch is stale (newer than current). if (version >= 9) { assertThrows(StaleMemberEpochException.class, () -> group.validateOffsetCommit("new-protocol-member-id", "", 10, isTransactional, version)); @@ -675,7 +670,7 @@ public void testValidateOffsetCommit(short version) { group.validateOffsetCommit("new-protocol-member-id", "", 10, isTransactional, version)); } - // This should succeed. + // This should succeed (matching member epoch). if (version >= 9) { group.validateOffsetCommit("new-protocol-member-id", "", 0, isTransactional, version); } else { @@ -684,6 +679,108 @@ public void testValidateOffsetCommit(short version) { } } + @Test + public void testValidateOffsetCommitWithOlderEpoch() { + StreamsGroup group = createStreamsGroup("group-foo"); + + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("input-topic"))))); + + group.updateMember(new StreamsGroupMember.Builder("member-1") + .setMemberEpoch(2) + .setAssignedTasks(new TasksTupleWithEpochs( + Map.of("0", Map.of(0, 2, 1, 1)), + Map.of(), Map.of())) + .build()); + + CommitPartitionValidator validator = group.validateOffsetCommit( + "member-1", "", 1, false, ApiKeys.OFFSET_COMMIT.latestVersion()); + + // Received epoch (1) < assignment epoch (2) should throw + assertThrows(StaleMemberEpochException.class, () -> + validator.validate("input-topic", Uuid.ZERO_UUID, 0)); + } + + @Test + public void testValidateOffsetCommitWithOlderEpochMissingTopology() { + StreamsGroup group = createStreamsGroup("group-foo"); + + group.updateMember(new StreamsGroupMember.Builder("member-1") + .setMemberEpoch(2) + .build()); + + // Topology is retrieved when creating validator, so exception is thrown here + assertThrows(StaleMemberEpochException.class, () -> + group.validateOffsetCommit("member-1", "", 1, false, ApiKeys.OFFSET_COMMIT.latestVersion())); + } + + @Test + public void testValidateOffsetCommitWithOlderEpochMissingSubtopology() { + StreamsGroup group = createStreamsGroup("group-foo"); + + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("input-topic"))))); + + group.updateMember(new StreamsGroupMember.Builder("member-1") + .setMemberEpoch(2) + .build()); + + CommitPartitionValidator validator = group.validateOffsetCommit( + "member-1", "", 1, false, ApiKeys.OFFSET_COMMIT.latestVersion()); + + assertThrows(StaleMemberEpochException.class, () -> + validator.validate("unknown-topic", Uuid.ZERO_UUID, 0)); + } + + @Test + public void testValidateOffsetCommitWithOlderEpochUnassignedPartition() { + StreamsGroup group = createStreamsGroup("group-foo"); + + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("input-topic"))))); + + group.updateMember(new StreamsGroupMember.Builder("member-1") + .setMemberEpoch(2) + .setAssignedTasks(new TasksTupleWithEpochs( + Map.of("0", Map.of(0, 1)), + Map.of(), Map.of())) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) + .build()); + + CommitPartitionValidator validator = group.validateOffsetCommit( + "member-1", "", 1, false, ApiKeys.OFFSET_COMMIT.latestVersion()); + + // Partition 1 not assigned should throw + assertThrows(StaleMemberEpochException.class, () -> + validator.validate("input-topic", Uuid.ZERO_UUID, 1)); + } + + @Test + public void testValidateOffsetCommitWithOlderEpochValidAssignment() { + StreamsGroup group = createStreamsGroup("group-foo"); + + group.setTopology(new StreamsTopology(1, Map.of("0", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("0") + .setSourceTopics(List.of("input-topic"))))); + + group.updateMember(new StreamsGroupMember.Builder("member-1") + .setMemberEpoch(5) + .setAssignedTasks(new TasksTupleWithEpochs( + Map.of("0", Map.of(0, 2, 1, 2)), + Map.of(), Map.of())) + .build()); + + CommitPartitionValidator validator = group.validateOffsetCommit( + "member-1", "", 2, false, ApiKeys.OFFSET_COMMIT.latestVersion()); + + // Received epoch 2 == assignment epoch 2 should succeed + validator.validate("input-topic", Uuid.ZERO_UUID, 0); + validator.validate("input-topic", Uuid.ZERO_UUID, 1); + } + @Test public void testAsListedGroup() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT); @@ -822,8 +919,8 @@ public void testAsDescribedGroup() { .setProcessId("process1") .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host1").setPort(9092)) .setClientTags(Map.of("tag1", "value1")) - .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) - .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build()); group.updateMember(new StreamsGroupMember.Builder("member2") .setMemberEpoch(1) @@ -838,8 +935,8 @@ public void testAsDescribedGroup() { .setProcessId("process2") .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host2").setPort(9092)) .setClientTags(Map.of("tag2", "value2")) - .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) - .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build()); snapshotRegistry.idempotentCreateSnapshot(1); @@ -1047,8 +1144,8 @@ public void testAsDescribedGroupWithStreamsTopologyHavingSubtopologies() { .setProcessId("process1") .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host1").setPort(9092)) .setClientTags(Map.of("tag1", "value1")) - .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) - .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setAssignedTasks(TasksTupleWithEpochs.EMPTY) + .setTasksPendingRevocation(TasksTupleWithEpochs.EMPTY) .build()); snapshotRegistry.idempotentCreateSnapshot(1); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java index f633fec80f7c1..56da4678b7234 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java @@ -40,6 +40,24 @@ public static TasksTuple mkTasksTuple(TaskRole taskRole, Map.Entry>... entries) { + return switch (taskRole) { + case ACTIVE -> new TasksTupleWithEpochs(mkTasksPerSubtopologyWithCommonEpoch(defaultAssignmentEpoch, entries), new HashMap<>(), new HashMap<>()); + case STANDBY -> new TasksTupleWithEpochs(new HashMap<>(), mkTasksPerSubtopology(entries), new HashMap<>()); + case WARMUP -> new TasksTupleWithEpochs(new HashMap<>(), new HashMap<>(), mkTasksPerSubtopology(entries)); + }; + } + + @SafeVarargs + public static TasksTupleWithEpochs mkTasksTupleWithEpochs(TaskRole taskRole, Map.Entry>... entries) { + return switch (taskRole) { + case ACTIVE -> new TasksTupleWithEpochs(mkTasksWithEpochsPerSubtopology(entries), new HashMap<>(), new HashMap<>()); + case STANDBY -> new TasksTupleWithEpochs(new HashMap<>(), mkTasksPerSubtopologyStripEpoch(entries), new HashMap<>()); + case WARMUP -> new TasksTupleWithEpochs(new HashMap<>(), new HashMap<>(), mkTasksPerSubtopologyStripEpoch(entries)); + }; + } + public static Map.Entry> mkTasks(String subtopologyId, Integer... tasks) { return new AbstractMap.SimpleEntry<>( @@ -48,6 +66,14 @@ public static Map.Entry> mkTasks(String subtopologyId, ); } + public static Map.Entry> mkTasksWithEpochs(String subtopologyId, + Map tasks) { + return new AbstractMap.SimpleEntry<>( + subtopologyId, + tasks + ); + } + @SafeVarargs public static Map> mkTasksPerSubtopology(Map.Entry>... entries) { Map> assignment = new HashMap<>(); @@ -56,4 +82,35 @@ public static Map> mkTasksPerSubtopology(Map.Entry> mkTasksPerSubtopologyStripEpoch(Map.Entry>... entries) { + Map> assignment = new HashMap<>(); + for (Map.Entry> entry : entries) { + assignment.put(entry.getKey(), entry.getValue().keySet()); + } + return assignment; + } + + @SafeVarargs + public static Map> mkTasksPerSubtopologyWithCommonEpoch(int defaultAssignmentEpoch, Map.Entry>... entries) { + Map> assignment = new HashMap<>(); + for (Map.Entry> entry : entries) { + Map partitionEpochMap = new HashMap<>(); + for (Integer partition : entry.getValue()) { + partitionEpochMap.put(partition, defaultAssignmentEpoch); + } + assignment.put(entry.getKey(), partitionEpochMap); + } + return assignment; + } + + @SafeVarargs + public static Map> mkTasksWithEpochsPerSubtopology(Map.Entry>... entries) { + Map> assignment = new HashMap<>(); + for (Map.Entry> entry : entries) { + assignment.put(entry.getKey(), entry.getValue()); + } + return assignment; + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java index 836e42fb6e9a7..94eee2484e363 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java @@ -122,49 +122,48 @@ public void testFromTargetAssignmentRecord() { } @Test - public void testMerge() { + public void testContainsAny() { TasksTuple tuple1 = new TasksTuple( Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3)), Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6)), Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9)) ); - TasksTuple tuple2 = new TasksTuple( - Map.of(SUBTOPOLOGY_1, Set.of(10, 11)), + // Test with overlapping active tasks + TasksTupleWithEpochs tuple2 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(3, 5, 10, 5, 11, 5)), Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) ); - TasksTuple mergedTuple = tuple1.merge(tuple2); - - assertEquals(Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3, 10, 11)), mergedTuple.activeTasks()); - assertEquals(Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6, 12, 13)), mergedTuple.standbyTasks()); - assertEquals(Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9, 14, 15)), mergedTuple.warmupTasks()); - } + assertTrue(tuple1.containsAny(tuple2)); - @Test - public void testContainsAny() { - TasksTuple tuple1 = new TasksTuple( - Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3)), - Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6)), - Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9)) + // Test with no overlapping tasks + TasksTupleWithEpochs tuple3 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(10, 5, 11, 5)), + Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), + Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) ); - TasksTuple tuple2 = new TasksTuple( - Map.of(SUBTOPOLOGY_1, Set.of(3, 10, 11)), - Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), + assertFalse(tuple1.containsAny(tuple3)); + + // Test with overlapping standby tasks + TasksTupleWithEpochs tuple4 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(10, 5, 11, 5)), + Map.of(SUBTOPOLOGY_2, Set.of(4, 12, 13)), Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) ); - assertTrue(tuple1.containsAny(tuple2)); + assertTrue(tuple1.containsAny(tuple4)); - TasksTuple tuple3 = new TasksTuple( - Map.of(SUBTOPOLOGY_1, Set.of(10, 11)), + // Test with overlapping warmup tasks + TasksTupleWithEpochs tuple5 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(10, 5, 11, 5)), Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), - Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) + Map.of(SUBTOPOLOGY_3, Set.of(7, 14, 15)) ); - assertFalse(tuple1.containsAny(tuple3)); + assertTrue(tuple1.containsAny(tuple5)); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochsTest.java new file mode 100644 index 0000000000000..f92a23847fa59 --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleWithEpochsTest.java @@ -0,0 +1,258 @@ +/* + * 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.coordinator.group.streams; + +import org.apache.kafka.coordinator.group.generated.StreamsGroupCurrentMemberAssignmentValue; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TasksTupleWithEpochsTest { + + private static final String SUBTOPOLOGY_1 = "1"; + private static final String SUBTOPOLOGY_2 = "2"; + private static final String SUBTOPOLOGY_3 = "3"; + + @Test + public void testTasksCannotBeNull() { + assertThrows(NullPointerException.class, () -> new TasksTupleWithEpochs(null, Map.of(), Map.of())); + assertThrows(NullPointerException.class, () -> new TasksTupleWithEpochs(Map.of(), null, Map.of())); + assertThrows(NullPointerException.class, () -> new TasksTupleWithEpochs(Map.of(), Map.of(), null)); + } + + @Test + public void testReturnUnmodifiableTaskAssignments() { + Map> activeTasks = Map.of( + SUBTOPOLOGY_1, Map.of(1, 10, 2, 11, 3, 12) + ); + Map> standbyTasks = mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_2, 9, 8, 7) + ); + Map> warmupTasks = mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_3, 4, 5, 6) + ); + TasksTupleWithEpochs tuple = new TasksTupleWithEpochs(activeTasks, standbyTasks, warmupTasks); + + assertEquals(activeTasks, tuple.activeTasksWithEpochs()); + assertThrows(UnsupportedOperationException.class, () -> tuple.activeTasksWithEpochs().put("not allowed", Map.of())); + assertEquals(standbyTasks, tuple.standbyTasks()); + assertThrows(UnsupportedOperationException.class, () -> tuple.standbyTasks().put("not allowed", Set.of())); + assertEquals(warmupTasks, tuple.warmupTasks()); + assertThrows(UnsupportedOperationException.class, () -> tuple.warmupTasks().put("not allowed", Set.of())); + } + + + @Test + public void testFromCurrentAssignmentRecord() { + List activeTasks = new ArrayList<>(); + activeTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_1) + .setPartitions(Arrays.asList(1, 2, 3)) + .setAssignmentEpochs(Arrays.asList(10, 11, 12))); + activeTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_2) + .setPartitions(Arrays.asList(4, 5, 6)) + .setAssignmentEpochs(Arrays.asList(20, 21, 22))); + + List standbyTasks = new ArrayList<>(); + standbyTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_1) + .setPartitions(Arrays.asList(7, 8, 9))); + standbyTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_2) + .setPartitions(Arrays.asList(1, 2, 3))); + + List warmupTasks = new ArrayList<>(); + warmupTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_1) + .setPartitions(Arrays.asList(4, 5, 6))); + warmupTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_2) + .setPartitions(Arrays.asList(7, 8, 9))); + + TasksTupleWithEpochs tuple = TasksTupleWithEpochs.fromCurrentAssignmentRecord( + activeTasks, standbyTasks, warmupTasks, 100 + ); + + assertEquals( + Map.of( + SUBTOPOLOGY_1, Map.of(1, 10, 2, 11, 3, 12), + SUBTOPOLOGY_2, Map.of(4, 20, 5, 21, 6, 22) + ), + tuple.activeTasksWithEpochs() + ); + assertEquals( + mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_1, 7, 8, 9), + mkTasks(SUBTOPOLOGY_2, 1, 2, 3) + ), + tuple.standbyTasks() + ); + assertEquals( + mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_1, 4, 5, 6), + mkTasks(SUBTOPOLOGY_2, 7, 8, 9) + ), + tuple.warmupTasks() + ); + } + + @Test + public void testFromCurrentAssignmentRecordWithoutEpochs() { + // Test legacy format where epochs are not present + List activeTasks = new ArrayList<>(); + activeTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_1) + .setPartitions(Arrays.asList(1, 2, 3))); + + int memberEpoch = 100; + TasksTupleWithEpochs tuple = TasksTupleWithEpochs.fromCurrentAssignmentRecord( + activeTasks, List.of(), List.of(), memberEpoch + ); + + // Should use member epoch as default + assertEquals( + Map.of(SUBTOPOLOGY_1, Map.of(1, memberEpoch, 2, memberEpoch, 3, memberEpoch)), + tuple.activeTasksWithEpochs() + ); + } + + @Test + public void testFromCurrentAssignmentRecordWithMismatchedEpochs() { + // Test error case where number of epochs doesn't match number of partitions + List activeTasks = new ArrayList<>(); + activeTasks.add(new StreamsGroupCurrentMemberAssignmentValue.TaskIds() + .setSubtopologyId(SUBTOPOLOGY_1) + .setPartitions(Arrays.asList(1, 2, 3)) + .setAssignmentEpochs(Arrays.asList(10, 11))); // Only 2 epochs for 3 partitions + + assertThrows(IllegalStateException.class, () -> + TasksTupleWithEpochs.fromCurrentAssignmentRecord(activeTasks, List.of(), List.of(), 100) + ); + } + + @Test + public void testIsEmpty() { + TasksTupleWithEpochs emptyTuple = new TasksTupleWithEpochs(Map.of(), Map.of(), Map.of()); + assertTrue(emptyTuple.isEmpty()); + assertEquals(TasksTupleWithEpochs.EMPTY, emptyTuple); + + TasksTupleWithEpochs nonEmptyTuple = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(1, 10)), + Map.of(), + Map.of() + ); + assertFalse(nonEmptyTuple.isEmpty()); + } + + @Test + public void testMerge() { + TasksTupleWithEpochs tuple1 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(1, 10, 2, 11)), + Map.of(SUBTOPOLOGY_2, Set.of(4, 5)), + Map.of(SUBTOPOLOGY_3, Set.of(7, 8)) + ); + + TasksTupleWithEpochs tuple2 = new TasksTupleWithEpochs( + Map.of( + SUBTOPOLOGY_1, Map.of(3, 13), // Different partition in same subtopology + SUBTOPOLOGY_2, Map.of(6, 26) // Different subtopology + ), + Map.of(SUBTOPOLOGY_2, Set.of(9, 10)), + Map.of(SUBTOPOLOGY_3, Set.of(11, 12)) + ); + + TasksTupleWithEpochs merged = tuple1.merge(tuple2); + + assertEquals( + Map.of( + SUBTOPOLOGY_1, Map.of(1, 10, 2, 11, 3, 13), + SUBTOPOLOGY_2, Map.of(6, 26) + ), + merged.activeTasksWithEpochs() + ); + assertEquals( + mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_2, 4, 5, 9, 10) + ), + merged.standbyTasks() + ); + assertEquals( + mkTasksPerSubtopology( + mkTasks(SUBTOPOLOGY_3, 7, 8, 11, 12) + ), + merged.warmupTasks() + ); + } + + @Test + public void testMergeWithOverlappingActiveTasks() { + // When merging overlapping active tasks, epochs from the second tuple take precedence + TasksTupleWithEpochs tuple1 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(1, 10, 2, 11)), + Map.of(), + Map.of() + ); + + TasksTupleWithEpochs tuple2 = new TasksTupleWithEpochs( + Map.of(SUBTOPOLOGY_1, Map.of(1, 99, 3, 13)), // partition 1 overlaps with different epoch + Map.of(), + Map.of() + ); + + TasksTupleWithEpochs merged = tuple1.merge(tuple2); + + // Epoch for partition 1 should be from tuple2 (99, not 10) since the second tuple takes precedence + assertEquals(99, merged.activeTasksWithEpochs().get(SUBTOPOLOGY_1).get(1)); + assertEquals(11, merged.activeTasksWithEpochs().get(SUBTOPOLOGY_1).get(2)); + assertEquals(13, merged.activeTasksWithEpochs().get(SUBTOPOLOGY_1).get(3)); + } + + @Test + public void testToString() { + TasksTupleWithEpochs tuple = new TasksTupleWithEpochs( + Map.of( + SUBTOPOLOGY_1, Map.of(1, 10, 2, 11), + SUBTOPOLOGY_2, Map.of(3, 20) + ), + Map.of(SUBTOPOLOGY_2, Set.of(4, 5)), + Map.of(SUBTOPOLOGY_3, Set.of(6)) + ); + + String result = tuple.toString(); + + // Verify the exact toString format + assertEquals( + "(active=[1-1@10, 1-2@11, 2-3@20], " + + "standby=[2-4, 2-5], " + + "warmup=[3-6])", + result + ); + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java index a59bab697f844..97af1bd9ccfda 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java @@ -23,7 +23,7 @@ import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder; import org.apache.kafka.coordinator.group.streams.StreamsGroup; import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; -import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.TasksTupleWithEpochs; import org.apache.kafka.image.MetadataImage; import org.junit.jupiter.api.BeforeEach; @@ -44,6 +44,9 @@ import java.util.TreeMap; import java.util.stream.Stream; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopologyWithCommonEpoch; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.params.provider.Arguments.arguments; import static org.mockito.Mockito.mock; @@ -56,9 +59,6 @@ class EndpointToPartitionsManagerTest { private ConfiguredTopology configuredTopology; private ConfiguredSubtopology configuredSubtopologyOne; private ConfiguredSubtopology configuredSubtopologyTwo; - private final Map> activeTasks = new HashMap<>(); - private final Map> standbyTasks = new HashMap<>(); - private TasksTuple tasksTuple; private final StreamsGroupHeartbeatResponseData.Endpoint responseEndpoint = new StreamsGroupHeartbeatResponseData.Endpoint(); @BeforeEach @@ -86,11 +86,13 @@ void testEndpointToPartitionsWithStandbyTaskAssignments() { .addTopic(Uuid.randomUuid(), "Topic-B", 3) .build(); - activeTasks.put("0", Set.of(0, 1, 2)); - standbyTasks.put("1", Set.of(0, 1, 2)); - tasksTuple = new TasksTuple(activeTasks, standbyTasks, Collections.emptyMap()); - when(streamsGroupMember.assignedTasks()).thenReturn(tasksTuple); - //when(streamsGroupMember.assignedTasks().standbyTasks()).thenReturn(tasksTuple.standbyTasks()); + when(streamsGroupMember.assignedTasks()).thenReturn( + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(0, mkEntry("0", Set.of(0, 1, 2))), + mkTasksPerSubtopology(mkEntry("1", Set.of(0, 1, 2))), + Map.of() + ) + ); when(streamsGroup.configuredTopology()).thenReturn(Optional.of(configuredTopology)); SortedMap configuredSubtopologyMap = new TreeMap<>(); configuredSubtopologyMap.put("0", configuredSubtopologyOne); @@ -131,8 +133,13 @@ void testEndpointToPartitionsWithTwoTopicsAndDifferentPartitions(int topicAParti .build(); configuredSubtopologyOne = new ConfiguredSubtopology(Math.max(topicAPartitions, topicBPartitions), Set.of("Topic-A", "Topic-B"), new HashMap<>(), new HashSet<>(), new HashMap<>()); - activeTasks.put("0", Set.of(0, 1, 2, 3, 4)); - when(streamsGroupMember.assignedTasks()).thenReturn(new TasksTuple(activeTasks, Collections.emptyMap(), Collections.emptyMap())); + when(streamsGroupMember.assignedTasks()).thenReturn( + new TasksTupleWithEpochs( + mkTasksPerSubtopologyWithCommonEpoch(0, mkEntry("0", Set.of(0, 1, 2, 3, 4))), + Map.of(), + Map.of() + ) + ); when(streamsGroup.configuredTopology()).thenReturn(Optional.of(configuredTopology)); SortedMap configuredSubtopologyOneMap = new TreeMap<>(); configuredSubtopologyOneMap.put("0", configuredSubtopologyOne); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsAssignorBenchmarkUtils.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsAssignorBenchmarkUtils.java index 910b81429b712..fa3324d012e64 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsAssignorBenchmarkUtils.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsAssignorBenchmarkUtils.java @@ -56,9 +56,9 @@ public static GroupSpec createGroupSpec( memberSpecs.put(memberId, new AssignmentMemberSpec( member.instanceId(), member.rackId(), - member.assignedTasks().activeTasks(), - member.assignedTasks().standbyTasks(), - member.assignedTasks().warmupTasks(), + Map.of(), + Map.of(), + Map.of(), member.processId(), member.clientTags(), Map.of(), diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java index 4e59e9523c468..3a6b15e29b1ba 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java @@ -47,6 +47,7 @@ import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; import static org.apache.kafka.streams.tests.SmokeTestDriver.verify; import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -140,11 +141,14 @@ public void shouldWorkWithRebalance( throw new AssertionError("Test called halt(). code:" + statusCode + " message:" + message); }); int numClientsCreated = 0; + int numDataRecordsProcessed = 0; + final int numKeys = 10; + final int maxRecordsPerKey = 1000; IntegrationTestUtils.cleanStateBeforeTest(cluster, SmokeTestDriver.topics()); final String bootstrapServers = cluster.bootstrapServers(); - final Driver driver = new Driver(bootstrapServers, 10, 1000); + final Driver driver = new Driver(bootstrapServers, numKeys, maxRecordsPerKey); driver.start(); System.out.println("started driver"); @@ -183,6 +187,7 @@ public void shouldWorkWithRebalance( assertFalse(client.error(), "The streams application seems to have crashed."); Thread.sleep(100); } + numDataRecordsProcessed += client.totalDataRecordsProcessed(); } } @@ -201,6 +206,7 @@ public void shouldWorkWithRebalance( assertFalse(client.error(), "The streams application seems to have crashed."); Thread.sleep(100); } + numDataRecordsProcessed += client.totalDataRecordsProcessed(); } } @@ -210,5 +216,16 @@ public void shouldWorkWithRebalance( throw new AssertionError(driver.exception()); } assertTrue(driver.result().passed(), driver.result().result()); + + // The one extra record is a record that the driver produces to flush suppress + final int expectedRecords = numKeys * maxRecordsPerKey + 1; + + // We check that we did no have to reprocess any records, which would indicate a bug since everything + // runs locally in this test. + assertEquals(expectedRecords, numDataRecordsProcessed, + String.format("It seems we had to reprocess records, expected %d records, processed %d records.", + expectedRecords, + numDataRecordsProcessed) + ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 7f8057c559787..b0012fa61b403 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -44,6 +44,7 @@ import java.util.Properties; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; @@ -55,6 +56,7 @@ public class SmokeTestClient extends SmokeTestUtil { private boolean uncaughtException = false; private volatile boolean closed; private volatile boolean error; + private final AtomicInteger totalDataRecordsProcessed = new AtomicInteger(0); private static void addShutdownHook(final String name, final Runnable runnable) { if (name != null) { @@ -76,6 +78,10 @@ public boolean error() { return error; } + public int totalDataRecordsProcessed() { + return totalDataRecordsProcessed.get(); + } + public void start(final Properties streamsProperties) { final Topology build = getTopology(); streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); @@ -156,7 +162,7 @@ public Topology getTopology() { source.filterNot((k, v) -> k.equals("flush")) .to("echo", Produced.with(stringSerde, intSerde)); final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + data.process(SmokeTestUtil.printProcessorSupplier("data", name, totalDataRecordsProcessed)); // min final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 7e670802b93ad..d0ad6c8cabba8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.api.Record; import java.time.Instant; +import java.util.concurrent.atomic.AtomicInteger; public class SmokeTestUtil { @@ -39,6 +40,10 @@ static ProcessorSupplier printProcessorSupplier(fina } static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return printProcessorSupplier(topic, name, new AtomicInteger()); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name, final AtomicInteger totalRecordsProcessed) { return () -> new ContextualProcessor<>() { private int numRecordsProcessed = 0; private long smallestOffset = Long.MAX_VALUE; @@ -84,6 +89,7 @@ public void close() { } System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); System.out.flush(); + totalRecordsProcessed.addAndGet(numRecordsProcessed); } }; }