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

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

                Author: ASF GitHub Bot
            Created on: 20/Apr/18 18:51
            Start Date: 20/Apr/18 18:51
    Worklog Time Spent: 10m 
      Work Description: jkff commented on a change in pull request #5141: 
[BEAM-3925] Allow ValueProvider for KafkaIO so that we can create Beam 
Templates using KafkaIO
URL: https://github.com/apache/beam/pull/5141#discussion_r183139278
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
 ##########
 @@ -335,22 +346,38 @@
      * of how the partitions are distributed among the splits.
      */
     public Read<K, V> withTopic(String topic) {
-      return withTopics(ImmutableList.of(topic));
+      return withTopic(ValueProvider.StaticValueProvider.of(topic));
     }
 
     /**
-     * Sets a list of topics to read from. All the partitions from each
-     * of the topics are read.
+     * Like above but with a {@link ValueProvider ValueProvider&lt;String&gt;}.
+     */
+    public Read<K, V> withTopic(ValueProvider<String> topic) {
+      return withTopics(topic);
+    }
+
+    /**
+     * Sets a list of topics to read from with a {@link List 
List&lt;String&gt;}.
+     * All the partitions from each of the topics are read.
      *
      * <p>See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for 
description
      * of how the partitions are distributed among the splits.
      */
     public Read<K, V> withTopics(List<String> topics) {
-      checkState(
-          getTopicPartitions().isEmpty(), "Only topics or topicPartitions can 
be set, not both");
-      return toBuilder().setTopics(ImmutableList.copyOf(topics)).build();
+      return 
withTopics(ValueProvider.StaticValueProvider.of(Joiner.on(',').join(topics)));
     }
 
+    /**
+     * Like above but with a {@link ValueProvider ValueProvider&lt;String&gt;}.
+     * The format is comma separated String of topics
+     */
+    public Read<K, V> withTopics(ValueProvider<String> topics) {
+      checkState(
+              getTopicPartitions().get().isEmpty(),
 
 Review comment:
   Calling `.get()` here means that this can not be used with a runtime value 
provider at all which defeats the purpose of this change. `.get()` can only be 
called inside code that runs while the pipeline is running, but not at 
construction time - i.e. only inside code of `DoFn`s, effectively.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 93363)
    Time Spent: 1h 10m  (was: 1h)

> Allow ValueProvider for KafkaIO
> -------------------------------
>
>                 Key: BEAM-3925
>                 URL: https://issues.apache.org/jira/browse/BEAM-3925
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-core
>            Reporter: Sameer Abhyankar
>            Assignee: Pramod Upamanyu
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Add ValueProvider support for the various methods in KafkaIO. This would 
> allow us to use KafkaIO in reusable pipeline templates.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to