dajac commented on code in PR #14800:
URL: https://github.com/apache/kafka/pull/14800#discussion_r1401815563


##########
clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java:
##########
@@ -37,6 +37,12 @@ public JoinGroupResponse(JoinGroupResponseData data, short 
version) {
         if (version < 7 && data.protocolName() == null) {
             data.setProtocolName("");
         }
+
+        // If nullable string for the protocol name is supported,
+        // we set empty string to be null to ensure compliance.
+        if (version >= 7 && data.protocolName() != null && 
data.protocolName().isEmpty()) {
+            data.setProtocolName(null);
+        }

Review Comment:
   Should we add a unit test in JoinGroupResponseTest to cover this change?



##########
core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala:
##########
@@ -0,0 +1,283 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.test.ClusterInstance
+import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, 
ClusterTestDefaults, Type}
+import kafka.test.junit.ClusterTestExtensions
+import kafka.utils.TestUtils
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
+import org.apache.kafka.common.message.SyncGroupRequestData
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.junit.jupiter.api.{Tag, Timeout}
+import org.junit.jupiter.api.extension.ExtendWith
+
+import java.util.Collections
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future}
+
+@Timeout(120)
+@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+@ClusterTestDefaults(clusterType = Type.KRAFT, brokers = 1)
+@Tag("integration")
+class SyncGroupRequestTest(cluster: ClusterInstance) extends 
GroupCoordinatorBaseRequestTest(cluster) {
+  @ClusterTest(serverProperties = Array(
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"false"),
+    new ClusterConfigProperty(key = "group.coordinator.new.enable", value = 
"true"),
+    new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+    new ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1")
+  ))
+  def testSyncGroupWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit 
= {
+    testSyncGroup()
+  }
+
+  @ClusterTest(clusterType = Type.ALL, serverProperties = Array(
+    new ClusterConfigProperty(key = "unstable.api.versions.enable", value = 
"false"),
+    new ClusterConfigProperty(key = "group.coordinator.new.enable", value = 
"false"),
+    new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+    new ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1")
+  ))
+  def testSyncGroupWithOldConsumerGroupProtocolAndOldGroupCoordinator(): Unit 
= {
+    testSyncGroup()
+  }
+
+  private def testSyncGroup(): Unit = {
+    // Creates the __consumer_offsets topics because it won't be created 
automatically
+    // in this test because it does not use FindCoordinator API.
+    createOffsetsTopic()
+
+    // Create the topic.
+    createTopic(
+      topic = "foo",
+      numPartitions = 3
+    )
+
+    for (version <- 0 to 
ApiKeys.SYNC_GROUP.latestVersion(isUnstableApiEnabled)) {
+      // Sync with unknown group id.
+      syncGroupWithOldProtocol(
+        groupId = "grp-unknown",
+        memberId = "member-id",
+        generationId = -1,
+        expectedProtocolType = null,
+        expectedProtocolName = null,
+        expectedError = Errors.UNKNOWN_MEMBER_ID,
+        version = version.toShort
+      )
+
+      val metadata = ConsumerProtocol.serializeSubscription(
+        new 
ConsumerPartitionAssignor.Subscription(Collections.singletonList("foo"))
+      ).array
+
+      // Join a dynamic member without member id.
+      // Prior to JoinGroup version 4, a new member is immediately added if it 
sends a join group request with UNKNOWN_MEMBER_ID.
+      val joinLeaderResponseData = sendJoinRequest(
+        groupId = "grp",
+        metadata = metadata
+      )
+      val leaderMemberId = joinLeaderResponseData.memberId
+
+      // Rejoin the group with the member id.
+      sendJoinRequest(
+        groupId = "grp",
+        memberId = leaderMemberId,
+        metadata = metadata
+      )
+
+      if (version >= 5) {
+        // Sync the leader with unmatched protocolName.
+        syncGroupWithOldProtocol(
+          groupId = "grp",
+          memberId = leaderMemberId,
+          generationId = 1,
+          protocolName = "unmatched",
+          assignments = List(new 
SyncGroupRequestData.SyncGroupRequestAssignment()
+            .setMemberId(leaderMemberId)
+            .setAssignment(Array[Byte](1))
+          ),
+          expectedProtocolType = null,
+          expectedProtocolName = null,
+          expectedError = Errors.INCONSISTENT_GROUP_PROTOCOL,
+          version = version.toShort
+        )
+
+        // Sync the leader with unmatched protocolType.
+        syncGroupWithOldProtocol(
+          groupId = "grp",
+          memberId = leaderMemberId,
+          generationId = 1,
+          protocolType = "unmatched",
+          assignments = List(new 
SyncGroupRequestData.SyncGroupRequestAssignment()
+            .setMemberId(leaderMemberId)
+            .setAssignment(Array[Byte](1))
+          ),
+          expectedProtocolType = null,
+          expectedProtocolName = null,
+          expectedError = Errors.INCONSISTENT_GROUP_PROTOCOL,
+          version = version.toShort
+        )
+      }
+
+      // Sync with unknown member id.
+      syncGroupWithOldProtocol(
+        groupId = "grp",
+        memberId = "member-id-unknown",
+        generationId = -1,
+        expectedProtocolType = null,
+        expectedProtocolName = null,
+        expectedError = Errors.UNKNOWN_MEMBER_ID,
+        version = version.toShort
+      )
+
+      // Sync with illegal generation id.
+      syncGroupWithOldProtocol(
+        groupId = "grp",
+        memberId = leaderMemberId,
+        generationId = 2,
+        expectedProtocolType = null,
+        expectedProtocolName = null,
+        expectedError = Errors.ILLEGAL_GENERATION,
+        version = version.toShort
+      )
+
+      // Sync the leader with empty protocolType and protocolName.
+      syncGroupWithOldProtocol(
+        groupId = "grp",
+        memberId = leaderMemberId,
+        generationId = 1,
+        assignments = List(new 
SyncGroupRequestData.SyncGroupRequestAssignment()
+          .setMemberId(leaderMemberId)
+          .setAssignment(Array[Byte](1))
+        ),
+        expectedAssignment = Array[Byte](1),
+        version = version.toShort
+      )
+
+      // Sync the leader with matched protocolType and protocolName.

Review Comment:
   So how is it different from L158? 



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to