[
https://issues.apache.org/jira/browse/BEAM-3925?focusedWorklogId=93365&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-93365
]
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_r183139657
##########
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<String>}.
+ */
+ 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<String>}.
+ */
+ 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<TopicPartition>}.
+ */
+ 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<K>} for
interpreting key bytes read.
+ * This uses the {@link String} provided to set the Deserializer
+ */
+ public Read<K, V> withKeyDeserializer(String keyDeserializer) {
+ return
withKeyDeserializer(ValueProvider.StaticValueProvider.of(keyDeserializer));
+ }
+
+ /**
+ * Like above but with a {@link ValueProvider ValueProvider<String>}.
+ */
+ public Read<K, V> withKeyDeserializer(ValueProvider<String>
keyDeserializer) {
+ return toBuilder().setKeyDeserializer(ValueProvider
+ .NestedValueProvider.of(keyDeserializer, new
KeyDeserializerTranslator())).build();
}
/**
- * Sets a Kafka {@link Deserializer} to interpret key bytes read from
Kafka.
+ * Sets a Kafka {@link Deserializer Deserializer<K>} 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();
+ return toBuilder().setKeyDeserializer(
+ ValueProvider.StaticValueProvider.of(keyDeserializer)).build();
+ }
+
+ /**
+ * Used to build a {@link ValueProvider} for {@link Deserializer
Deserializer<K>}.
+ */
+ private class KeyDeserializerTranslator
+ implements SerializableFunction<String, Class<? extends
Deserializer<K>>> {
+ @SuppressWarnings("unchecked")
+ @Override
+ public Class apply(String deserializer) {
+ Class deserializerClass;
+ try {
+ deserializerClass = Class.forName(deserializer);
+ } catch (ClassNotFoundException e) {
+ LOG.error("Provided class for KeySerializer not found {}",
deserializer);
Review comment:
The log message is redundant with the exception below - you can remove the
log message.
----------------------------------------------------------------
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: 93365)
> 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 20m
> 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)