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

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

                Author: ASF GitHub Bot
            Created on: 25/Apr/18 17:35
            Start Date: 25/Apr/18 17:35
    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_r184146288
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
 ##########
 @@ -359,20 +386,129 @@
      * 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(Joiner.on(',').join(topicPartitions)));
+    }
+
+    /**
+     * Sets a list of partitions to read from. Partitions are provided as a 
comma separated list of
+     * Strings in the format: topic1-partition1,topic1-partition2...
+     * This allows reading only a subset of partitions for one or more topics 
when (if ever) needed.
+     *
+     * <p>See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for 
description
+     * of how the partitions are distributed among the splits.
+     */
+    public Read<K, V> withTopicPartitions(String topicPartitions) {
+      return 
withTopicPartitions(ValueProvider.StaticValueProvider.of(topicPartitions));
+    }
+
+    /**
+     * Like above but with a {@link ValueProvider ValueProvider&lt;String&gt;}.
+     */
+    public Read<K, V> withTopicPartitions(ValueProvider<String> 
topicPartitions) {
+      checkState(getTopics().get().isEmpty(),
+          "Only topics or topicPartitions can be set, not both");
+      return toBuilder().setTopicPartitions(ValueProvider
+              .NestedValueProvider.of(topicPartitions, new 
TopicPartitionTranslator())).build();
+    }
+
+    /**
+     * Used to build a {@link ValueProvider} for {@link List 
List&lt;String&gt;}.
+     */
+    private static class TopicTranslator implements 
SerializableFunction<String, List<String>> {
+      @Override
+      public List<String> apply(String topics) {
+        return ImmutableList.copyOf(
+            
Splitter.on(',').trimResults().omitEmptyStrings().splitToList(topics));
+      }
+
+    }
+
+    /**
+     * Used to build a {@link ValueProvider} for {@link List 
List&lt;TopicPartition&gt;}.
+     */
+    private static class TopicPartitionTranslator
+        implements SerializableFunction<String, List<TopicPartition>> {
+      @Override
+      public List<TopicPartition> apply(String topicPartitions) {
+        List<TopicPartition> topicPartitionList = new ArrayList<>();
+        for (String topicPartition: 
Splitter.on(',').trimResults().omitEmptyStrings()
+              .splitToList(topicPartitions)) {
+          topicPartitionList
+              .add(new 
TopicPartition(Splitter.on('-').splitToList(topicPartition).get(0),
+                  
Integer.parseInt(Splitter.on('-').splitToList(topicPartition).get(1))));
+        }
+      return ImmutableList.copyOf(topicPartitionList);
+      }
+    }
+
+    /**
+     * 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> withKeyDeserializer(String keyDeserializer) {
 
 Review comment:
   I have renamed withKeyDeserializerClass to withKeyDeserializerClassName

----------------------------------------------------------------
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: 95128)
    Time Spent: 2h 40m  (was: 2.5h)

> 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: 2h 40m
>  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