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

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_r183139080
 
 

 ##########
 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
 
 Review comment:
   Please do not use "string typing" (as opposed to "strong typing") - this has 
to be a `ValueProvider<List<String>>`. That means that this value can not be 
directly passed as a command-line argument, only via a NestedValueProvider - 
please add a helper class somewhere that simplifies this (e.g. add a static 
utility method to `ValueProviders`, such as `ValueProvider<List<String>> 
ValueProviders.commaSeparatedStrings(ValueProvider<String> csv)`).

----------------------------------------------------------------
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: 93360)
    Time Spent: 1h  (was: 50m)

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