Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,8 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {

def createStreamsGroup[K, V](configOverrides: Properties = new Properties,
configsToRemove: List[String] = List(),
inputTopic: String,
inputTopics: Set[String],
changelogTopics: Set[String] = Set(),
streamsGroupId: String): AsyncKafkaConsumer[K, V] = {
val props = new Properties()
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
Expand All @@ -255,10 +256,10 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
Optional.empty(),
util.Map.of(
"subtopology-0", new StreamsRebalanceData.Subtopology(
util.Set.of(inputTopic),
inputTopics.asJava,
util.Set.of(),
util.Map.of(),
util.Map.of(inputTopic + "-store-changelog", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.empty(), util.Map.of())),
changelogTopics.map(c => (c, new StreamsRebalanceData.TopicInfo(Optional.empty(), Optional.empty(), util.Map.of()))).toMap.asJava,
util.Set.of()
)),
Map.empty[String, String].asJava
Expand All @@ -270,7 +271,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
configOverrides = props,
streamsRebalanceData = streamsRebalanceData
)
consumer.subscribe(util.Set.of(inputTopic),
consumer.subscribe(inputTopics.asJava,
new StreamsRebalanceListener {
override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Unit = ()
override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Unit = ()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2318,7 +2318,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}
}


/**
* Test the consumer group APIs for member removal.
*/
Expand Down Expand Up @@ -2601,7 +2600,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
val shareGroup = createShareConsumer(configOverrides = shareGroupConfig)

val streamsGroup = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId
)

Expand Down Expand Up @@ -4426,7 +4426,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
prepareRecords(testTopicName)

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId
)
streams.poll(JDuration.ofMillis(500L))
Expand All @@ -4436,7 +4437,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
val firstGroup = client.listGroups().all().get().stream()
.filter(g => g.groupId() == streamsGroupId).findFirst().orElse(null)
firstGroup.groupState().orElse(null) == GroupState.STABLE && firstGroup.groupId() == streamsGroupId
}, "Stream group not stable yet")
}, "Streams group did not transition to STABLE before timeout")

// Verify the describe call works correctly
val describedGroups = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get()
Expand Down Expand Up @@ -4472,7 +4473,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
client = Admin.create(config)

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId
)
streams.poll(JDuration.ofMillis(500L))
Expand All @@ -4482,7 +4484,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
val firstGroup = client.listGroups().all().get().stream()
.filter(g => g.groupId() == streamsGroupId).findFirst().orElse(null)
firstGroup.groupState().orElse(null) == GroupState.NOT_READY && firstGroup.groupId() == streamsGroupId
}, "Stream group not NOT_READY yet")
}, "Streams group did not transition to NOT_READY before timeout")

// Verify the describe call works correctly
val describedGroups = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get()
Expand All @@ -4504,6 +4506,55 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}
}

@Test
def testDescribeStreamsGroupsForStatelessTopology(): Unit = {
val streamsGroupId = "stream_group_id"
val testTopicName = "test_topic"
val testNumPartitions = 1

val config = createConfig
client = Admin.create(config)

prepareTopics(List(testTopicName), testNumPartitions)
prepareRecords(testTopicName)

val streams = createStreamsGroup(
inputTopics = Set(testTopicName),
streamsGroupId = streamsGroupId
)
streams.poll(JDuration.ofMillis(500L))

try {
TestUtils.waitUntilTrue(() => {
val firstGroup = client.listGroups().all().get().stream().findFirst().orElse(null)
firstGroup.groupState().orElse(null) == GroupState.STABLE && firstGroup.groupId() == streamsGroupId
}, "Streams group did not transition to STABLE before timeout")

// Verify the describe call works correctly
val describedGroups = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get()
val group = describedGroups.get(streamsGroupId)
assertNotNull(group)
assertEquals(streamsGroupId, group.groupId())
assertFalse(group.members().isEmpty)
assertNotNull(group.subtopologies())
assertFalse(group.subtopologies().isEmpty)

// Verify the topology contains the expected source and sink topics
val subtopologies = group.subtopologies().asScala
assertTrue(subtopologies.exists(subtopology =>
subtopology.sourceTopics().contains(testTopicName)))

// Test describing a non-existing group
val nonExistingGroup = "non_existing_stream_group"
val describedNonExistingGroupResponse = client.describeStreamsGroups(util.List.of(nonExistingGroup))
assertFutureThrows(classOf[GroupIdNotFoundException], describedNonExistingGroupResponse.all())

} finally {
Utils.closeQuietly(streams, "streams")
Utils.closeQuietly(client, "adminClient")
}
}

