jolshan commented on code in PR #13639: URL: https://github.com/apache/kafka/pull/13639#discussion_r1203225900
########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ########## @@ -0,0 +1,2017 @@ +/* + * 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; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.GroupMaxSizeReachedException; +import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnsupportedAssignorException; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.network.ClientInformation; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.requests.RequestContext; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; +import org.apache.kafka.coordinator.group.consumer.Assignment; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.TopicMetadata; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsDelta; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Arrays; +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; + +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; +import static org.junit.jupiter.api.AssertionFailureBuilder.assertionFailure; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class GroupMetadataManagerTest { + static class MockPartitionAssignor implements PartitionAssignor { + private final String name; + private AssignmentSpec lastSpecReceived = null; + private GroupAssignment prepareGroupAssignment = null; + + MockPartitionAssignor(String name) { + this.name = name; + } + + public AssignmentSpec lastSpecReceived() { + return lastSpecReceived; + } + + public void prepareGroupAssignment(GroupAssignment prepareGroupAssignment) { + this.prepareGroupAssignment = prepareGroupAssignment; + } + + @Override + public String name() { + return name; + } + + @Override + public GroupAssignment assign(AssignmentSpec assignmentSpec) throws PartitionAssignorException { + lastSpecReceived = assignmentSpec; + return prepareGroupAssignment; + } + } + + public static class TopicsImageBuilder { + private TopicsDelta delta = new TopicsDelta(TopicsImage.EMPTY); + + public TopicsImageBuilder addTopic( + Uuid topicId, + String topicName, + int numPartitions + ) { + delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName)); + for (int i = 0; i < numPartitions; i++) { + delta.replay(new PartitionRecord() + .setTopicId(topicId) + .setPartitionId(i)); + } + return this; + } + + public TopicsImage build() { + return delta.apply(); + } + } + + static class ConsumerGroupBuilder { + private final String groupId; + private final int groupEpoch; + private int assignmentEpoch; + private final Map<String, ConsumerGroupMember> members = new HashMap<>(); + private final Map<String, Assignment> assignments = new HashMap<>(); + + public ConsumerGroupBuilder(String groupId, int groupEpoch) { + this.groupId = groupId; + this.groupEpoch = groupEpoch; + this.assignmentEpoch = 0; + } + + public ConsumerGroupBuilder withMember(ConsumerGroupMember member) { + this.members.put(member.memberId(), member); + return this; + } + + public ConsumerGroupBuilder withAssignment(String memberId, Map<Uuid, Set<Integer>> assignment) { + this.assignments.put(memberId, new Assignment(assignment)); + return this; + } + + public ConsumerGroupBuilder withAssignment(String memberId, Assignment assignment) { + this.assignments.put(memberId, assignment); + return this; + } + + public ConsumerGroupBuilder withAssignmentEpoch(int assignmentEpoch) { + this.assignmentEpoch = assignmentEpoch; + return this; + } + + public List<Record> build(TopicsImage topicsImage) { + List<Record> records = new ArrayList<>(); + + // Add subscription records for members. + members.forEach((memberId, member) -> { + records.add(RecordHelpers.newMemberSubscriptionRecord(groupId, member)); + }); + + // Add subscription metadata. + Map<String, TopicMetadata> subscriptionMetadata = new HashMap<>(); + members.forEach((memberId, member) -> { + member.subscribedTopicNames().forEach(topicName -> + subscriptionMetadata.computeIfAbsent(topicName, __ -> { + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage == null) { + return null; + } else { + return new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + ); + } + }) + ); + }); + if (!subscriptionMetadata.isEmpty()) { + records.add(RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, subscriptionMetadata)); + } + + // Add group epoch record. + records.add(RecordHelpers.newGroupEpochRecord(groupId, groupEpoch)); + + // Add target assignment records. + assignments.forEach((memberId, assignment) -> { + records.add(RecordHelpers.newTargetAssignmentRecord(groupId, memberId, assignment.partitions())); + }); + + // Add target assignment epoch. + records.add(RecordHelpers.newTargetAssignmentEpochRecord(groupId, assignmentEpoch)); + + // Add current assignment records for members. + members.forEach((memberId, member) -> { + records.add(RecordHelpers.newCurrentAssignmentRecord(groupId, member)); + }); + + return records; + } + } + + static class GroupMetadataManagerTestContext { + static class Builder { + private LogContext logContext; + private SnapshotRegistry snapshotRegistry; + private TopicsImage topicsImage; + private List<PartitionAssignor> assignors; + private List<ConsumerGroupBuilder> consumerGroupBuilders = new ArrayList<>(); + private int consumerGroupMaxSize = Integer.MAX_VALUE; + + public Builder withLogContext(LogContext logContext) { + this.logContext = logContext; + return this; + } + + public Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { + this.snapshotRegistry = snapshotRegistry; + return this; + } + + public Builder withTopicsImage(TopicsImage topicsImage) { + this.topicsImage = topicsImage; + return this; + } + + public Builder withAssignors(List<PartitionAssignor> assignors) { + this.assignors = assignors; + return this; + } + + public Builder withConsumerGroup(ConsumerGroupBuilder builder) { + this.consumerGroupBuilders.add(builder); + return this; + } + + public Builder withConsumerGroupMaxSize(int consumerGroupMaxSize) { + this.consumerGroupMaxSize = consumerGroupMaxSize; + return this; + } + + public GroupMetadataManagerTestContext build() { + if (logContext == null) logContext = new LogContext(); + if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); + if (topicsImage == null) topicsImage = TopicsImage.EMPTY; + if (assignors == null) assignors = Collections.emptyList(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext( + snapshotRegistry, + new GroupMetadataManager.Builder() + .withSnapshotRegistry(snapshotRegistry) + .withLogContext(logContext) + .withTopicsImage(topicsImage) + .withConsumerGroupHeartbeatInterval(5000) + .withConsumerGroupMaxSize(consumerGroupMaxSize) + .withAssignors(assignors) + .build() + ); + + consumerGroupBuilders.forEach(builder -> { + builder.build(topicsImage).forEach(context::replay); + }); + + context.commit(); + + return context; + } + } + + final SnapshotRegistry snapshotRegistry; + final GroupMetadataManager groupMetadataManager; + + long lastCommittedOffset = 0L; + long lastWrittenOffset = 0L; + + public GroupMetadataManagerTestContext( + SnapshotRegistry snapshotRegistry, + GroupMetadataManager groupMetadataManager + ) { + this.snapshotRegistry = snapshotRegistry; + this.groupMetadataManager = groupMetadataManager; + } + + public void commit() { + long lastCommittedOffset = this.lastCommittedOffset; + this.lastCommittedOffset = lastWrittenOffset; + snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset); + } + + public void rollback() { + lastWrittenOffset = lastCommittedOffset; + snapshotRegistry.revertToSnapshot(lastCommittedOffset); + } + + public ConsumerGroup.ConsumerGroupState consumerGroupState( + String groupId + ) { + return groupMetadataManager + .getOrMaybeCreateConsumerGroup(groupId, false) + .state(); + } + + public ConsumerGroupMember.MemberState consumerGroupMemberState( + String groupId, + String memberId + ) { + return groupMetadataManager + .getOrMaybeCreateConsumerGroup(groupId, false) + .getOrMaybeCreateMember(memberId, false) + .state(); + } + + public Result<ConsumerGroupHeartbeatResponseData> consumerGroupHeartbeat( + ConsumerGroupHeartbeatRequestData request + ) { + snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset); + + RequestContext context = new RequestContext( + new RequestHeader( + ApiKeys.CONSUMER_GROUP_HEARTBEAT, + ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), + "client", + 0 + ), + "1", + InetAddress.getLoopbackAddress(), + KafkaPrincipal.ANONYMOUS, + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + SecurityProtocol.PLAINTEXT, + ClientInformation.EMPTY, + false + ); + + Result<ConsumerGroupHeartbeatResponseData> result = groupMetadataManager.consumerGroupHeartbeat( + context, + request + ); + + result.records().forEach(this::replay); + return result; + } + + private ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) { + if (apiMessageAndVersion == null) { + return null; + } else { + return apiMessageAndVersion.message(); + } + } + + private void replay( + Record record + ) { + ApiMessageAndVersion key = record.key(); + ApiMessageAndVersion value = record.value(); + + if (key == null) { + throw new IllegalStateException("Received a null key in " + record); + } + + switch (key.version()) { + case ConsumerGroupMemberMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupMemberMetadataKey) key.message(), + (ConsumerGroupMemberMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupMetadataKey) key.message(), + (ConsumerGroupMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupPartitionMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupPartitionMetadataKey) key.message(), + (ConsumerGroupPartitionMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupTargetAssignmentMemberKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupTargetAssignmentMemberKey) key.message(), + (ConsumerGroupTargetAssignmentMemberValue) messageOrNull(value) + ); + break; + + case ConsumerGroupTargetAssignmentMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupTargetAssignmentMetadataKey) key.message(), + (ConsumerGroupTargetAssignmentMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupCurrentMemberAssignmentKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupCurrentMemberAssignmentKey) key.message(), + (ConsumerGroupCurrentMemberAssignmentValue) messageOrNull(value) + ); + break; + + default: + throw new IllegalStateException("Received an unknown record type " + key.version() + + " in " + record); + } + + lastWrittenOffset++; + } + } + + @Test + public void testConsumerHeartbeatRequestValidation() { + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + Exception ex; + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData())); + assertEquals("GroupId can't be empty.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0))); + assertEquals("RebalanceTimeoutMs must be provided in first request.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000))); + assertEquals("TopicPartitions must be empty when (re-)joining.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setTopicPartitions(Collections.emptyList()))); + assertEquals("SubscribedTopicNames must be set in first request.", ex.getMessage()); + + ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setTopicPartitions(Collections.emptyList()) + .setSubscribedTopicNames(Collections.singletonList("foo")) + .setServerAssignor("bar"))); + assertEquals("ServerAssignor bar is not supported. Supported assignors: range.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(1))); + assertEquals("MemberId can't be empty.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setInstanceId("instance-id"))); + assertEquals("InstanceId should only be provided in first request.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setRackId("rack-id"))); + assertEquals("RackId should only be provided in first request.", ex.getMessage()); + + ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setServerAssignor("bar"))); + assertEquals("ServerAssignor bar is not supported. Supported assignors: range.", ex.getMessage()); + } + + @Test + public void testMemberIdGeneration() { + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(TopicsImage.EMPTY) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.emptyMap() + )); + + Result<ConsumerGroupHeartbeatResponseData> result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("group-foo") + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + // Verify that a member id was generated for the new member. + String memberId = result.response().memberId(); + assertNotNull(memberId); + assertNotEquals("", memberId); + + // The response should get a bumped epoch and should not + // contain any assignment because we did not provide + // topics metadata. + assertEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), + result.response() + ); + } + + @Test + public void testUnknownGroupId() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + assertThrows(GroupIdNotFoundException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(100) // Epoch must be > 0. + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testUnknownMemberIdJoinsConsumerGroup() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + + // A first member joins to create the group. + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + // The second member is rejected because the member id is unknown and + // the member epoch is not zero. + assertThrows(UnknownMemberIdException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testConsumerGroupMemberEpochValidation() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + Uuid fooTopicId = Uuid.randomUuid(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(100) + .setPreviousMemberEpoch(99) + .setNextMemberEpoch(100) + .setRebalanceTimeoutMs(5000) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) + .build(); + + context.replay(RecordHelpers.newMemberSubscriptionRecord(groupId, member)); + + context.replay(RecordHelpers.newGroupEpochRecord(groupId, 100)); + + context.replay(RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3) + ))); + + context.replay(RecordHelpers.newTargetAssignmentEpochRecord(groupId, 100)); + + context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, member)); + + // Member epoch is greater than the expected epoch. + assertThrows(FencedMemberEpochException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(200) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member epoch is smaller than the expected epoch. + assertThrows(FencedMemberEpochException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(50) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member joins with previous epoch but without providing partitions. + assertThrows(FencedMemberEpochException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member joins with previous epoch and has a subset of the owned partitions. This + // is accepted as the response with the bumped epoch may have been lost. In this + // case, we provide back the correct epoch to the member. + Result<ConsumerGroupHeartbeatResponseData> result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(1, 2))))); + assertEquals(100, result.response().memberEpoch()); + } + + @Test + public void testMemberJoinsEmptyConsumerGroup() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + ))) + )); + + assertThrows(GroupIdNotFoundException.class, () -> + context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false)); + + Result<ConsumerGroupHeartbeatResponseData> result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1, 2, 3, 4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0, 1, 2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(1) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2))) + .build(); + + List<Record> expectedRecords = Arrays.asList( + RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember), + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap<String, TopicMetadata>() {{ + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); + }}), + RecordHelpers.newGroupEpochRecord(groupId, 1), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 1), + RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertEquals(expectedRecords, result.records()); + } + + @Test + public void testUpdatingSubscriptionTriggersNewTargetAssignment() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5))) + .build()) + .withAssignment(memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5))) + .withAssignmentEpoch(10)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + ))) + )); + + Result<ConsumerGroupHeartbeatResponseData> result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10) + .setSubscribedTopicNames(Arrays.asList("foo", "bar"))); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1, 2, 3, 4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0, 1, 2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2))) + .build(); + + List<Record> expectedRecords = Arrays.asList( + RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember), + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap<String, TopicMetadata>() { + { + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); + } + }), + RecordHelpers.newGroupEpochRecord(groupId, 11), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11), + RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertEquals(expectedRecords, result.records()); + } + + @Test + public void testNewJoiningMemberTriggersNewTargetAssignment() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .withAssignmentEpoch(10)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap<String, org.apache.kafka.coordinator.group.assignor.MemberAssignment>() { + { + put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0) + ))); + put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2, 3), + mkTopicAssignment(barTopicId, 1) + ))); + put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 2) + ))); + } + } + )); + + // Member 3 joins the consumer group. + Result<ConsumerGroupHeartbeatResponseData> result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember3 = new ConsumerGroupMember.Builder(memberId3) Review Comment: I guess we sort of duplicate the code and test in testReconciliationProcess -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org