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]