@Test
def testDeleteStreamsGroups(): Unit = {
val testTopicName = "test_topic"
Expand All @@ -4526,7 +4577,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
val streamsGroupId = s"stream_group_id_$i"

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId,
)
streams.poll(JDuration.ofMillis(500L))
Expand Down Expand Up @@ -4609,7 +4661,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId,
)

Expand All @@ -4625,7 +4678,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
TestUtils.waitUntilTrue(() => {
val firstGroup = client.listGroups().all().get().stream().findFirst().orElse(null)
firstGroup.groupState().orElse(null) == GroupState.STABLE && firstGroup.groupId() == streamsGroupId
}, "Stream group not stable yet")
}, "Streams group did not transition to STABLE before timeout")

val allTopicPartitions = client.listStreamsGroupOffsets(
util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec())
Expand Down Expand Up @@ -4669,7 +4722,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId,
)

Expand Down Expand Up @@ -4746,7 +4800,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}

val streams = createStreamsGroup(
inputTopic = testTopicName,
inputTopics = Set(testTopicName),
changelogTopics = Set(testTopicName + "-changelog"),
streamsGroupId = streamsGroupId,
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,9 +250,9 @@
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.convertToStreamsGroupTopologyRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupMetadataRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord;
import static org.apache.kafka.coordinator.group.streams.StreamsGroupMember.hasAssignedTasksChanged;
Expand Down Expand Up @@ -1960,19 +1960,26 @@ private CoordinatorResult<StreamsGroupHeartbeatResult, CoordinatorRecord> stream
updatedConfiguredTopology = group.configuredTopology().get();
}

// 3b. If the topology is validated, persist the fact that it is validated.
int validatedTopologyEpoch = -1;
if (updatedConfiguredTopology.isReady()) {
validatedTopologyEpoch = updatedTopology.topologyEpoch();
SortedMap<String, ConfiguredSubtopology> subtopologySortedMap = updatedConfiguredTopology.subtopologies().get();
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedActiveTasks);
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedStandbyTasks);
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedWarmupTasks);
}
// We validated a topology that was not validated before, so bump the group epoch as we may have to reassign tasks.
if (validatedTopologyEpoch != group.validatedTopologyEpoch()) {
bumpGroupEpoch = true;
}

