lucasbru commented on code in PR #21642:
URL: https://github.com/apache/kafka/pull/21642#discussion_r2911141650


##########
core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala:
##########
@@ -279,6 +284,182 @@ class GroupCoordinatorIntegrationTest(cluster: 
ClusterInstance) {
     }
   }
 
+  @ClusterTest(
+    types = Array(Type.KRAFT),
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1")
+    )
+  )
+  def 
testCoordinatorFailoverAfterCompactingPartitionWithUpgradedConsumerGroupAndTombstoneRemoved():
 Unit = {
+    withAdmin { admin =>
+      TestUtils.createTopicWithAdminRaw(
+        admin = admin,
+        topic = "foo",
+        numPartitions = 3
+      )
+
+      // Create a classic group with one member and commit offsets.
+      withConsumer(groupId = "grp4", groupProtocol = GroupProtocol.CLASSIC, 
enableAutoCommit = false) { consumer =>

Review Comment:
   Good idea, added two tests: 
`testCoordinatorFailoverAfterCompactingPartitionWithUpgradedSimpleConsumerGroup`
 and 
`testCoordinatorFailoverAfterCompactingPartitionWithUpgradedSimpleStreamsGroup`.
 These create a simple classic group by committing offsets directly (without 
subscribing), then upgrade to consumer/streams protocol. Since simple classic 
groups have no GroupMetadata records, there are no tombstones — the offset 
commit records always survive compaction, so a single compaction pass suffices.



##########
core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala:
##########
@@ -331,12 +512,34 @@ class GroupCoordinatorIntegrationTest(cluster: 
ClusterInstance) {
     }
   }
 
+  private def writeOneOffsetCommit(): Unit = {
+    // Write a single offset commit to create dirty data past the cleaner
+    // checkpoint so the cleaner will re-process previously compacted
+    // segments on the next compaction pass.
+    withConsumer(groupId = "compaction-trigger", groupProtocol = 
GroupProtocol.CLASSIC, enableAutoCommit = false) { consumer =>
+      val tp = new TopicPartition("foo", 0)
+      consumer.assign(java.util.List.of(tp))
+      consumer.commitSync(java.util.Map.of(tp, new OffsetAndMetadata(0)))
+    }
+  }
+
+  private def configureDeleteRetention(): Unit = {
+    withAdmin { admin =>
+      val resource = new ConfigResource(ConfigResource.Type.TOPIC, 
"__consumer_offsets")
+      admin.incrementalAlterConfigs(java.util.Map.of(resource, 
java.util.List.of(
+        new AlterConfigOp(new ConfigEntry("delete.retention.ms", "0"), 
AlterConfigOp.OpType.SET),
+        new AlterConfigOp(new ConfigEntry("min.cleanable.dirty.ratio", "0.0"), 
AlterConfigOp.OpType.SET)

Review Comment:
   Done, replaced with `TopicConfig.DELETE_RETENTION_MS_CONFIG`, 
`TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG`, and 
`Topic.GROUP_METADATA_TOPIC_NAME`.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to