[ 
https://issues.apache.org/jira/browse/BEAM-11325?focusedWorklogId=537896&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-537896
 ]

ASF GitHub Bot logged work on BEAM-11325:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Jan/21 16:52
            Start Date: 19/Jan/21 16:52
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on a change in pull request 
#13710:
URL: https://github.com/apache/beam/pull/13710#discussion_r560272725



##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -1402,6 +1407,15 @@ public void populateDisplayData(DisplayData.Builder 
builder) {
       return toBuilder().setConsumerFactoryFn(consumerFactoryFn).build();
     }
 
+    /**
+     * A custom {@link SerializableFunction} that determines whether the 
{@link ReadFromKafkaDoFn}
+     * should stop reading from the given {@link TopicPartition}.
+     */
+    public ReadSourceDescriptors<K, V> withCheckStopReadingFn(

Review comment:
       Could you add an example to the main `KafkaIO` Javadoc how to use this 
functionality to make it more clear for users?

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java
##########
@@ -288,6 +316,19 @@ public ProcessContinuation processElement(
               Optional.ofNullable(watermarkEstimator.currentWatermark()));
     }
     try (Consumer<byte[], byte[]> consumer = 
consumerFactoryFn.apply(updatedConsumerConfig)) {
+      // Check whether current TopicPartition is still available to read.
+      Set<TopicPartition> existingTopicPartitions = new HashSet<>();
+      for (List<PartitionInfo> topicPartitionList : 
consumer.listTopics().values()) {

Review comment:
       How often `consumer.listTopics()` will be called? Won't it affect a 
performance? Will it make sense to cache the result? 

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java
##########
@@ -288,6 +316,19 @@ public ProcessContinuation processElement(
               Optional.ofNullable(watermarkEstimator.currentWatermark()));
     }
     try (Consumer<byte[], byte[]> consumer = 
consumerFactoryFn.apply(updatedConsumerConfig)) {
+      // Check whether current TopicPartition is still available to read.
+      Set<TopicPartition> existingTopicPartitions = new HashSet<>();
+      for (List<PartitionInfo> topicPartitionList : 
consumer.listTopics().values()) {
+        topicPartitionList.forEach(
+            partitionInfo -> {
+              existingTopicPartitions.add(
+                  new TopicPartition(partitionInfo.topic(), 
partitionInfo.partition()));
+            });
+      }
+      if 
(!existingTopicPartitions.contains(kafkaSourceDescriptor.getTopicPartition())) {
+        return ProcessContinuation.stop();

Review comment:
       What's about checkpointing for already read records in this case? Are 
they going to be processed?




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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 537896)
    Time Spent: 20m  (was: 10m)

> KafkaIO should be able to read from new added topic/partition automatically 
> during pipeline execution time
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-11325
>                 URL: https://issues.apache.org/jira/browse/BEAM-11325
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-kafka
>            Reporter: Boyuan Zhang
>            Priority: P2
>          Time Spent: 20m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to