You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/02/28 17:34:05 UTC

[GitHub] [kafka] dajac commented on a change in pull request #11173: KAFKA-13509: Support max timestamp in GetOffsetShell

dajac commented on a change in pull request #11173:
URL: https://github.com/apache/kafka/pull/11173#discussion_r816093355



##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -70,12 +71,13 @@ object GetOffsetShell {
                            .withRequiredArg
                            .describedAs("partition ids")
                            .ofType(classOf[String])
-    val timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]")
+    val timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]" +
+                                  " It also accepts \"earliest\", \"latest\", \"max-timestamp\", -1(latest), -2(earliest) -3(max timestamp) ")
                            .withRequiredArg
-                           .describedAs("timestamp/-1(latest)/-2(earliest)")
-                           .ofType(classOf[java.lang.Long])
-                           .defaultsTo(-1L)
-    val commandConfigOpt = parser.accepts("command-config", s"Property file containing configs to be passed to Consumer Client.")
+                           .describedAs("timestamp/latest/earliest/max-timestamp/-1(latest)/-2(earliest)/-3(max timestamp)")

Review comment:
       nit: I wonder if we could just say: `<timestamp> / -1 or latest / -2 or earliest / -3 or max-timestamp`. What do you think?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -103,59 +105,76 @@ object GetOffsetShell {
       throw new IllegalArgumentException("--topic-partitions cannot be used with --topic or --partitions")
     }
 
-    val listOffsetsTimestamp = options.valueOf(timeOpt).longValue
+    val listOffsetsTimestamp = options.valueOf(timeOpt)
+    val offsetSpec = listOffsetsTimestamp match {

Review comment:
       nit: Could we extract this into a method?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -70,12 +71,13 @@ object GetOffsetShell {
                            .withRequiredArg
                            .describedAs("partition ids")
                            .ofType(classOf[String])
-    val timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]")
+    val timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]" +
+                                  " It also accepts \"earliest\", \"latest\", \"max-timestamp\", -1(latest), -2(earliest) -3(max timestamp) ")

Review comment:
       nit: Is this redundant with the `describedAs` field?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -103,59 +105,76 @@ object GetOffsetShell {
       throw new IllegalArgumentException("--topic-partitions cannot be used with --topic or --partitions")
     }
 
