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



##########
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 offsets for the specified partitions and OffsetSpec. This 
operation enables to find
+         * the beginning offset, end offset as well as the offset matching a 
timestamp in
+         * partitions.
+         *
+         * @see KafkaAdminClient#listOffsets(Map)
+         * @param topicPartitionOffsets The mapping from partition to the 
OffsetSpec to look up.
+         * @return The list offsets result.
+         */
+        private Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo> 
listOffsets(
+                Map<TopicPartition, OffsetSpec> topicPartitionOffsets) {
+            try {
+                return adminClient
+                        .listOffsets(topicPartitionOffsets)
+                        .all()
+                        .thenApply(
+                                result -> {
+                                    Map<TopicPartition, 
ListOffsetsResult.ListOffsetsResultInfo>
+                                            offsets = new HashMap<>();
+                                    result.forEach(
+                                            (tp, listOffsetsResultInfo) -> {
+                                                if (listOffsetsResultInfo != 
null) {
+                                                    offsets.put(tp, 
listOffsetsResultInfo);
+                                                }
+                                            });
+                                    return offsets;
+                                })
+                        .get();
+            } catch (InterruptedException e) {
+                throw new FlinkRuntimeException(

Review comment:
       Please also set the interrupted flag if you handle the 
`InterruptedException`
   
   ```java
   Thread.currentThread().interrupt();
   ```

##########
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 offsets for the specified partitions and OffsetSpec. This 
operation enables to find
+         * the beginning offset, end offset as well as the offset matching a 
timestamp in
+         * partitions.
+         *
+         * @see KafkaAdminClient#listOffsets(Map)
+         * @param topicPartitionOffsets The mapping from partition to the 
OffsetSpec to look up.
+         * @return The list offsets result.
+         */
+        private Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo> 
listOffsets(
+                Map<TopicPartition, OffsetSpec> topicPartitionOffsets) {
+            try {
+                return adminClient
+                        .listOffsets(topicPartitionOffsets)
+                        .all()
+                        .thenApply(
+                                result -> {
+                                    Map<TopicPartition, 
ListOffsetsResult.ListOffsetsResultInfo>
+                                            offsets = new HashMap<>();
+                                    result.forEach(
+                                            (tp, listOffsetsResultInfo) -> {
+                                                if (listOffsetsResultInfo != 
null) {
+                                                    offsets.put(tp, 
listOffsetsResultInfo);
+                                                }
+                                            });
+                                    return offsets;
+                                })
+                        .get();
+            } catch (InterruptedException e) {
+                throw new FlinkRuntimeException(
+                        "Interrupted while listing offsets for topic 
partitions: "
+                                + topicPartitionOffsets,
+                        e);
+            } catch (ExecutionException e) {
+                throw new FlinkRuntimeException(
+                        "Failed to list offsets for topic partitions: "
+                                + topicPartitionOffsets
+                                + " due to",
+                        e);
+            }
+        }
+
         @Override
         public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> 
partitions) {
-            return consumer.endOffsets(partitions);
+            return listOffsets(

Review comment:
       The methods `endOffset` and `beginningOffsets` look very similar, please 
extract a common method that takes as input the `OffsetSpec`.

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
##########
@@ -281,29 +283,41 @@ 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()) {
-
-                result.put(
-                        new KafkaTopicPartition(
-                                partitionToOffset.getKey().topic(),
-                                partitionToOffset.getKey().partition()),
-                        (partitionToOffset.getValue() == null)
-                                ? null
-                                : partitionToOffset.getValue().offset());
-            }
+        try (Admin adminClient = Admin.create(properties)) {

Review comment:
       I wouldn't add this change to the `FlinkKafkaConsumer` because it is 
already marked as deprecated and will be removed in one of the next releases. 
We want to incentivize users to upgrade to the KafkaSource and do not continue 
using this class.

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
##########
@@ -152,7 +152,7 @@ public static void startClusters(boolean secureMode) throws 
Exception {
 
     public static void startClusters(KafkaTestEnvironment.Config 
environmentConfig)
             throws Exception {
-        kafkaServer = constructKafkaTestEnvionment();
+        kafkaServer = constructKafkaTestEnvironment();

Review comment:
       Nit: Can you make the renaming on a separate commit as hotfix?




-- 
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: [email protected]

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


Reply via email to