-
Notifications
You must be signed in to change notification settings - Fork 14.8k
KAFKA-19779: Add per-partition epoch validation to streams groups [4/N] #20760
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -1120,4 +1130,54 @@ public void setLastAssignmentConfigs(Map<String, String> 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."); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This case is actually impossible right now because we do not allow updating the topology yet. But I think this would be the correct behavior once we allow changing the topology: We are trying to commit for a subtopology that does not exist anymore, so we should fence the member. |
||
| } | ||
|
|
||
| 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)); | ||
| } | ||
| }; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -660,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)); | ||
|
|
@@ -669,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 { | ||
|
|
@@ -678,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)), | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seem we only test epoch quality? Should we use two different epochs for each partition to extend test coverage? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point. I made them different |
||
| Map.of(), Map.of())) | ||
| .build()); | ||
|
|
||
| CommitPartitionValidator validator = group.validateOffsetCommit( | ||
| "member-1", "", 2, false, ApiKeys.OFFSET_COMMIT.latestVersion()); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we set member epoch to 5 above, should we also pass in 5 in here for the commit? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Well, with the relaxed check, it should just be smaller I guess, but larger or equals than the assignment epoch. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, exactly. |
||
|
|
||
| // 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); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We do not allow removing the topology, so I think this may almost impossible. We'd have to recreate the group of the same name, and get the same member ID back to reach this point. If that would ever happen, I think fencing the member would be okay.