HeartSaVioR commented on a change in pull request #25911: 
[SPARK-29223][SQL][SS] Enable global timestamp per topic while specifying 
offset by timestamp in Kafka source
URL: https://github.com/apache/spark/pull/25911#discussion_r328908222
 
 

 ##########
 File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
 ##########
 @@ -166,83 +169,118 @@ private[kafka010] class KafkaOffsetReader(
   def fetchSpecificOffsets(
       partitionOffsets: Map[TopicPartition, Long],
       reportDataLoss: String => Unit): KafkaSourceOffset = {
-    val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { 
partitions =>
-      assert(partitions.asScala == partitionOffsets.keySet,
-        "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")
+    val fnRetrievePartitionOffsets: (TPToOffsets, ju.Set[TopicPartition]) => 
TPToOffsets = {
+      case (newParams, _) => newParams
     }
 
-    val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long] = { _ =>
-      partitionOffsets
-    }
+    fetchSpecificOffsets0(partitionOffsets, 
adjustParamsWithPartitionsForOffsets,
+      fnRetrievePartitionOffsets, 
assertFetchedOffsetsForOffsets(reportDataLoss))
+  }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched 
=>
-      partitionOffsets.foreach {
-        case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
-          off != KafkaOffsetRangeLimit.EARLIEST =>
-          if (fetched(tp) != off) {
-            reportDataLoss(
-              s"startingOffsets for $tp was $off but consumer reset to 
${fetched(tp)}")
-          }
-        case _ =>
-        // no real way to check that beginning or end is reasonable
-      }
-    }
+  private def adjustParamsWithPartitionsForOffsets
+      : (TPToOffsets, ju.Set[TopicPartition]) => TPToOffsets = { case (params, 
partitions) =>
+    assert(partitions.asScala == params.keySet,
+      "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: ${params.keySet} Assigned: ${partitions.asScala}")
+    logDebug(s"Partitions assigned to consumer: $partitions. Seeking to 
$params")
+    params
+  }
 
-    fetchSpecificOffsets0(fnAssertParametersWithPartitions, 
fnRetrievePartitionOffsets,
-      fnAssertFetchedOffsets)
+  private def assertFetchedOffsetsForOffsets(reportDataLoss: String => Unit)
+      : (TPToOffsets, TPToOffsets) => Unit = { case (newParams, fetched) =>
+    newParams.foreach {
+      case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
+        off != KafkaOffsetRangeLimit.EARLIEST =>
+        if (fetched(tp) != off) {
+          reportDataLoss(
+            s"startingOffsets for $tp was $off but consumer reset to 
${fetched(tp)}")
+        }
+      case _ =>
+      // no real way to check that beginning or end is reasonable
+    }
   }
 
   def fetchSpecificTimestampBasedOffsets(
       partitionTimestamps: Map[TopicPartition, Long],
       failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = {
-    val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { 
partitions =>
-      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")
-    }
+    val fnAssertFetchedOffsets: (TPToOffsets, TPToOffsets) => Unit = { (_, _) 
=> }
 
-    val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => 
Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
-        }.asJava
+    fetchSpecificOffsets0(partitionTimestamps,
+      adjustParamsWithPartitionsForTimestampBasedOffset,
+      retrievePartitionOffsetsForTimestampBasedOffset(failsOnNoMatchingOffset),
+      fnAssertFetchedOffsets)
+  }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
+  private def adjustParamsWithPartitionsForTimestampBasedOffset
+      : (TPToOffsets, ju.Set[TopicPartition]) => TPToOffsets = { case (params, 
partitions) =>
+    val paramsGroupedByTopic = params.groupBy(_._1.topic())
+    val partitionsGroupedByTopic = partitions.asScala.groupBy(_.topic())
+
+    assert(paramsGroupedByTopic.keySet == partitionsGroupedByTopic.keySet,
+      s"Not all specified topics are assigned. Specified: ${params.keySet} " +
+        s"Assigned: ${partitions.asScala}")
+
+    val newParams: Map[TopicPartition, Long] = paramsGroupedByTopic.map {
+      case (topic, tpToOffset) =>
+        if 
(tpToOffset.keySet.map(_.partition()).contains(GLOBAL_PARTITION_NUM)) {
 
 Review comment:
   NOTE: This if statement is the only effective change. Others are mostly 
refactoring.

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


With regards,
Apache Git Services

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

Reply via email to