-    val listOffsetsTimestamp = options.valueOf(timeOpt).longValue
+    val listOffsetsTimestamp = options.valueOf(timeOpt)
+    val offsetSpec = listOffsetsTimestamp match {
+      case "earliest" => OffsetSpec.earliest()
+      case "latest" => OffsetSpec.latest()
+      case "max-timestamp" => OffsetSpec.maxTimestamp()
+      case _ =>
+        try {
+          java.lang.Long.parseLong(listOffsetsTimestamp) match {
+            case ListOffsetsRequest.EARLIEST_TIMESTAMP => OffsetSpec.earliest()
+            case ListOffsetsRequest.LATEST_TIMESTAMP => OffsetSpec.latest()
+            case ListOffsetsRequest.MAX_TIMESTAMP => OffsetSpec.maxTimestamp()
+            case value => OffsetSpec.forTimestamp(value)
+          }
+        } catch {
+          case e: NumberFormatException =>
+            throw new IllegalArgumentException(s"Malformed time argument $listOffsetsTimestamp, please use latest/earliest/max-timestamp/-1(latest)/-2(earliest)/-3(max timestamp), or a specified long format timestamp", e)
+        }
+    }
 
     val topicPartitionFilter = if (options.has(topicPartitionsOpt)) {
-      createTopicPartitionFilterWithPatternList(options.valueOf(topicPartitionsOpt), excludeInternalTopics)
+      createTopicPartitionFilterWithPatternList(options.valueOf(topicPartitionsOpt))
     } else {
       val partitionIdsRequested = createPartitionSet(options.valueOf(partitionsOpt))
 
-      createTopicPartitionFilterWithTopicAndPartitionPattern(
-        if (options.has(topicOpt)) Some(options.valueOf(topicOpt)) else None,
-        excludeInternalTopics,
-        partitionIdsRequested
+      TopicFilterAndPartitionFilter(
+        if (options.has(topicOpt)) IncludeList(options.valueOf(topicOpt)) else IncludeList(".*"),
+        PartitionsSetFilter(partitionIdsRequested)
       )
     }
 
     val config = if (options.has(commandConfigOpt))
       Utils.loadProps(options.valueOf(commandConfigOpt))
     else
       new Properties
-    config.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
-    config.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, clientId)
-    val consumer = new KafkaConsumer(config, new ByteArrayDeserializer, new ByteArrayDeserializer)
+    config.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+    config.setProperty(AdminClientConfig.CLIENT_ID_CONFIG, clientId)
+    val client = Admin.create(config)
 
     try {
-      val partitionInfos = listPartitionInfos(consumer, topicPartitionFilter)
+      val partitionInfos = listPartitionInfos(client, topicPartitionFilter, excludeInternalTopics)
 
       if (partitionInfos.isEmpty) {
         throw new IllegalArgumentException("Could not match any topic-partitions with the specified filters")
       }
 
-      val topicPartitions = partitionInfos.flatMap { p =>
-        if (p.leader == null) {
-          System.err.println(s"Error: topic-partition ${p.topic}:${p.partition} does not have a leader. Skip getting offsets")
-          None
-        } else
-          Some(new TopicPartition(p.topic, p.partition))
-      }
+      val timestampsToSearch = partitionInfos.map(tp => tp -> offsetSpec).toMap.asJava
 
-      /* Note that the value of the map can be null */
-      val partitionOffsets: collection.Map[TopicPartition, java.lang.Long] = listOffsetsTimestamp match {
-        case ListOffsetsRequest.EARLIEST_TIMESTAMP => consumer.beginningOffsets(topicPartitions.asJava).asScala
-        case ListOffsetsRequest.LATEST_TIMESTAMP => consumer.endOffsets(topicPartitions.asJava).asScala
-        case _ =>
-          val timestampsToSearch = topicPartitions.map(tp => tp -> (listOffsetsTimestamp: java.lang.Long)).toMap.asJava
-          consumer.offsetsForTimes(timestampsToSearch).asScala.map { case (k, x) =>
-            if (x == null) (k, null) else (k, x.offset: java.lang.Long)
-          }
+      val listOffsetsResult = client.listOffsets(timestampsToSearch)
+      val partitionOffsets = partitionInfos.flatMap { tp =>
+        try {
+          val partitionInfo = listOffsetsResult.partitionResult(tp).get
+          Some((tp, partitionInfo.offset))
+        } catch {
+          case e: ExecutionException =>
+            e.getCause match {
+              case _: LeaderNotAvailableException =>
+                System.err.println(s"Error: topic-partition ${tp.topic}:${tp.partition} does not have a leader. Skip getting offsets")
+              case _ =>
+                System.err.println(s"Error while getting end offsets for topic-partition ${tp.topic}:${tp.partition}")
+                e.printStackTrace()

Review comment:
       This looks a bit weird to me. Do we really need this?

##########
File path: core/src/test/scala/kafka/tools/TopicPartitionFilterTest.scala
##########
@@ -0,0 +1,95 @@
+/**
+ * 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.tools
+
+import kafka.utils.IncludeList
+import org.apache.kafka.common.TopicPartition
+import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue}
+import org.junit.jupiter.api.Test
+
+class TopicPartitionFilterTest {

Review comment:
       Could we merge those tests into `GetOffsetShellParsingTest.scala` or do we really need this new class?

##########
File path: core/src/test/scala/kafka/tools/GetOffsetShellTest.scala
##########
@@ -109,6 +109,49 @@ class GetOffsetShellTest extends KafkaServerTestHarness with Logging {
     )
   }
 
+  @Test
+  def testGetLatestOffsets(): Unit = {
+    for (time <- Array("-1", "latest")) {

Review comment:
       Could we use parameterized tests?

##########
File path: core/src/test/scala/kafka/tools/GetOffsetShellTest.scala
##########
@@ -109,6 +109,49 @@ class GetOffsetShellTest extends KafkaServerTestHarness with Logging {
     )
   }
 
+  @Test
+  def testGetLatestOffsets(): Unit = {
+    for (time <- Array("-1", "latest")) {
+      val offsets = executeAndParse(Array("--topic-partitions", "topic.*:0", "--time", time))
+      assertEquals(
+        List(
+          ("topic1", 0, Some(0)),
+          ("topic2", 0, Some(1)),
+          ("topic3", 0, Some(2)),
+          ("topic4", 0, Some(3))
+        ),
+        offsets
+      )
+    }
+  }
+
+  @Test
+  def testGetEarliestOffsets(): Unit = {
+    for (time <- Array("-2", "earliest")) {
+      val offsets = executeAndParse(Array("--topic-partitions", "topic.*:0", "--time", time))
+      assertEquals(
+        List(
+          ("topic1", 0, Some(0)),
+          ("topic2", 0, Some(0)),
+          ("topic3", 0, Some(0)),
+          ("topic4", 0, Some(0))
+        ),
+        offsets
+      )
+    }
+  }
+
+  @Test
+  def testGetOffsetsByMaxTimestamp(): Unit = {
+    for (time <- Array("-3", "max-timestamp")) {
+      val offsets = executeAndParse(Array("--topic-partitions", "topic.*", "--time", time))
+      offsets.foreach( offset =>

Review comment:
       nit: We usually use curly braces instead of parenthesis for blocks. Could we use pattern matching to deconstruct  `offset` instead of using `get` everywhere?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -224,9 +239,81 @@ object GetOffsetShell {
   /**
    * Return the partition infos. Filter them with topicPartitionFilter.
    */
-  private def listPartitionInfos(consumer: KafkaConsumer[_, _], topicPartitionFilter: PartitionInfo => Boolean): Seq[PartitionInfo] = {
-    consumer.listTopics.asScala.values.flatMap { partitions =>
-      partitions.asScala.filter(topicPartitionFilter)
+  private def listPartitionInfos(
+    client: Admin,
+    topicPartitionFilter: TopicPartitionFilter,
+    excludeInternalTopics: Boolean
+  ): Seq[TopicPartition] = {
+    val listTopicsOptions = new ListTopicsOptions().listInternal(!excludeInternalTopics)
+    val topics = client.listTopics(listTopicsOptions).names.get
+    val filteredTopics = topics.asScala.filter(topicPartitionFilter.isTopicAllowed)
+
+    client.describeTopics(filteredTopics.asJava).allTopicNames.get.asScala.flatMap { case (topic, description) =>
+      description
+        .partitions
+        .asScala
+        .map(tp => new TopicPartition(topic, tp.partition))
+        .filter(topicPartitionFilter.isPartitionAllowed)
     }.toBuffer
   }
 }
+
+/**
+ * Used to filter partitions after describing them
+ */
+trait PartitionFilter {
+  def isPartitionAllowed(partition: Int): Boolean
+}
+
+case class PartitionsSetFilter(partitionIds: Set[Int]) extends PartitionFilter {
+  override def isPartitionAllowed(partition: Int): Boolean = partitionIds.isEmpty || partitionIds.contains(partition)
+}
+
+case class UniquePartitionFilter(partition: Int) extends PartitionFilter {
+  override def isPartitionAllowed(partition: Int): Boolean = partition == this.partition
+}
+
+case class PartitionRangeFilter(lowerRange: Int, upperRange: Int) extends PartitionFilter {
+  override def isPartitionAllowed(partition: Int): Boolean = partition >= lowerRange && partition < upperRange
+}
+
+trait TopicPartitionFilter {
+
+  /**
+   * Used to filter topics before describing them
+   */
+  def isTopicAllowed(topic: String): Boolean
+
+  /**
+   * Used to filter topics and topic-partitions after describing them
+   */
+  def isPartitionAllowed(partition: TopicPartition): Boolean

Review comment:
       nit: Should we call is `isTopicPartitionAllowed`?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -224,9 +239,81 @@ object GetOffsetShell {
   /**
    * Return the partition infos. Filter them with topicPartitionFilter.
    */
-  private def listPartitionInfos(consumer: KafkaConsumer[_, _], topicPartitionFilter: PartitionInfo => Boolean): Seq[PartitionInfo] = {
-    consumer.listTopics.asScala.values.flatMap { partitions =>
-      partitions.asScala.filter(topicPartitionFilter)
+  private def listPartitionInfos(
+    client: Admin,
+    topicPartitionFilter: TopicPartitionFilter,
+    excludeInternalTopics: Boolean
+  ): Seq[TopicPartition] = {
+    val listTopicsOptions = new ListTopicsOptions().listInternal(!excludeInternalTopics)
+    val topics = client.listTopics(listTopicsOptions).names.get
+    val filteredTopics = topics.asScala.filter(topicPartitionFilter.isTopicAllowed)
+
+    client.describeTopics(filteredTopics.asJava).allTopicNames.get.asScala.flatMap { case (topic, description) =>
+      description
+        .partitions
+        .asScala
+        .map(tp => new TopicPartition(topic, tp.partition))
+        .filter(topicPartitionFilter.isPartitionAllowed)

Review comment:
       Is it worth changing `isPartitionAllowed` to take the topic and the partition directly? We would not have to create the `TopicPartition` then.

##########
File path: core/src/test/scala/kafka/tools/GetOffsetShellParsingTest.scala
##########
@@ -17,191 +17,231 @@
 
 package kafka.tools
 
-import org.apache.kafka.common.PartitionInfo
-import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
+import org.apache.kafka.common.TopicPartition
+import org.junit.jupiter.api.Assertions.{assertFalse, assertThrows, assertTrue}
 import org.junit.jupiter.api.Test
-import org.junit.jupiter.params.ParameterizedTest
-import org.junit.jupiter.params.provider.ValueSource
 
 class GetOffsetShellParsingTest {
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForTopicName(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 0)))
-    assertTrue(filter.apply(partitionInfo("test", 1)))
-    assertFalse(filter.apply(partitionInfo("test1", 0)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForInternalTopicName(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("__consumer_offsets", excludeInternal)
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0)))
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 1)))
-    assertFalse(filter.apply(partitionInfo("test1", 0)))
-    assertFalse(filter.apply(partitionInfo("test2", 0)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForTopicNameList(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test,test1,__consumer_offsets", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 0)))
-    assertTrue(filter.apply(partitionInfo("test1", 1)))
-    assertFalse(filter.apply(partitionInfo("test2", 0)))
-
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForRegex(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test.*", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 0)))
-    assertTrue(filter.apply(partitionInfo("test1", 1)))
-    assertTrue(filter.apply(partitionInfo("test2", 0)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForPartitionIndexSpec(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":0", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 0)))
-    assertTrue(filter.apply(partitionInfo("test1", 0)))
-    assertFalse(filter.apply(partitionInfo("test2", 1)))
-
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 1)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForPartitionRangeSpec(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-3", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 1)))
-    assertTrue(filter.apply(partitionInfo("test1", 2)))
-    assertFalse(filter.apply(partitionInfo("test2", 0)))
-    assertFalse(filter.apply(partitionInfo("test2", 3)))
-
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 3)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForPartitionLowerBoundSpec(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 1)))
-    assertTrue(filter.apply(partitionInfo("test1", 2)))
-    assertFalse(filter.apply(partitionInfo("test2", 0)))
-
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))
-  def testTopicPartitionFilterForPartitionUpperBoundSpec(excludeInternal: Boolean): Unit = {
-    val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":-3", excludeInternal)
-    assertTrue(filter.apply(partitionInfo("test", 0)))
-    assertTrue(filter.apply(partitionInfo("test1", 1)))
-    assertTrue(filter.apply(partitionInfo("test2", 2)))
-    assertFalse(filter.apply(partitionInfo("test3", 3)))
-
-    assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2)))
-    assertFalse(filter.apply(partitionInfo("__consumer_offsets", 3)))
-  }
-
-  @ParameterizedTest
-  @ValueSource(booleans = Array(true, false))

Review comment:
       Why do we remove those parameterised tests?

##########
File path: core/src/main/scala/kafka/tools/GetOffsetShell.scala
##########
@@ -103,59 +105,76 @@ object GetOffsetShell {
       throw new IllegalArgumentException("--topic-partitions cannot be used with --topic or --partitions")
     }
 
-    val listOffsetsTimestamp = options.valueOf(timeOpt).longValue
+    val listOffsetsTimestamp = options.valueOf(timeOpt)
+    val offsetSpec = listOffsetsTimestamp match {
+      case "earliest" => OffsetSpec.earliest()
+      case "latest" => OffsetSpec.latest()
+      case "max-timestamp" => OffsetSpec.maxTimestamp()
+      case _ =>
+        try {
+          java.lang.Long.parseLong(listOffsetsTimestamp) match {

Review comment:
       nit: Could we use `listOffsetsTimestamp.toLong`?




-- 
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-unsubscribe@kafka.apache.org

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