// Actually bump the group epoch
int groupEpoch = group.groupEpoch();
if (bumpGroupEpoch) {
groupEpoch += 1;
records.add(newStreamsGroupEpochRecord(groupId, groupEpoch, metadataHash));
log.info("[GroupId {}][MemberId {}] Bumped streams group epoch to {} with metadata hash {}.", groupId, memberId, groupEpoch, metadataHash);
records.add(newStreamsGroupMetadataRecord(groupId, groupEpoch, metadataHash, validatedTopologyEpoch));
log.info("[GroupId {}][MemberId {}] Bumped streams group epoch to {} with metadata hash {} and validated topic epoch {}.", groupId, memberId, groupEpoch, metadataHash, validatedTopologyEpoch);
metrics.record(STREAMS_GROUP_REBALANCES_SENSOR_NAME);
group.setMetadataRefreshDeadline(currentTimeMs + METADATA_REFRESH_INTERVAL_MS, groupEpoch);
}
Expand Down Expand Up @@ -4245,7 +4252,7 @@ private <T> CoordinatorResult<T, CoordinatorRecord> streamsGroupFenceMember(

// We bump the group epoch.
int groupEpoch = group.groupEpoch() + 1;
records.add(newStreamsGroupEpochRecord(group.groupId(), groupEpoch, 0));
records.add(newStreamsGroupMetadataRecord(group.groupId(), groupEpoch, group.metadataHash(), group.validatedTopologyEpoch()));
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There was a bug here - every time we'd fence a member, we'd reset the metadata hash to 0. We don't need to do that, otherwise we'll need to recompute the metadata hash on the next heartbeat and bump the group epoch again.


cancelTimers(group.groupId(), member.memberId());

Expand Down Expand Up @@ -5365,6 +5372,7 @@ public void replay(
StreamsGroup streamsGroup = getOrMaybeCreatePersistedStreamsGroup(groupId, true);
streamsGroup.setGroupEpoch(value.epoch());
streamsGroup.setMetadataHash(value.metadataHash());
streamsGroup.setValidatedTopologyEpoch(value.validatedTopologyEpoch());
} else {
StreamsGroup streamsGroup;
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,10 +96,11 @@ public static CoordinatorRecord newStreamsGroupMemberTombstoneRecord(
);
}

public static CoordinatorRecord newStreamsGroupEpochRecord(
public static CoordinatorRecord newStreamsGroupMetadataRecord(
String groupId,
int newGroupEpoch,
long metadataHash
long metadataHash,
int validatedTopologyEpoch
) {
Objects.requireNonNull(groupId, "groupId should not be null here");

Expand All @@ -109,7 +110,8 @@ public static CoordinatorRecord newStreamsGroupEpochRecord(
new ApiMessageAndVersion(
new StreamsGroupMetadataValue()
.setEpoch(newGroupEpoch)
.setMetadataHash(metadataHash),
.setMetadataHash(metadataHash)
.setValidatedTopologyEpoch(validatedTopologyEpoch),
(short) 0
)
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,9 +148,9 @@ public static class DeadlineAndEpoch {
private final TimelineHashMap<String, String> staticMembers;

/**
* The metadata associated with each subscribed topic name.
* The topology epoch for which the subscribed topics identified by metadataHash are validated.
*/
private final TimelineHashMap<String, TopicMetadata> partitionMetadata;
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the partition metadata field was not used anymore and wasn't removed in a previous PR

protected final TimelineInteger validatedTopologyEpoch;

/**
* The metadata hash which is computed based on the all subscribed topics.
Expand Down Expand Up @@ -224,7 +224,7 @@ public StreamsGroup(
this.groupEpoch = new TimelineInteger(snapshotRegistry);
this.members = new TimelineHashMap<>(snapshotRegistry, 0);
this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
this.partitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
this.validatedTopologyEpoch = new TimelineInteger(snapshotRegistry);
this.metadataHash = new TimelineLong(snapshotRegistry);
this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry);
this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
Expand Down Expand Up @@ -284,7 +284,6 @@ public void setTopology(StreamsTopology topology) {

public void setConfiguredTopology(ConfiguredTopology configuredTopology) {
this.configuredTopology.set(Optional.ofNullable(configuredTopology));
maybeUpdateGroupState();
}

/**
Expand Down Expand Up @@ -600,6 +599,23 @@ public void setMetadataHash(long metadataHash) {
this.metadataHash.set(metadataHash);
}

/**
* @return The validated topology epoch.
*/
public int validatedTopologyEpoch() {
return validatedTopologyEpoch.get();
}

/**
* Updates the validated topology epoch.
*
* @param validatedTopologyEpoch The validated topology epoch
*/
public void setValidatedTopologyEpoch(int validatedTopologyEpoch) {
this.validatedTopologyEpoch.set(validatedTopologyEpoch);
maybeUpdateGroupState();
}

/**
* Computes the metadata hash based on the current topology and the current metadata image.
*
Expand Down Expand Up @@ -837,7 +853,7 @@ private void maybeUpdateGroupState() {
if (members.isEmpty()) {
newState = EMPTY;
clearShutdownRequestMemberId();
} else if (topology().isEmpty() || configuredTopology().isEmpty() || !configuredTopology().get().isReady()) {
} else if (topology().filter(x -> x.topologyEpoch() == validatedTopologyEpoch.get()).isEmpty()) {
newState = NOT_READY;
} else if (groupEpoch.get() > targetAssignmentEpoch.get()) {
newState = ASSIGNING;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
{ "name": "Epoch", "versions": "0+", "type": "int32",
"about": "The group epoch." },
{ "name": "MetadataHash", "versions": "0+", "type": "int64",
"about": "The hash of all topics in the group." }
"about": "The hash of all topics in the group." },
{ "name": "ValidatedTopologyEpoch", "versions": "0+", "taggedVersions": "0+", "tag": 0, "default": -1, "type": "int32",
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We add the field as a tagged field, so if we are compatible without bumping the version.

If a new kafka reads a record without ValidatedTopologyEpoch, it will assume the default -1 and revalidate the topology.

If an old kafka reads a record with ValidatedTopologyEpoch, it will be ignored.

"about": "The topology epoch whose topics where validated to be present in a valid configuration in the metadata." }
]
}
Loading
Loading