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

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

                Author: ASF GitHub Bot
            Created on: 03/Feb/20 13:53
            Start Date: 03/Feb/20 13:53
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on pull request #10563: [BEAM-7310] 
Add support of Confluent Schema Registry for KafkaIO
URL: https://github.com/apache/beam/pull/10563#discussion_r374101759
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
 ##########
 @@ -1784,4 +1966,75 @@ void shutdown() {
       }
     }
   }
+
+  private static class AvroDeSe {
+    private static Serializer<AvroGeneratedUser> getSerializer(
+        boolean isKey, String schemaRegistryUrl) {
+      SchemaRegistryClient registryClient = new MockSchemaRegistryClient();
+      Map<String, Object> map = new HashMap<>();
+      map.put(KafkaAvroDeserializerConfig.AUTO_REGISTER_SCHEMAS, true);
+      map.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, 
schemaRegistryUrl);
+      Serializer<AvroGeneratedUser> serializer =
+          (Serializer) new KafkaAvroSerializer(registryClient);
+      serializer.configure(map, isKey);
+      return serializer;
+    }
+
+    private static Deserializer<AvroGeneratedUser> getDeserializer(
+        boolean key, String schemaRegistryUrl) {
+      SchemaRegistryClient registryClient = new MockSchemaRegistryClient();
+      Map<String, Object> map = new HashMap<>();
+      map.put(KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG, "true");
+      map.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, 
schemaRegistryUrl);
+      Deserializer<AvroGeneratedUser> deserializer =
+          (Deserializer) new KafkaAvroDeserializer(registryClient);
+      deserializer.configure(map, key);
+      return deserializer;
+    }
+  }
+
+  private abstract static class BaseAvroSerializableFunction
+      implements SerializableFunction<Integer, byte[]> {
+    static transient Serializer<AvroGeneratedUser> serializer = null;
+    final String topic;
+    final String schemaRegistryUrl;
+    final boolean isKey;
+
+    BaseAvroSerializableFunction(String topic, String schemaRegistryUrl, 
boolean isKey) {
+      this.topic = topic;
+      this.schemaRegistryUrl = schemaRegistryUrl;
+      this.isKey = isKey;
+    }
+
+    static Serializer<AvroGeneratedUser> getSerializer(boolean isKey, String 
schemaRegistryUrl) {
+      if (serializer == null) {
+        serializer = AvroDeSe.getSerializer(isKey, schemaRegistryUrl);
+      }
+      return serializer;
+    }
+  }
+
+  private static class KeyAvroSerializableFunction extends 
BaseAvroSerializableFunction {
+    KeyAvroSerializableFunction(String topic, String schemaRegistryUrl, 
boolean isKey) {
+      super(topic, schemaRegistryUrl, isKey);
+    }
+
+    @Override
+    public byte[] apply(Integer i) {
+      return getSerializer(isKey, schemaRegistryUrl)
+          .serialize(topic, new AvroGeneratedUser("KeyName" + i, i, "color" + 
i));
+    }
+  }
+
+  private static class ValueAvroSerializableFunction extends 
BaseAvroSerializableFunction {
+    ValueAvroSerializableFunction(String topic, String schemaRegistryUrl, 
boolean isKey) {
 
 Review comment:
   remove `isKey` and fix it in the call to super to `false`
 
----------------------------------------------------------------
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: 380966)

> 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: 11.5h
>  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