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



##########
File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ 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 { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.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 == null) {
+              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]
+    ): KafkaSourceOffset = {
     val fetched = partitionsAssignedToConsumer {
       partitions => {
         fnAssertParametersWithPartitions(partitions)
 
         val partitionOffsets = fnRetrievePartitionOffsets(partitions)
 
-        partitionOffsets.foreach {
-          case (tp, KafkaOffsetRangeLimit.LATEST) =>
-            consumer.seekToEnd(ju.Arrays.asList(tp))
-          case (tp, KafkaOffsetRangeLimit.EARLIEST) =>
-            consumer.seekToBeginning(ju.Arrays.asList(tp))
-          case (tp, off) => consumer.seek(tp, off)
+        val listOffsetsParams = partitionOffsets.filter { case (_, off) =>
+          off == KafkaOffsetRangeLimit.LATEST || off == 
KafkaOffsetRangeLimit.EARLIEST
+        }.map { case (tp, off) =>
+          off match {
+            case KafkaOffsetRangeLimit.LATEST =>
+              tp -> OffsetSpec.latest()
+            case KafkaOffsetRangeLimit.EARLIEST =>
+              tp -> OffsetSpec.earliest()
+          }
         }
-
-        partitionOffsets.map {
-          case (tp, _) => tp -> consumer.position(tp)
+        val resolvedPartitionOffsets = 
admin.listOffsets(listOffsetsParams.asJava,

Review comment:
       Good point, such way implementation details can be hidden.




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