[
https://issues.apache.org/jira/browse/BEAM-6063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16686971#comment-16686971
]
Alexey Romanenko commented on BEAM-6063:
----------------------------------------
[~rangadi] What do you think if I will change Write transform to {{Write<K, V>
extends PTransform<PCollection<ProducerRecord<K, V>>, PDone>}} and will
introduce new transform {{WriteKV<K, V> extends PTransform<PCollection<KV<K,
V>>, PDone>}} which will be using Write under the hood? It will allow us to use
{{ProducerRecord}} all over Write transform instead of limited {{KV}}
So, the new {{write()}} function will be look like this:
{code:java}
<K, V> WriteKV<K, V> write() {
return new AutoValue_KafkaIO_WriteKV.Builder<K, V>().setWriteTransform(
new AutoValue_KafkaIO_Write.Builder<K, V>()
.setFoo()
...
.build()).build();
}
{code}
The return type (WriteKV instead of Write) will be the only change in API.
> KafkaIO: add writing support for ProducerRecord
> -----------------------------------------------
>
> Key: BEAM-6063
> URL: https://issues.apache.org/jira/browse/BEAM-6063
> Project: Beam
> Issue Type: Improvement
> Components: io-java-kafka
> Reporter: Alexey Romanenko
> Assignee: Alexey Romanenko
> Priority: Major
>
> Currently, the default input collection for {{KafkaIO.Write}} is
> {{PCollection<KV<K,V>>}}. To support writing of Kafka headers or different
> output Kafka topics, we need to change type of input collection to
> {{PCollection<ProducerRecord<K,V>>}}. Also, it will make sense to use
> {{ProducerRecord<K,V>}} instead of {{KV<K,V>}} internally in {{KafkaIO}} to
> keep all meta information.
> In the same time, we need to keep compatibility for old interface based on
> {{KV<K,V>}} but make it deprecated and totally move to
> {{ProducerRecord<K,V>}} later.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)