fozzie15 commented on code in PR #35644:
URL: https://github.com/apache/beam/pull/35644#discussion_r2333798157


##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java:
##########
@@ -198,36 +230,70 @@ public PCollectionRowTuple expand(PCollectionRowTuple 
input) {
           throw new IllegalArgumentException(
               "At least a descriptorPath or a proto Schema is required.");
         }
-
       } else {
-        toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema);
+        if (configuration.getProducerConfigUpdates() != null
+            && 
configuration.getProducerConfigUpdates().containsKey("schema.registry.url")) {
+          toRecordsFn = AvroUtils.getRowToGenericRecordFunction(avroSchema);
+          toBytesFn = null;
+        } else {
+          toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema);
+        }
       }
 
       boolean handleErrors = 
ErrorHandling.hasOutput(configuration.getErrorHandling());
       final Map<String, String> configOverrides = 
configuration.getProducerConfigUpdates();
       Schema errorSchema = ErrorHandling.errorSchema(inputSchema);
-      PCollectionTuple outputTuple =
-          input
-              .get("input")
-              .apply(
-                  "Map rows to Kafka messages",
-                  ParDo.of(
-                          new ErrorCounterFn(
-                              "Kafka-write-error-counter", toBytesFn, 
errorSchema, handleErrors))
-                      .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG)));
-
-      outputTuple
-          .get(OUTPUT_TAG)
-          .apply(
-              KafkaIO.<byte[], byte[]>write()
-                  .withTopic(configuration.getTopic())
-                  .withBootstrapServers(configuration.getBootstrapServers())
-                  .withProducerConfigUpdates(
-                      configOverrides == null
-                          ? new HashMap<>()
-                          : new HashMap<String, Object>(configOverrides))
-                  .withKeySerializer(ByteArraySerializer.class)
-                  .withValueSerializer(ByteArraySerializer.class));
+      PCollectionTuple outputTuple;
+      if (toRecordsFn != null) {
+        LOG.info("Convert to GenericRecord");
+        outputTuple =
+            input
+                .get("input")
+                .apply(
+                    "Map rows to Kafka messages",
+                    ParDo.of(
+                            new RecordErrorCounterFn(
+                                "Kafka-write-error-counter",
+                                toRecordsFn,
+                                errorSchema,
+                                handleErrors))
+                        .withOutputTags(RECORD_OUTPUT_TAG, 
TupleTagList.of(ERROR_TAG)));
+        HashMap<String, Object> producerConfig = new 
HashMap<>(configOverrides);
+        outputTuple
+            .get(RECORD_OUTPUT_TAG)
+            .setCoder(KvCoder.of(ByteArrayCoder.of(), 
AvroCoder.of(avroSchema)))
+            .apply(
+                "Map Rows to GenericRecords",
+                KafkaIO.<byte[], GenericRecord>write()
+                    .withTopic(configuration.getTopic())
+                    .withBootstrapServers(configuration.getBootstrapServers())
+                    .withProducerConfigUpdates(producerConfig)
+                    .withKeySerializer(ByteArraySerializer.class)
+                    .withValueSerializer((Class) KafkaAvroSerializer.class));

Review Comment:
   Yes without the cast there is a compiler error that prevents execution. The 
class is not recognized as the correct type without the cast.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to