junrao commented on code in PR #15474:
URL: https://github.com/apache/kafka/pull/15474#discussion_r1523649729


##########
core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala:
##########
@@ -19,82 +19,236 @@ package kafka.admin
 
 import kafka.integration.KafkaServerTestHarness
 import kafka.server.KafkaConfig
+import kafka.utils.TestUtils.{createProducer, plaintextBootstrapServers}
 import kafka.utils.{TestInfoUtils, TestUtils}
 import org.apache.kafka.clients.admin._
 import org.apache.kafka.clients.producer.ProducerRecord
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.config.TopicConfig
+import org.apache.kafka.common.utils.{MockTime, Time, Utils}
 import org.junit.jupiter.api.Assertions.assertEquals
 import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
 
+import java.util.Properties
 import scala.collection.{Map, Seq}
 import scala.jdk.CollectionConverters._
 
 class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
 
   val topicName = "foo"
+  val topicNameWithCustomConfigs = "foo2"
   var adminClient: Admin = _
+  var setOldMessageFormat: Boolean = false
+  val mockTime: Time = new MockTime(1)
 
   @BeforeEach
   override def setUp(testInfo: TestInfo): Unit = {
     super.setUp(testInfo)
-    createTopic(topicName, 1, 1.toShort)
-    produceMessages()
+    createTopicWithConfig(topicName, new Properties())
     adminClient = Admin.create(Map[String, Object](
       AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> bootstrapServers()
     ).asJava)
   }
 
+  override def brokerTime(brokerId: Int): Time = mockTime
+
   @AfterEach
   override def tearDown(): Unit = {
+    setOldMessageFormat = false
     Utils.closeQuietly(adminClient, "ListOffsetsAdminClient")
     super.tearDown()
   }
 
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
-  def testEarliestOffset(quorum: String): Unit = {
-    val earliestOffset = runFetchOffsets(adminClient, OffsetSpec.earliest())
-    assertEquals(0, earliestOffset.offset())
+  def testThreeCompressedRecordsInOneBatch(quorum: String): Unit = {
+    produceMessagesInOneBatch("gzip")
+    verifyListOffsets()
+
+    // test LogAppendTime case
+    val props: Properties = new Properties()
+    props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, 
"LogAppendTime")
+    createTopicWithConfig(topicNameWithCustomConfigs, props)
+    produceMessagesInOneBatch("gzip", topicNameWithCustomConfigs)
+    // In LogAppendTime's case, the maxTimestampOffset should be the first 
message of the batch.
+    // So in this one batch test, it'll be the first offset 0
+    verifyListOffsets(topic = topicNameWithCustomConfigs, 0)
   }
 
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
-  def testLatestOffset(quorum: String): Unit = {
-    val latestOffset = runFetchOffsets(adminClient, OffsetSpec.latest())
-    assertEquals(3, latestOffset.offset())
+  def testThreeRecordsInSeparateBatch(quorum: String): Unit = {
+    produceMessagesInSeparateBatch()
+    verifyListOffsets()

Review Comment:
   Why did we exclude the LogAppendTime test in this method?



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