xuanyuanking commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488842486



##########
File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +187,68 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you 
must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: 
${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to 
$partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to 
$partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { case (tp, timestamp) 
=>
+          tp -> OffsetSpec.forTimestamp(timestamp)
         }.asJava
+        admin.listOffsets(listOffsetsParams, 
listOffsetsOptions).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != 
OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request 
of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec.offset() == OffsetFetchResponse.INVALID_OFFSET) {
+              tp -> KafkaOffsetRangeLimit.LATEST
+            } else {
+              tp -> offsetSpec.offset()
+            }
         }.toMap
       }
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
-
-    fetchSpecificOffsets0(fnAssertParametersWithPartitions, 
fnRetrievePartitionOffsets,
-      fnAssertFetchedOffsets)
+    fetchSpecificOffsets0(fnAssertParametersWithPartitions, 
fnRetrievePartitionOffsets)
   }
 
   private def fetchSpecificOffsets0(
       fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
-      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long],
-      fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): 
KafkaSourceOffset = {
+      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long]

Review comment:
       Is it a behavior change of removing the `fnAssertFetchedOffsets`?

##########
File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  private def nextGroupId(): String = {
-    groupId = driverGroupIdPrefix + "-" + nextId
-    nextId += 1
-    groupId
-  }
-
   override def toString(): String = consumerStrategy.toString
 
   /**
    * Closes the connection to Kafka, and cleans up state.
    */
   def close(): Unit = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { 
stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { 
stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = 
uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       Shall we keep the UninterruptibleThread checking in 
https://github.com/apache/spark/pull/29729/files#diff-870db3f661cb22047ad39d67c7c8b22fR139?

##########
File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -182,28 +170,14 @@ private[kafka010] class KafkaOffsetReader(
         "If startingOffsets contains specific offsets, you must specify all 
TopicPartitions.\n" +
           "Use -1 for latest, -2 for earliest, if you don't care.\n" +
           s"Specified: ${partitionOffsets.keySet} Assigned: 
${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to 
$partitionOffsets")
+      logDebug(s"Assigned partitions: $partitions. Seeking to 
$partitionOffsets")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long] = { _ =>
       partitionOffsets
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched 
=>
-      partitionOffsets.foreach {
-        case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
-          off != KafkaOffsetRangeLimit.EARLIEST =>
-          if (fetched(tp) != off) {
-            reportDataLoss(

Review comment:
       The `reportDataLoss` is not called in the new version of `def 
fetchSpecificOffsets`.




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to