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

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

                Author: ASF GitHub Bot
            Created on: 24/Jan/20 16:05
            Start Date: 24/Jan/20 16:05
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on pull request #10563: 
[BEAM-7310] Add support of Confluent Schema Registry for KafkaIO
URL: https://github.com/apache/beam/pull/10563#discussion_r370712008
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
 ##########
 @@ -891,6 +1014,50 @@ public void setValueDeserializer(String 
valueDeserializer) {
       return input.getPipeline().apply(transform);
     }
 
+    private Coder<K> getKeyCoder(CoderRegistry coderRegistry, String 
schemaRegistryURL) {
+      Schema avroKeySchema = null;
+      if (schemaRegistryURL != null && getKeySchemaSubject() != null) {
+        avroKeySchema = fetchAvroSchema(schemaRegistryURL, 
getKeySchemaSubject());
+        checkArgument(avroKeySchema != null, "Avro key schema can't be null");
+      }
+
+      Coder<K> keyCoder;
+      if (avroKeySchema != null) {
+        keyCoder = (Coder<K>) AvroCoder.of(avroKeySchema);
+      } else {
+        keyCoder =
+            getKeyCoder() != null ? getKeyCoder() : inferCoder(coderRegistry, 
getKeyDeserializer());
+      }
+      checkState(
+          keyCoder != null,
+          "Key coder could not be inferred from key deserializer. Please 
provide"
+              + "key coder explicitly using withKeyDeserializerAndCoder()");
+      return keyCoder;
+    }
+
+    private Coder<V> getValueCoder(CoderRegistry coderRegistry, String 
schemaRegistryURL) {
+      Schema avroValueSchema = null;
+      if (schemaRegistryURL != null && getValueSchemaSubject() != null) {
+        avroValueSchema = fetchAvroSchema(schemaRegistryURL, 
getValueSchemaSubject());
+        checkArgument(avroValueSchema != null, "Avro value schema can't be 
null");
+      }
+
+      Coder<V> valueCoder;
+      if (avroValueSchema != null) {
+        valueCoder = (Coder<V>) AvroCoder.of(avroValueSchema);
+      } else {
+        valueCoder =
+            getValueCoder() != null
+                ? getValueCoder()
+                : inferCoder(coderRegistry, getValueDeserializer());
+      }
+      checkState(
+          valueCoder != null,
+          "Value coder could not be inferred from value deserializer. Please 
provide"
+              + "value coder explicitly using 
withValueDeserializerAndCoder()");
+      return valueCoder;
+    }
 
 Review comment:
   I planned to do this in the next PR for 
https://issues.apache.org/jira/browse/BEAM-7336
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 376886)
    Time Spent: 9h 40m  (was: 9.5h)

> Confluent Schema Registry support in KafkaIO
> --------------------------------------------
>
>                 Key: BEAM-7310
>                 URL: https://issues.apache.org/jira/browse/BEAM-7310
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-kafka
>    Affects Versions: 2.12.0
>            Reporter: Yohei Shimomae
>            Assignee: Alexey Romanenko
>            Priority: Minor
>          Time Spent: 9h 40m
>  Remaining Estimate: 0h
>
> Confluent Schema Registry is useful when we manage Avro Schema but  KafkaIO 
> does not support Confluent Schema Registry as discussed here.
> https://stackoverflow.com/questions/56035121/unable-to-connect-from-dataflow-job-to-schema-registry-when-schema-registry-requ
> https://lists.apache.org/thread.html/7695fccddebd08733b80ae1e43b79b636b63cd5fe583a2bdeecda6c4@%3Cuser.beam.apache.org%3E



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to