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

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

                Author: ASF GitHub Bot
            Created on: 25/Apr/19 17:35
            Start Date: 25/Apr/19 17:35
    Worklog Time Spent: 10m 
      Work Description: mxm commented on pull request #8322: [BEAM-7029] Add 
KafkaIO.Write as an external transform
URL: https://github.com/apache/beam/pull/8322#discussion_r278657210
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
 ##########
 @@ -1325,12 +1325,89 @@ public void populateDisplayData(DisplayData.Builder 
builder) {
     abstract Builder<K, V> toBuilder();
 
     @AutoValue.Builder
-    abstract static class Builder<K, V> {
+    abstract static class Builder<K, V>
+        implements ExternalTransformBuilder<External.Configuration, 
PCollection<KV<K, V>>, PDone> {
       abstract Builder<K, V> setTopic(String topic);
 
       abstract Builder<K, V> setWriteRecordsTransform(WriteRecords<K, V> 
transform);
 
       abstract Write<K, V> build();
+
+      @Override
+      public PTransform<PCollection<KV<K, V>>, PDone> buildExternal(
+          External.Configuration configuration) {
+        String topic = utf8String(configuration.topic);
+        setTopic(topic);
+
+        Map<String, Object> producerConfig = new HashMap<>();
+        for (KV<byte[], byte[]> kv : configuration.producerConfig) {
+          String key = utf8String(kv.getKey());
+          String value = utf8String(kv.getValue());
+          producerConfig.put(key, value);
+        }
+        Class keySerializer = 
resolveClass(utf8String(configuration.keySerializer));
+        Class valSerializer = 
resolveClass(utf8String(configuration.valueSerializer));
+
+        WriteRecords<K, V> writeRecords =
+            KafkaIO.<K, V>writeRecords()
+                .updateProducerProperties(producerConfig)
+                .withKeySerializer(keySerializer)
+                .withValueSerializer(valSerializer)
+                .withTopic(topic);
+        setWriteRecordsTransform(writeRecords);
+
+        return build();
+      }
+
+      private static Class resolveClass(String className) {
+        try {
+          return Class.forName(className);
+        } catch (ClassNotFoundException e) {
+          throw new RuntimeException("Could not find Serializer class: " + 
className);
+        }
+      }
+
+      private static String utf8String(byte[] bytes) {
 
 Review comment:
   I think it makes sense not to repeat this. Let me see if I can find 
something. `StringUtils` seems like a good place otherwise.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 233003)
    Time Spent: 12h 50m  (was: 12h 40m)

> Support KafkaIO to be configured externally for use with other SDKs
> -------------------------------------------------------------------
>
>                 Key: BEAM-7029
>                 URL: https://issues.apache.org/jira/browse/BEAM-7029
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-kafka, runner-flink, sdk-py-core
>            Reporter: Maximilian Michels
>            Assignee: Maximilian Michels
>            Priority: Major
>          Time Spent: 12h 50m
>  Remaining Estimate: 0h
>
> As of BEAM-6730, we can externally configure existing transforms from SDKs. 
> We should add more useful transforms then just {{GenerateSequence}}. 
> {{KafkaIO}} is a good candidate.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to