johnjcasey commented on code in PR #34659:
URL: https://github.com/apache/beam/pull/34659#discussion_r2058684495


##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java:
##########
@@ -1910,7 +1919,15 @@ public void 
processElement(OutputReceiver<KafkaSourceDescriptor> receiver) {
               }
             } else {
               for (String topic : topics) {
-                for (PartitionInfo p : consumer.partitionsFor(topic)) {
+                List<PartitionInfo> partitionInfoList = 
consumer.partitionsFor(topic);

Review Comment:
   do we need to inspect the log verification flag here?



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java:
##########
@@ -100,8 +104,53 @@ public List<KafkaUnboundedSource<K, V>> split(int 
desiredNumSplits, PipelineOpti
         }
       }
     } else {
+      final Map<String, List<Integer>> topicsAndPartitions = new HashMap<>();
       for (TopicPartition p : partitions) {
-        Lineage.getSources().add("kafka", ImmutableList.of(bootStrapServers, 
p.topic()));
+        topicsAndPartitions.computeIfAbsent(p.topic(), k -> new 
ArrayList<>()).add(p.partition());
+      }
+
+      try (Consumer<?, ?> consumer = 
spec.getConsumerFactoryFn().apply(spec.getConsumerConfig())) {
+        for (Map.Entry<String, List<Integer>> e : 
topicsAndPartitions.entrySet()) {
+          final String providedTopic = e.getKey();
+          final List<Integer> providedPartitions = e.getValue();
+          final Set<Integer> partitionsForTopic;
+          try {
+            partitionsForTopic =
+                consumer.partitionsFor(providedTopic).stream()
+                    .map(PartitionInfo::partition)
+                    .collect(Collectors.toSet());
+            if (spec.getLogTopicVerification() == null || 
!spec.getLogTopicVerification()) {

Review Comment:
   we could simplify these checks by defaulting the value to false



-- 
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: github-unsubscr...@beam.apache.org

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

Reply via email to