[ https://issues.apache.org/jira/browse/BEAM-6063?focusedWorklogId=166953&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-166953 ]
ASF GitHub Bot logged work on BEAM-6063: ---------------------------------------- Author: ASF GitHub Bot Created on: 16/Nov/18 17:17 Start Date: 16/Nov/18 17:17 Worklog Time Spent: 10m Work Description: aromanenko-dev commented on a change in pull request #7052: [BEAM-6063] KafkaIO: add writing support with ProducerRecord URL: https://github.com/apache/beam/pull/7052#discussion_r234284043 ########## File path: sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java ########## @@ -1122,6 +1130,182 @@ public void populateDisplayData(DisplayData.Builder builder) { } } + /** + * A {@link PTransform} to write to a Kafka topic with KVs . See {@link KafkaIO} for more + * information on usage and configuration. + */ + @AutoValue + public abstract static class Write<K, V> extends PTransform<PCollection<KV<K, V>>, PDone> { + @Nullable + abstract String getTopic(); + + abstract WriteRecords<K, V> getWriteRecordsTransform(); + + abstract Builder<K, V> toBuilder(); + + @AutoValue.Builder + abstract static class Builder<K, V> { + abstract Builder<K, V> setTopic(String topic); + + abstract Builder<K, V> setWriteRecordsTransform(WriteRecords<K, V> transform); + + abstract Write<K, V> build(); + } + + /** + * Wrapper method over {@link WriteRecords#withBootstrapServers(String)}, used to keep the + * compatibility with old API based on KV type of element. + */ + public Write<K, V> withBootstrapServers(String bootstrapServers) { + return toBuilder() + .setWriteRecordsTransform( + getWriteRecordsTransform().withBootstrapServers(bootstrapServers)) + .build(); + } + + /** + * Wrapper method over {@link WriteRecords#withTopic(String)}, used to keep the compatibility + * with old API based on KV type of element. + */ + public Write<K, V> withTopic(String topic) { + return toBuilder() + .setTopic(topic) + .setWriteRecordsTransform(getWriteRecordsTransform().withTopic(topic)) + .build(); + } + + /** + * Wrapper method over {@link WriteRecords#withKeySerializer(Class)}, used to keep the + * compatibility with old API based on KV type of element. + */ + public Write<K, V> withKeySerializer(Class<? extends Serializer<K>> keySerializer) { + return toBuilder() + .setWriteRecordsTransform(getWriteRecordsTransform().withKeySerializer(keySerializer)) + .build(); + } + + /** + * Wrapper method over {@link WriteRecords#withValueSerializer(Class)}, used to keep the + * compatibility with old API based on KV type of element. + */ + public Write<K, V> withValueSerializer(Class<? extends Serializer<V>> valueSerializer) { + return toBuilder() + .setWriteRecordsTransform(getWriteRecordsTransform().withValueSerializer(valueSerializer)) + .build(); + } + + /** + * Wrapper method over {@link WriteRecords#withProducerFactoryFn(SerializableFunction)}, used to + * keep the compatibility with old API based on KV type of element. + */ + public Write<K, V> withProducerFactoryFn( + SerializableFunction<Map<String, Object>, Producer<K, V>> producerFactoryFn) { + return toBuilder() + .setWriteRecordsTransform( + getWriteRecordsTransform().withProducerFactoryFn(producerFactoryFn)) + .build(); + } + + /** + * Wrapper method over {@link WriteRecords#withInputTimestamp()}, used to keep the compatibility + * with old API based on KV type of element. + */ + public Write<K, V> withInputTimestamp() { + return toBuilder() + .setWriteRecordsTransform(getWriteRecordsTransform().withInputTimestamp()) + .build(); + } + + /** + * Wrapper method over {@link + * WriteRecords#withPublishTimestampFunction(KafkaPublishTimestampFunction)}, used to keep the + * compatibility with old API based on KV type of element. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public Write<K, V> withPublishTimestampFunction( Review comment: Agree, I'll add this ---------------------------------------------------------------- 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: 166953) Time Spent: 1h 20m (was: 1h 10m) > KafkaIO: add writing support with 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 > Time Spent: 1h 20m > Remaining Estimate: 0h > > 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)