Copilot commented on code in PR #20757:
URL: https://github.com/apache/kafka/pull/20757#discussion_r2454233687


##########
core/src/test/scala/unit/kafka/server/StreamsGroupHeartbeatRequestTest.scala:
##########
@@ -0,0 +1,554 @@
+package kafka.server
+
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.message.{StreamsGroupHeartbeatRequestData, 
StreamsGroupHeartbeatResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{StreamsGroupHeartbeatRequest, 
StreamsGroupHeartbeatResponse}
+import org.apache.kafka.common.test.ClusterInstance
+import org.apache.kafka.common.test.api.{ClusterConfigProperty, 
ClusterFeature, ClusterTest, ClusterTestDefaults, Type}
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.common.errors.UnsupportedVersionException
+import org.apache.kafka.server.common.Feature
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, 
assertTrue}
+
+import java.util.Collections
+import scala.jdk.CollectionConverters._
+
+@ClusterTestDefaults(
+  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"),
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"true")
+  )
+)
+class StreamsGroupHeartbeatRequestTest(cluster: ClusterInstance) extends 
GroupCoordinatorBaseRequestTest(cluster) {
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def testStreamsGroupHeartbeatWithInvalidAPIVersion(): Unit = {
+    // Test that invalid API version throws UnsupportedVersionException
+    assertThrows(classOf[UnsupportedVersionException], () => {
+      new StreamsGroupHeartbeatRequest.Builder(
+        new StreamsGroupHeartbeatRequestData()
+      ).build(-1)
+    })
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    ),
+    features = Array(
+      new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
+    )
+  )
+  def testStreamsGroupHeartbeatIsInaccessableWhenDisabledByFeatureConfig(): 
Unit = {
+    // Test with streams.version = 0, the API is disabled at server level
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology)
+    ).build(0)
+    
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer"),
+    )
+  )
+  def 
testStreamsGroupHeartbeatIsInaccessableWhenDisabledByStaticGroupCoordinatorProtocolConfig():
 Unit = {
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology),
+      true  // enableUnstableLastVersion = true
+    ).build(0)
+
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def 
testStreamsGroupHeartbeatIsInaccessibleWhenUnstableLatestVersionNotEnabled(): 
Unit = {
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology),
+      false  // enableUnstableLastVersion = false
+    ).build(0)
+
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.NOT_COORDINATOR.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest
+  def 
tesStreamsGroupHeartbeatIsAccessibleWhenNewGroupCoordinatorIsEnabledTopicNotExistFirst():
 Unit = {

Review Comment:
   Corrected spelling of 'tesStreamsGroupHeartbeat' to 
'testStreamsGroupHeartbeat' in test method name (missing 't').
   ```suggestion
     def 
testStreamsGroupHeartbeatIsAccessibleWhenNewGroupCoordinatorIsEnabledTopicNotExistFirst():
 Unit = {
   ```



##########
core/src/test/scala/unit/kafka/server/StreamsGroupHeartbeatRequestTest.scala:
##########
@@ -0,0 +1,554 @@
+package kafka.server
+
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.message.{StreamsGroupHeartbeatRequestData, 
StreamsGroupHeartbeatResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{StreamsGroupHeartbeatRequest, 
StreamsGroupHeartbeatResponse}
+import org.apache.kafka.common.test.ClusterInstance
+import org.apache.kafka.common.test.api.{ClusterConfigProperty, 
ClusterFeature, ClusterTest, ClusterTestDefaults, Type}
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.common.errors.UnsupportedVersionException
+import org.apache.kafka.server.common.Feature
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, 
assertTrue}
+
+import java.util.Collections
+import scala.jdk.CollectionConverters._
+
+@ClusterTestDefaults(
+  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"),
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"true")
+  )
+)
+class StreamsGroupHeartbeatRequestTest(cluster: ClusterInstance) extends 
GroupCoordinatorBaseRequestTest(cluster) {
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def testStreamsGroupHeartbeatWithInvalidAPIVersion(): Unit = {
+    // Test that invalid API version throws UnsupportedVersionException
+    assertThrows(classOf[UnsupportedVersionException], () => {
+      new StreamsGroupHeartbeatRequest.Builder(
+        new StreamsGroupHeartbeatRequestData()
+      ).build(-1)
+    })
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    ),
+    features = Array(
+      new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
+    )
+  )
+  def testStreamsGroupHeartbeatIsInaccessableWhenDisabledByFeatureConfig(): 
Unit = {

Review Comment:
   Corrected spelling of 'Inaccessable' to 'Inaccessible' in test method name.
   ```suggestion
     def testStreamsGroupHeartbeatIsInaccessibleWhenDisabledByFeatureConfig(): 
Unit = {
   ```



##########
core/src/test/scala/unit/kafka/server/StreamsGroupHeartbeatRequestTest.scala:
##########
@@ -0,0 +1,554 @@
+package kafka.server
+
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.message.{StreamsGroupHeartbeatRequestData, 
StreamsGroupHeartbeatResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{StreamsGroupHeartbeatRequest, 
StreamsGroupHeartbeatResponse}
+import org.apache.kafka.common.test.ClusterInstance
+import org.apache.kafka.common.test.api.{ClusterConfigProperty, 
ClusterFeature, ClusterTest, ClusterTestDefaults, Type}
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.common.errors.UnsupportedVersionException
+import org.apache.kafka.server.common.Feature
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, 
assertTrue}
+
+import java.util.Collections
+import scala.jdk.CollectionConverters._
+
+@ClusterTestDefaults(
+  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"),
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"true")
+  )
+)
+class StreamsGroupHeartbeatRequestTest(cluster: ClusterInstance) extends 
GroupCoordinatorBaseRequestTest(cluster) {
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def testStreamsGroupHeartbeatWithInvalidAPIVersion(): Unit = {
+    // Test that invalid API version throws UnsupportedVersionException
+    assertThrows(classOf[UnsupportedVersionException], () => {
+      new StreamsGroupHeartbeatRequest.Builder(
+        new StreamsGroupHeartbeatRequestData()
+      ).build(-1)
+    })
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    ),
+    features = Array(
+      new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
+    )
+  )
+  def testStreamsGroupHeartbeatIsInaccessableWhenDisabledByFeatureConfig(): 
Unit = {
+    // Test with streams.version = 0, the API is disabled at server level
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology)
+    ).build(0)
+    
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer"),
+    )
+  )
+  def 
testStreamsGroupHeartbeatIsInaccessableWhenDisabledByStaticGroupCoordinatorProtocolConfig():
 Unit = {

Review Comment:
   Corrected spelling of 'Inaccessable' to 'Inaccessible' in test method name.
   ```suggestion
     def 
testStreamsGroupHeartbeatIsInaccessibleWhenDisabledByStaticGroupCoordinatorProtocolConfig():
 Unit = {
   ```



##########
core/src/test/scala/unit/kafka/server/StreamsGroupHeartbeatRequestTest.scala:
##########
@@ -0,0 +1,554 @@
+package kafka.server
+
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.message.{StreamsGroupHeartbeatRequestData, 
StreamsGroupHeartbeatResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{StreamsGroupHeartbeatRequest, 
StreamsGroupHeartbeatResponse}
+import org.apache.kafka.common.test.ClusterInstance
+import org.apache.kafka.common.test.api.{ClusterConfigProperty, 
ClusterFeature, ClusterTest, ClusterTestDefaults, Type}
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.common.errors.UnsupportedVersionException
+import org.apache.kafka.server.common.Feature
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, 
assertTrue}
+
+import java.util.Collections
+import scala.jdk.CollectionConverters._
+
+@ClusterTestDefaults(
+  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"),
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"true")
+  )
+)
+class StreamsGroupHeartbeatRequestTest(cluster: ClusterInstance) extends 
GroupCoordinatorBaseRequestTest(cluster) {
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def testStreamsGroupHeartbeatWithInvalidAPIVersion(): Unit = {
+    // Test that invalid API version throws UnsupportedVersionException
+    assertThrows(classOf[UnsupportedVersionException], () => {
+      new StreamsGroupHeartbeatRequest.Builder(
+        new StreamsGroupHeartbeatRequestData()
+      ).build(-1)
+    })
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    ),
+    features = Array(
+      new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
+    )
+  )
+  def testStreamsGroupHeartbeatIsInaccessableWhenDisabledByFeatureConfig(): 
Unit = {
+    // Test with streams.version = 0, the API is disabled at server level
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology)
+    ).build(0)
+    
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer"),
+    )
+  )
+  def 
testStreamsGroupHeartbeatIsInaccessableWhenDisabledByStaticGroupCoordinatorProtocolConfig():
 Unit = {
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology),
+      true  // enableUnstableLastVersion = true
+    ).build(0)
+
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest(
+    serverProperties = Array(
+      new ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = 
"classic,consumer,streams"),
+    )
+  )
+  def 
testStreamsGroupHeartbeatIsInaccessibleWhenUnstableLatestVersionNotEnabled(): 
Unit = {
+    val topology = new StreamsGroupHeartbeatRequestData.Topology()
+      .setEpoch(1)
+      .setSubtopologies(java.util.Collections.emptyList())
+    
+    val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+      new StreamsGroupHeartbeatRequestData()
+        .setGroupId("test-group")
+        .setMemberId("test-member")
+        .setMemberEpoch(0)
+        .setRebalanceTimeoutMs(1000)
+        .setActiveTasks(java.util.Collections.emptyList())
+        .setStandbyTasks(java.util.Collections.emptyList())
+        .setWarmupTasks(java.util.Collections.emptyList())
+        .setTopology(topology),
+      false  // enableUnstableLastVersion = false
+    ).build(0)
+
+    val streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+    val expectedResponse = new 
StreamsGroupHeartbeatResponseData().setErrorCode(Errors.NOT_COORDINATOR.code())
+    assertEquals(expectedResponse, streamsGroupHeartbeatResponse.data)
+  }
+
+  @ClusterTest
+  def 
tesStreamsGroupHeartbeatIsAccessibleWhenNewGroupCoordinatorIsEnabledTopicNotExistFirst():
 Unit = {
+    val admin = cluster.admin()
+    val memberId = "test-member"
+    val groupId = "test-group"
+    val topicName = "test-topic"
+
+    // Creates the __consumer_offsets topics because it won't be created 
automatically
+    // in this test because it does not use FindCoordinator API.
+    try {
+      TestUtils.createOffsetsTopicWithAdmin(
+        admin = admin,
+        brokers = cluster.brokers.values().asScala.toSeq,
+        controllers = cluster.controllers().values().asScala.toSeq
+      )
+
+      val topology = createMockTopology(topicName)
+
+      val streamsGroupHeartbeatRequest = new 
StreamsGroupHeartbeatRequest.Builder(
+        new StreamsGroupHeartbeatRequestData()
+          .setGroupId(groupId)
+          .setMemberId(memberId)
+          .setMemberEpoch(0)
+          .setRebalanceTimeoutMs(1000)
+          .setActiveTasks(java.util.Collections.emptyList())
+          .setStandbyTasks(java.util.Collections.emptyList())
+          .setWarmupTasks(java.util.Collections.emptyList())
+          .setTopology(topology)
+      ).build(0)
+
+      // Heartbeat when topic does not exist
+      var streamsGroupHeartbeatResponse: StreamsGroupHeartbeatResponse = null
+      TestUtils.waitUntilTrue(() => {
+        streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+        streamsGroupHeartbeatResponse.data.errorCode == Errors.NONE.code()
+      }, "StreamsGroupHeartbeatRequest did not succeed within the timeout 
period.")
+
+      // Verify the response
+      assert(streamsGroupHeartbeatResponse != null, 
"StreamsGroupHeartbeatResponse should not be null")
+      assertEquals(memberId, streamsGroupHeartbeatResponse.data.memberId())
+      assertEquals(1, streamsGroupHeartbeatResponse.data.memberEpoch())
+      val expectedStatus = new StreamsGroupHeartbeatResponseData.Status()
+        .setStatusCode(1)
+        .setStatusDetail(s"Source topics $topicName are missing.")
+      assertEquals(expectedStatus, 
streamsGroupHeartbeatResponse.data.status().get(0))
+
+      // Create topic
+      TestUtils.createTopicWithAdmin(
+        admin = admin,
+        brokers = cluster.brokers.values().asScala.toSeq,
+        controllers = cluster.controllers().values().asScala.toSeq,
+        topic = topicName,
+        numPartitions = 3
+      )
+      // Wait for topic to be available
+      TestUtils.waitUntilTrue(() => {
+        admin.listTopics().names().get().contains(topicName)
+      }, msg = s"Topic $topicName is not available to the group coordinator")
+
+      // Heartbeat after topic is created
+      TestUtils.waitUntilTrue(() => {
+        streamsGroupHeartbeatResponse = 
connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
+        streamsGroupHeartbeatResponse.data.errorCode == Errors.NONE.code()
+      }, "StreamsGroupHeartbeatRequest did not succeed within the timeout 
period.")
+
+      // Active task assignment should be available
+      assert(streamsGroupHeartbeatResponse != null, 
"StreamsGroupHeartbeatResponse should not be null")
+      assertEquals(memberId, streamsGroupHeartbeatResponse.data.memberId())
+      assertEquals(2, streamsGroupHeartbeatResponse.data.memberEpoch())
+      assertEquals(null, streamsGroupHeartbeatResponse.data.status())
+      val expectedActiveTasks = List(
+        new StreamsGroupHeartbeatResponseData.TaskIds()
+          .setSubtopologyId("subtopology-1")
+          .setPartitions(List(0, 1, 2).map(_.asInstanceOf[Integer]).asJava)
+      ).asJava
+      assertEquals(expectedActiveTasks, 
streamsGroupHeartbeatResponse.data.activeTasks())
+    } finally {
+      admin.close()
+    }
+  }
+
+  @ClusterTest
+  def tesStreamsGroupHeartbeatForMultipleMembers(): Unit = {

Review Comment:
   Corrected spelling of 'tesStreamsGroupHeartbeat' to 
'testStreamsGroupHeartbeat' in test method name (missing 't').
   ```suggestion
     def testStreamsGroupHeartbeatForMultipleMembers(): Unit = {
   ```



-- 
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