-
Notifications
You must be signed in to change notification settings - Fork 14.6k
KAFKA-19550: Integration test for Streams-related Admin APIs [1/N] #20244
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 26 commits
6c4fba4
c272552
503483c
71e0ab0
6524912
d923d01
4196434
a3da136
316a417
a50ed7e
ab67b5b
ad59b00
03f64e4
f33fddb
885e626
d9246de
069a406
1e9a87a
ac83e37
7e6afe0
7f75f03
6464811
926e7e4
00f271a
c6a42bd
a1f1e88
aed6c1d
04433c7
be27bd8
b996f95
7b036a7
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 |
---|---|---|
|
@@ -34,6 +34,7 @@ import org.apache.kafka.clients.HostResolver | |
import org.apache.kafka.clients.admin.AlterConfigOp.OpType | ||
import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource | ||
import org.apache.kafka.clients.admin._ | ||
import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer | ||
import org.apache.kafka.clients.consumer.{CommitFailedException, Consumer, ConsumerConfig, GroupProtocol, KafkaConsumer, OffsetAndMetadata, ShareConsumer} | ||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} | ||
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} | ||
|
@@ -2573,8 +2574,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
val consumerGroupId = "consumer_group_id" | ||
val shareGroupId = "share_group_id" | ||
val simpleGroupId = "simple_group_id" | ||
val streamsGroupId = "streams_group_id" | ||
val testTopicName = "test_topic" | ||
|
||
consumerConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name) | ||
val classicGroupConfig = new Properties(consumerConfig) | ||
classicGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, classicGroupId) | ||
|
@@ -2589,13 +2590,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
shareGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, shareGroupId) | ||
val shareGroup = createShareConsumer(configOverrides = shareGroupConfig) | ||
|
||
val streamsGroup = createStreamsGroup( | ||
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. The consumer may create see https://github.com/apache/kafka/actions/runs/17495296037/job/49695550299?pr=20472 @lucliu1108 WDYT? Perhaps we could create the topic first? 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. That makes sense. Thanks for pointing out! I'll make a patch for that @chia7712 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. |
||
inputTopic = testTopicName, | ||
streamsGroupId = streamsGroupId | ||
) | ||
|
||
val config = createConfig | ||
client = Admin.create(config) | ||
try { | ||
lucliu1108 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
client.createTopics(util.Set.of( | ||
new NewTopic(testTopicName, 1, 1.toShort) | ||
)).all().get() | ||
waitForTopics(client, List(testTopicName), List()) | ||
val topicPartition = new TopicPartition(testTopicName, 0) | ||
|
||
classicGroup.subscribe(util.Set.of(testTopicName)) | ||
|
@@ -2604,6 +2606,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
consumerGroup.poll(JDuration.ofMillis(1000)) | ||
shareGroup.subscribe(util.Set.of(testTopicName)) | ||
shareGroup.poll(JDuration.ofMillis(1000)) | ||
streamsGroup.poll(JDuration.ofMillis(1000)) | ||
|
||
val alterConsumerGroupOffsetsResult = client.alterConsumerGroupOffsets(simpleGroupId, | ||
util.Map.of(topicPartition, new OffsetAndMetadata(0L))) | ||
|
@@ -2612,18 +2615,27 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
|
||
TestUtils.waitUntilTrue(() => { | ||
val groups = client.listGroups().all().get() | ||
groups.size() == 4 | ||
groups.size() == 5 | ||
}, "Expected to find all groups") | ||
|
||
val classicGroupListing = new GroupListing(classicGroupId, Optional.of(GroupType.CLASSIC), "consumer", Optional.of(GroupState.STABLE)) | ||
val consumerGroupListing = new GroupListing(consumerGroupId, Optional.of(GroupType.CONSUMER), "consumer", Optional.of(GroupState.STABLE)) | ||
val shareGroupListing = new GroupListing(shareGroupId, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.STABLE)) | ||
val simpleGroupListing = new GroupListing(simpleGroupId, Optional.of(GroupType.CLASSIC), "", Optional.of(GroupState.EMPTY)) | ||
// Streams group could either be in STABLE or NOT_READY state | ||
val streamsGroupListingStable = new GroupListing(streamsGroupId, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)) | ||
val streamsGroupListingNotReady = new GroupListing(streamsGroupId, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.NOT_READY)) | ||
|
||
var listGroupsResult = client.listGroups() | ||
assertTrue(listGroupsResult.errors().get().isEmpty) | ||
assertEquals(Set(classicGroupListing, simpleGroupListing, consumerGroupListing, shareGroupListing), listGroupsResult.all().get().asScala.toSet) | ||
assertEquals(Set(classicGroupListing, simpleGroupListing, consumerGroupListing, shareGroupListing), listGroupsResult.valid().get().asScala.toSet) | ||
|
||
val expectedStreamListings = Set(streamsGroupListingStable, streamsGroupListingNotReady) | ||
val expectedListings = Set(classicGroupListing, simpleGroupListing, consumerGroupListing, shareGroupListing) | ||
val actualListings = listGroupsResult.all().get().asScala.toSet | ||
|
||
// Check that actualListings contains all expectedListings and one of the streams listings | ||
assertTrue(expectedListings.subsetOf(actualListings)) | ||
assertTrue(actualListings.exists(expectedStreamListings.contains)) | ||
|
||
listGroupsResult = client.listGroups(new ListGroupsOptions().withTypes(util.Set.of(GroupType.CLASSIC))) | ||
assertTrue(listGroupsResult.errors().get().isEmpty) | ||
|
@@ -2639,11 +2651,20 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
assertTrue(listGroupsResult.errors().get().isEmpty) | ||
assertEquals(Set(shareGroupListing), listGroupsResult.all().get().asScala.toSet) | ||
assertEquals(Set(shareGroupListing), listGroupsResult.valid().get().asScala.toSet) | ||
|
||
listGroupsResult = client.listGroups(new ListGroupsOptions().withTypes(util.Set.of(GroupType.STREAMS))) | ||
assertTrue(listGroupsResult.errors().get().isEmpty) | ||
assertTrue(listGroupsResult.all().get().asScala.toSet.equals(Set(streamsGroupListingStable)) || | ||
listGroupsResult.all().get().asScala.toSet.equals(Set(streamsGroupListingNotReady))) | ||
assertTrue(listGroupsResult.valid().get().asScala.toSet.equals(Set(streamsGroupListingStable)) || | ||
listGroupsResult.valid().get().asScala.toSet.equals(Set(streamsGroupListingNotReady))) | ||
|
||
} finally { | ||
Utils.closeQuietly(classicGroup, "classicGroup") | ||
Utils.closeQuietly(consumerGroup, "consumerGroup") | ||
Utils.closeQuietly(shareGroup, "shareGroup") | ||
Utils.closeQuietly(client, "adminClient") | ||
Utils.closeQuietly(streamsGroup, "streamsGroup") | ||
} | ||
} | ||
|
||
|
@@ -3603,7 +3624,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, "0") | ||
client.incrementalAlterConfigs(util.Map.of(broker0Resource, | ||
util.List.of(new AlterConfigOp(new ConfigEntry(QuotaConfig.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, "123"), | ||
AlterConfigOp.OpType.SET), | ||
AlterConfigOp.OpType.SET), | ||
lucasbru marked this conversation as resolved.
Show resolved
Hide resolved
|
||
new AlterConfigOp(new ConfigEntry(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_RATE_CONFIG, "456"), | ||
AlterConfigOp.OpType.SET) | ||
))).all().get() | ||
|
@@ -3809,7 +3830,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
val longTopicName = String.join("", Collections.nCopies(249, "x")) | ||
val invalidTopicName = String.join("", Collections.nCopies(250, "x")) | ||
val newTopics2 = util.List.of(new NewTopic(invalidTopicName, 3, 3.toShort), | ||
new NewTopic(longTopicName, 3, 3.toShort)) | ||
new NewTopic(longTopicName, 3, 3.toShort)) | ||
lucasbru marked this conversation as resolved.
Show resolved
Hide resolved
|
||
val results = client.createTopics(newTopics2).values() | ||
assertTrue(results.containsKey(longTopicName)) | ||
results.get(longTopicName).get() | ||
|
@@ -4363,6 +4384,136 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { | |
} | ||
} | ||
} | ||
|
||
@Test | ||
def testDescribeStreamsGroups(): 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( | ||
inputTopic = 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 | ||
lucliu1108 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
}, "Stream group not stable yet") | ||
|
||
lucasbru marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// 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" | ||
val testNumPartitions = 3 | ||
val testNumStreamsGroup = 3 | ||
|
||
val targetDeletedGroups = util.List.of("stream_group_id_2", "stream_group_id_3") | ||
val targetRemainingGroups = util.List.of("stream_group_id_1") | ||
|
||
val config = createConfig | ||
client = Admin.create(config) | ||
|
||
prepareTopics(List(testTopicName), testNumPartitions) | ||
prepareRecords(testTopicName) | ||
|
||
val streamsList = scala.collection.mutable.ListBuffer[(String, AsyncKafkaConsumer[_,_])]() | ||
|
||
try { | ||
for (i <- 1 to testNumStreamsGroup) { | ||
val streamsGroupId = s"stream_group_id_$i" | ||
|
||
val streams = createStreamsGroup( | ||
inputTopic = testTopicName, | ||
streamsGroupId = streamsGroupId, | ||
) | ||
streams.poll(JDuration.ofMillis(500L)) | ||
streamsList += ((streamsGroupId, streams)) | ||
} | ||
|
||
TestUtils.waitUntilTrue(() => { | ||
val groups = client.listGroups().all().get() | ||
groups.stream() | ||
.anyMatch(g => g.groupId().startsWith("stream_group_id_")) && testNumStreamsGroup == groups.size() | ||
}, "Streams groups not ready to delete yet") | ||
|
||
// Test deletion of non-empty existing groups | ||
var deleteStreamsGroupResult = client.deleteStreamsGroups(targetDeletedGroups) | ||
assertFutureThrows(classOf[GroupNotEmptyException], deleteStreamsGroupResult.all()) | ||
assertEquals(2, deleteStreamsGroupResult.deletedGroups().size()) | ||
|
||
// Stop and clean up the streams for the groups that are going to be deleted | ||
streamsList | ||
.filter { case (groupId, _) => targetDeletedGroups.contains(groupId) } | ||
.foreach { case (_, streams) => | ||
streams.close() | ||
} | ||
|
||
val listTopicResult = client.listTopics() | ||
assertEquals(2, listTopicResult.names().get().size()) | ||
|
||
// Test deletion of emptied existing streams groups | ||
deleteStreamsGroupResult = client.deleteStreamsGroups(targetDeletedGroups) | ||
assertEquals(2, deleteStreamsGroupResult.deletedGroups().size()) | ||
|
||
// Wait for the deleted groups to be removed | ||
TestUtils.waitUntilTrue(() => { | ||
val groupIds = client.listGroups().all().get().asScala.map(_.groupId()).toSet | ||
targetDeletedGroups.asScala.forall(id => !groupIds.contains(id)) | ||
}, "Deleted groups not yet deleted") | ||
|
||
// Verify that the deleted groups are no longer present | ||
val remainingGroups = client.listGroups().all().get() | ||
assertEquals(targetRemainingGroups.size(), remainingGroups.size()) | ||
remainingGroups.stream().forEach(g => { | ||
assertTrue(targetRemainingGroups.contains(g.groupId())) | ||
}) | ||
|
||
// Test deletion of a non-existing group | ||
val nonExistingGroup = "non_existing_stream_group" | ||
val deleteNonExistingGroupResult = client.deleteStreamsGroups(util.List.of(nonExistingGroup)) | ||
assertFutureThrows(classOf[GroupIdNotFoundException], deleteNonExistingGroupResult.all()) | ||
assertEquals(deleteNonExistingGroupResult.deletedGroups().size(), 1) | ||
|
||
} finally{ | ||
streamsList.foreach { case (_, streams) => | ||
streams.close() | ||
} | ||
Utils.closeQuietly(client, "adminClient") | ||
} | ||
} | ||
} | ||
|
||
object PlaintextAdminIntegrationTest { | ||
|
Uh oh!
There was an error while loading. Please reload this page.