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


##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java:
##########
@@ -161,13 +171,35 @@ public void finish() {
       }
     }
 
+    public static class ErrorCounterFn extends BaseKafkaWriterFn<byte[]> {
+      public ErrorCounterFn(
+          String name,
+          SerializableFunction<Row, byte[]> toBytesFn,
+          Schema errorSchema,
+          boolean handleErrors) {
+        super(name, toBytesFn, errorSchema, handleErrors, OUTPUT_TAG);
+      }
+    }
+
+    public static class RecordErrorCounterFn extends 
BaseKafkaWriterFn<GenericRecord> {
+      public RecordErrorCounterFn(
+          String name,
+          SerializableFunction<Row, GenericRecord> toRecordsFn,
+          Schema errorSchema,
+          boolean handleErrors) {
+        super(name, toRecordsFn, errorSchema, handleErrors, RECORD_OUTPUT_TAG);
+      }
+    }
+
     @SuppressWarnings({
       "nullness" // TODO(https://github.com/apache/beam/issues/20497)
     })
     @Override
     public PCollectionRowTuple expand(PCollectionRowTuple input) {
       Schema inputSchema = input.get("input").getSchema();
+      org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(inputSchema);
       final SerializableFunction<Row, byte[]> toBytesFn;
+      SerializableFunction<Row, GenericRecord> toRecordsFn = null;

Review Comment:
   changed



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java:
##########
@@ -161,13 +171,35 @@ public void finish() {
       }
     }
 
+    public static class ErrorCounterFn extends BaseKafkaWriterFn<byte[]> {
+      public ErrorCounterFn(
+          String name,
+          SerializableFunction<Row, byte[]> toBytesFn,
+          Schema errorSchema,
+          boolean handleErrors) {
+        super(name, toBytesFn, errorSchema, handleErrors, OUTPUT_TAG);
+      }
+    }
+
+    public static class RecordErrorCounterFn extends 
BaseKafkaWriterFn<GenericRecord> {

Review Comment:
   changed



##########
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");

Review Comment:
   added



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