You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/03/19 08:13:05 UTC

[GitHub] [hudi] pratyakshsharma commented on a change in pull request #2438: [HUDI-1447] DeltaStreamer kafka source supports consuming from specified timestamp

pratyakshsharma commented on a change in pull request #2438:
URL: https://github.com/apache/hudi/pull/2438#discussion_r830456199



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -283,6 +323,41 @@ private Long delayOffsetCalculation(Option<String> lastCheckpointStr, Set<TopicP
     return delayCount;
   }
 
+  /**
+   * Get the checkpoint by timestamp.
+   * This method returns the checkpoint format based on the timestamp.
+   * example:
+   * 1. input: timestamp, etc.
+   * 2. output: topicName,partition_num_0:100,partition_num_1:101,partition_num_2:102.
+   *
+   * @param consumer
+   * @param topicName
+   * @param timestamp
+   * @return
+   */
+  private Option<String> getOffsetsByTimestamp(KafkaConsumer consumer, List<PartitionInfo> partitionInfoList, Set<TopicPartition> topicPartitions,
+                                               String topicName, Long timestamp) {
+
+    Map<TopicPartition, Long> topicPartitionsTimestamp = partitionInfoList.stream()
+                                                    .map(x -> new TopicPartition(x.topic(), x.partition()))
+                                                    .collect(Collectors.toMap(Function.identity(), x -> timestamp));
+
+    Map<TopicPartition, Long> earliestOffsets = consumer.beginningOffsets(topicPartitions);
+    Map<TopicPartition, OffsetAndTimestamp> offsetAndTimestamp = consumer.offsetsForTimes(topicPartitionsTimestamp);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append(topicName + ",");
+    for (Map.Entry<TopicPartition, OffsetAndTimestamp> map : offsetAndTimestamp.entrySet()) {
+      if (map.getValue() != null) {
+        sb.append(map.getKey().partition()).append(":").append(map.getValue().offset()).append(",");
+      } else {
+        sb.append(map.getKey().partition()).append(":").append(earliestOffsets.get(map.getKey())).append(",");

Review comment:
       @liujinhui1994 @nsivabalan Can you help me understand why are we adding this value here from earliestOffsets? From what I understand, the whole point of consuming from specified timestamp is we do not want to consume records whose offset has timestamp lesser than the specified timestamp. Let us take an example of topic A with 3 partitions 0,1,2. Offsets are as below - 
   `partition 0 - 100 (ts-210),101 (ts-220),102 (ts-230),103 (ts-240) .....
   partition 1 - 50 (ts 200), 51 (ts-205), 52 (ts-225) ....
   partition 2 - 51 (ts - 100), 60 (ts - 150) (only 2 records present in this)`
   
   Now suppose if the timestamp is passed as 220, the expected results from consumer api will be - 
   `partition 0 -> 101
   partition 1 -> 52
   partition 2 -> null`
   
   As per the code, we return - 
   `partition 0 -> 101
   partition 1 -> 52
   partition 2 -> 51`
   
   I want to understand why are we populating this value here for partition 2? If the corresponding offsets in partition 2 have timestamp less than 220, this implies these offsets have either been already consumed or the records are not needed at all for ingestion into hudi table. Ideally no offset should be returned from this method for partition 2. 
   
   Even if this functionality is added only for one time initial bootstrap, then also consuming the records from partition 2 above does not make sense. Please let me know the thought process behind this logic. 




-- 
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: commits-unsubscribe@hudi.apache.org

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