alpreu commented on a change in pull request #18145:
URL: https://github.com/apache/flink/pull/18145#discussion_r790522381



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
##########
@@ -281,27 +283,50 @@ protected AbstractPartitionDiscoverer 
createPartitionDiscoverer(
     protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
             Collection<KafkaTopicPartition> partitions, long timestamp) {
 
-        Map<TopicPartition, Long> partitionOffsetsRequest = new 
HashMap<>(partitions.size());
+        Map<TopicPartition, OffsetSpec> partitionOffsetsRequest = new 
HashMap<>(partitions.size());
         for (KafkaTopicPartition partition : partitions) {
             partitionOffsetsRequest.put(
-                    new TopicPartition(partition.getTopic(), 
partition.getPartition()), timestamp);
+                    new TopicPartition(partition.getTopic(), 
partition.getPartition()),
+                    OffsetSpec.forTimestamp(timestamp));
         }
 
         final Map<KafkaTopicPartition, Long> result = new 
HashMap<>(partitions.size());
         // use a short-lived consumer to fetch the offsets;
         // this is ok because this is a one-time operation that happens only 
on startup
-        try (KafkaConsumer<?, ?> consumer = new KafkaConsumer(properties)) {
-            for (Map.Entry<TopicPartition, OffsetAndTimestamp> 
partitionToOffset :
-                    
consumer.offsetsForTimes(partitionOffsetsRequest).entrySet()) {
-
+        try (Admin adminClient = Admin.create(properties)) {
+            Map<TopicPartition, Long> topicPartitionOffsets =
+                    adminClient
+                            .listOffsets(partitionOffsetsRequest)
+                            .all()
+                            .thenApply(
+                                    info -> {
+                                        Map<TopicPartition, Long> offsets = 
new HashMap<>();
+                                        info.forEach(
+                                                (tp, listOffsetsResultInfo) -> 
{
+                                                    if (listOffsetsResultInfo 
!= null) {
+                                                        offsets.put(
+                                                                tp, 
listOffsetsResultInfo.offset());
+                                                    }
+                                                });
+                                        return offsets;
+                                    })
+                            .get();
+            for (Map.Entry<TopicPartition, Long> partitionToOffset :
+                    topicPartitionOffsets.entrySet()) {
                 result.put(
                         new KafkaTopicPartition(
                                 partitionToOffset.getKey().topic(),
                                 partitionToOffset.getKey().partition()),
                         (partitionToOffset.getValue() == null)
                                 ? null
-                                : partitionToOffset.getValue().offset());
+                                : partitionToOffset.getValue());

Review comment:
       I believe the code can be easier to read if this for loop was moved into 
the above one (L304), WDYT?

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java
##########
@@ -558,25 +533,100 @@ public PartitionOffsetsRetrieverImpl(
             }
         }
 
+        /**
+         * List offset for the specified partitions and OffsetSpec. This 
operation enables to find

Review comment:
       ```suggestion
            * List offsets for the specified partitions and OffsetSpec. This 
operation enables to find
   ```




-- 
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: issues-unsubscr...@flink.apache.org

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


Reply via email to