chia7712 commented on code in PR #16898:
URL: https://github.com/apache/kafka/pull/16898#discussion_r1730343775


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -3225,7 +3228,14 @@ public void replay(
                 .updateWith(value)
                 .build());
         } else {
-            ConsumerGroupMember oldMember = 
consumerGroup.getOrMaybeCreateMember(memberId, false);
+            ConsumerGroupMember oldMember;
+            try {
+                oldMember = consumerGroup.getOrMaybeCreateMember(memberId, 
false);

Review Comment:
   Could you please add `UnknownMemberIdException` to `getOrMaybeCreateMember` 
signature? 



##########
core/src/main/scala/kafka/coordinator/group/CoordinatorLoaderImpl.scala:
##########
@@ -156,17 +164,42 @@ class CoordinatorLoaderImpl[T](
               } else {
                 batch.asScala.foreach { record =>
                   numRecords = numRecords + 1
-                  try {
-                    coordinator.replay(
-                      record.offset(),
-                      batch.producerId,
-                      batch.producerEpoch,
-                      deserializer.deserialize(record.key, record.value)
-                    )
-                  } catch {
-                    case ex: UnknownRecordTypeException =>
-                      warn(s"Unknown record type ${ex.unknownType} while 
loading offsets and group metadata " +
-                        s"from $tp. Ignoring it. It could be a left over from 
an aborted upgrade.")
+
+                  val coordinatorRecordOpt = {
+                    try {
+                      Some(deserializer.deserialize(record.key, record.value))
+                    } catch {
+                      case ex: UnknownRecordTypeException =>
+                        warn(s"Unknown record type ${ex.unknownType} while 
loading offsets and group metadata " +
+                          s"from $tp. Ignoring it. It could be a left over 
from an aborted upgrade.")
+                        None
+                      case ex: RuntimeException =>
+                        val msg = s"Deserializing record $record from $tp 
failed due to: ${ex.getMessage}"
+                        error(s"$msg.")

Review Comment:
   How about `error(msg)`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -3215,7 +3211,14 @@ public void replay(
         String groupId = key.groupId();
         String memberId = key.memberId();
 
-        ConsumerGroup consumerGroup = 
getOrMaybeCreatePersistedConsumerGroup(groupId, value != null);
+        ConsumerGroup consumerGroup;
+        try {
+            consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, 
value != null);
+        } catch (IllegalStateException ex) {
+            // If the group does not exist and a tombstone is replayed, we can 
ignore it.

Review Comment:
   > Is there any other reason we could throw this IllegalStateException
   
   I have the same question. What if the group type is not `CONSUMER`?



##########
core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala:
##########
@@ -12,53 +12,332 @@
  */
 package kafka.api
 
-import kafka.integration.KafkaServerTestHarness
 import kafka.log.UnifiedLog
-import kafka.server.KafkaConfig
+import kafka.test.ClusterInstance
+import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, Type}
+import kafka.test.junit.ClusterTestExtensions
 import kafka.utils.TestUtils
-import org.apache.kafka.clients.consumer.OffsetAndMetadata
-import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription}
+import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, 
OffsetAndMetadata}
+import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaFuture, 
TopicPartition}
 import org.junit.jupiter.api.Assertions._
-import org.junit.jupiter.params.ParameterizedTest
-import org.junit.jupiter.params.provider.ValueSource
 
 import scala.jdk.CollectionConverters._
-import java.util.Properties
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.record.CompressionType
 import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.server.config.ServerConfigs
+import org.junit.jupiter.api.Timeout
+import org.junit.jupiter.api.extension.ExtendWith
 
-class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
-  val offsetsTopicCompressionCodec = CompressionType.GZIP
-  val overridingProps = new Properties()
-  overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 
"1")
-  
overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG,
 offsetsTopicCompressionCodec.id.toString)
+import java.time.Duration
+import java.util.Collections
+import java.util.concurrent.TimeUnit
 
-  override def generateConfigs = TestUtils.createBrokerConfigs(1, 
zkConnectOrNull, enableControlledShutdown = false).map {
-    KafkaConfig.fromProps(_, overridingProps)
+@Timeout(120)
+@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
+
+  @ClusterTest(
+    types = Array(Type.KRAFT, Type.ZK),
+    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 = 
GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG, value = "1"),
+      new ClusterConfigProperty(key = 
ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value = "false"),
+    )
+  )
+  def testGroupCoordinatorPropagatesOffsetsTopicCompressionCodec(): Unit = {
+    val logManager = cluster.brokers().asScala.head._2.logManager
+    val consumer = TestUtils.createConsumer(cluster.bootstrapServers())

Review Comment:
   why not using `withConsumer` ?



##########
core/src/main/scala/kafka/coordinator/group/CoordinatorLoaderImpl.scala:
##########
@@ -156,17 +164,42 @@ class CoordinatorLoaderImpl[T](
               } else {
                 batch.asScala.foreach { record =>
                   numRecords = numRecords + 1
-                  try {
-                    coordinator.replay(
-                      record.offset(),
-                      batch.producerId,
-                      batch.producerEpoch,
-                      deserializer.deserialize(record.key, record.value)
-                    )
-                  } catch {
-                    case ex: UnknownRecordTypeException =>
-                      warn(s"Unknown record type ${ex.unknownType} while 
loading offsets and group metadata " +
-                        s"from $tp. Ignoring it. It could be a left over from 
an aborted upgrade.")
+
+                  val coordinatorRecordOpt = {
+                    try {
+                      Some(deserializer.deserialize(record.key, record.value))
+                    } catch {
+                      case ex: UnknownRecordTypeException =>
+                        warn(s"Unknown record type ${ex.unknownType} while 
loading offsets and group metadata " +
+                          s"from $tp. Ignoring it. It could be a left over 
from an aborted upgrade.")
+                        None
+                      case ex: RuntimeException =>
+                        val msg = s"Deserializing record $record from $tp 
failed due to: ${ex.getMessage}"
+                        error(s"$msg.")
+                        throw new RuntimeException(msg, ex)
+                    }
+                  }
+
+                  coordinatorRecordOpt.foreach { coordinatorRecord =>
+                    try {
+                      if (isTraceEnabled) {
+                        trace(s"Replaying record $coordinatorRecord from $tp 
at offset ${record.offset()} " +
+                          s"with producer id ${batch.producerId} and producer 
epoch ${batch.producerEpoch}.")
+                      }
+                      coordinator.replay(
+                        record.offset(),
+                        batch.producerId,
+                        batch.producerEpoch,
+                        coordinatorRecord
+                      )
+                    } catch {
+                      case ex: RuntimeException =>
+                        val msg = s"Replaying record $coordinatorRecord from 
$tp at offset ${record.offset()} " +
+                          s"with producer id ${batch.producerId} and producer 
epoch ${batch.producerEpoch} " +
+                          s"failed due to: ${ex.getMessage}"
+                        error(s"$msg.")

Review Comment:
   How about `error(msg)`?



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