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

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

                Author: ASF GitHub Bot
            Created on: 26/Apr/18 17:25
            Start Date: 26/Apr/18 17:25
    Worklog Time Spent: 10m 
      Work Description: pupamanyu 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_r184469631
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
 ##########
 @@ -360,32 +387,85 @@
      * of how the partitions are distributed among the splits.
      */
     public Read<K, V> withTopicPartitions(List<TopicPartition> 
topicPartitions) {
-      checkState(getTopics().isEmpty(), "Only topics or topicPartitions can be 
set, not both");
-      return 
toBuilder().setTopicPartitions(ImmutableList.copyOf(topicPartitions)).build();
+      return 
withTopicPartitions(ValueProvider.StaticValueProvider.of(topicPartitions));
+    }
+
+    /**
+     * Like above but with a {@link ValueProvider 
ValueProvider&lt;List&lt;TopicPartition&gt;&gt;}.
+     *
+     */
+    public Read<K, V> withTopicPartitions(ValueProvider<List<TopicPartition>> 
topicPartitions) {
+      checkArgument(
+          getTopics().isAccessible(),
+          "Only topics or topicPartitions can be set, not both");
+      return toBuilder().setTopicPartitions(topicPartitions).build();
     }
 
     /**
-     * Sets a Kafka {@link Deserializer} to interpret key bytes read from 
Kafka.
+     * Sets a Kafka {@link Deserializer Deserializer&lt;K&gt;} for 
interpreting key bytes read.
+     * This uses the {@link String} provided to set the Deserializer
+     */
+    public Read<K, V> withKeyDeserializerClassName(String keyDeserializer) {
+      return 
withKeyDeserializerClassName(ValueProvider.StaticValueProvider.of(keyDeserializer));
+    }
+
+    /**
+     * Like above but with a {@link ValueProvider ValueProvider&lt;String&gt;}.
+     */
+    public Read<K, V> withKeyDeserializerClassName(ValueProvider<String> 
keyDeserializer) {
+      return toBuilder().setKeyDeserializer(ValueProvider
+              .NestedValueProvider.of(keyDeserializer, new 
KeyDeserializerTranslator())).build();
+    }
+
+    /**
+     * Sets a Kafka {@link Deserializer Deserializer&lt;K&gt;} to interpret 
key bytes read.
      *
      * <p>In addition, Beam also needs a {@link Coder} to serialize and 
deserialize key objects at
      * runtime. KafkaIO tries to infer a coder for the key based on the {@link 
Deserializer} class,
      * however in case that fails, you can use {@link 
#withKeyDeserializerAndCoder(Class, Coder)} to
      * provide the key coder explicitly.
      */
-    public Read<K, V> withKeyDeserializer(Class<? extends Deserializer<K>> 
keyDeserializer) {
-      return toBuilder().setKeyDeserializer(keyDeserializer).build();
+    public Read<K, V> withKeyDeserializerClassName(
 
 Review comment:
   It was requested to be changed as part of the review. 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 95653)
    Time Spent: 5h  (was: 4h 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: 5h
>  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