Abacn commented on code in PR #24495:
URL: https://github.com/apache/beam/pull/24495#discussion_r1041291087


##########
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java:
##########
@@ -502,6 +511,103 @@ public void testKafkaWithDelayedStopReadingFunction() {
         readElementMetric(readResult, NAMESPACE, READ_ELEMENT_METRIC_NAME));
   }
 
+  public static final Schema KAFKA_TOPIC_SCHEMA =
+      Schema.builder()
+          .addStringField("name")
+          .addInt64Field("userId")
+          .addInt32Field("age")
+          .addBooleanField("ageIsEven")
+          .addDoubleField("temperature")
+          .addArrayField("childrenNames", Schema.FieldType.STRING)
+          .build();
+
+  private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000;
+
+  @Test(timeout = FIVE_MINUTES_IN_MS)
+  public void testKafkaViaSchemaTransformJson() {}

Review Comment:
   Is this test supposed to be implemented in this PR? Or a "TODO" mark for 
empty test



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.kafka;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.schemas.utils.JsonUtils;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+
+public class KafkaWriteSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransformConfiguration> {
+
+  public static final Set<String> SUPPORTED_FORMATS = Sets.newHashSet("JSON", 
"AVRO");
+
+  @Override
+  protected @UnknownKeyFor @NonNull @Initialized 
Class<KafkaWriteSchemaTransformConfiguration>
+      configurationClass() {
+    return KafkaWriteSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
+      KafkaWriteSchemaTransformConfiguration configuration) {
+    if (!SUPPORTED_FORMATS.contains(configuration.getFormat())) {
+      throw new IllegalArgumentException(
+          "Format "
+              + configuration.getFormat()
+              + " is not supported. "
+              + "Supported formats are: "
+              + String.join(", ", SUPPORTED_FORMATS));
+    }
+    return new KafkaWriteSchemaTransform(configuration);
+  }
+
+  static final class KafkaWriteSchemaTransform implements SchemaTransform, 
Serializable {
+    final KafkaWriteSchemaTransformConfiguration configuration;
+
+    KafkaWriteSchemaTransform(KafkaWriteSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized PTransform<
+            @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+            @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+        buildTransform() {
+      return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+        @Override
+        public PCollectionRowTuple expand(PCollectionRowTuple input) {
+          Schema inputSchema = input.get("input").getSchema();
+          final SerializableFunction<Row, byte[]> toBytesFn =
+              configuration.getFormat().equals("JSON")
+                  ? JsonUtils.getRowToJsonBytesFunction(inputSchema)
+                  : AvroUtils.getRowToAvroBytesFunction(inputSchema);
+
+          final Map<String, String> configOverrides = 
configuration.getProducerConfigUpdates();
+          input
+              .get("input")
+              .apply(
+                  "Map Rows to Kafka Messages",
+                  MapElements.via(
+                      new SimpleFunction<Row, KV<byte[], byte[]>>(
+                          row -> KV.of(new byte[1], toBytesFn.apply(row))) {}))
+              .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));
+          return PCollectionRowTuple.empty(input.getPipeline());
+        }
+      };
+    }
+  }
+
+  @Override
+  public @UnknownKeyFor @NonNull @Initialized String identifier() {
+    return "beam:schematransform:org.apache.beam:kafka_write:v1";
+  }
+
+  @Override
+  public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull 
@Initialized String>
+      inputCollectionNames() {
+    return Collections.singletonList("input");
+  }
+
+  @Override
+  public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull 
@Initialized String>
+      outputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  @AutoValue
+  @DefaultSchema(AutoValueSchema.class)
+  public abstract static class KafkaWriteSchemaTransformConfiguration 
implements Serializable {

Review Comment:
   Have we decided the naming pattern? the design doc 
https://docs.google.com/document/d/1qW9O3VxdGxUM887TdwhD1iH9AdNbpu0_wXbCGvFP0OM/edit#
 suggested `KafkaSchemaTransformReadConfiguration` possibly need update on that 
side



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java:
##########
@@ -41,19 +41,32 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.joda.time.Duration;
 
 @AutoService(SchemaTransformProvider.class)
 public class KafkaReadSchemaTransformProvider
     extends 
TypedSchemaTransformProvider<KafkaReadSchemaTransformConfiguration> {
 
+  final Boolean isTest;
+  final Integer testTimeoutSecs;
+
+  public KafkaReadSchemaTransformProvider() {
+    this(false, 0);
+  }
+
+  KafkaReadSchemaTransformProvider(Boolean isTest, Integer testTimeoutSecs) {

Review Comment:
   Consider add a `@VisibleForTesting` annotation as this constructor is 
supposed to be used in test only?



##########
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java:
##########
@@ -502,6 +511,103 @@ public void testKafkaWithDelayedStopReadingFunction() {
         readElementMetric(readResult, NAMESPACE, READ_ELEMENT_METRIC_NAME));
   }
 
+  public static final Schema KAFKA_TOPIC_SCHEMA =
+      Schema.builder()
+          .addStringField("name")
+          .addInt64Field("userId")
+          .addInt32Field("age")
+          .addBooleanField("ageIsEven")
+          .addDoubleField("temperature")
+          .addArrayField("childrenNames", Schema.FieldType.STRING)
+          .build();
+
+  private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000;
+
+  @Test(timeout = FIVE_MINUTES_IN_MS)
+  public void testKafkaViaSchemaTransformJson() {}
+
+  @Test(timeout = FIVE_MINUTES_IN_MS)
+  public void testKafkaViaSchemaTransformAvro() {
+    runReadWriteKafkaViaSchemaTransforms(
+        "AVRO", AvroUtils.toAvroSchema(KAFKA_TOPIC_SCHEMA).toString());
+  }
+
+  public void runReadWriteKafkaViaSchemaTransforms(String format, String 
schemaDefinition) {
+    PCollectionRowTuple.of(
+            "input",
+            writePipeline
+                .apply("Generate records", GenerateSequence.from(0).to(1000))
+                .apply(
+                    "Transform to Beam Rows",
+                    MapElements.into(TypeDescriptors.rows())
+                        .via(
+                            numb ->
+                                Row.withSchema(KAFKA_TOPIC_SCHEMA)
+                                    .addValue(numb.toString()) // Name
+                                    .addValue(Long.valueOf(numb.hashCode())) 
// User ID
+                                    .addValue(numb.intValue()) // Age
+                                    .addValue(numb % 2 == 0) // ageIsEven
+                                    .addValue(new Random(numb).nextDouble())
+                                    .addValue(
+                                        Lists.newArrayList(
+                                            Long.toString(numb + 1),
+                                            Long.toString(numb + 2))) // 
childrenNames
+                                    .build()))
+                .setRowSchema(KAFKA_TOPIC_SCHEMA))
+        .apply(
+            "Write to Kafka",
+            new KafkaWriteSchemaTransformProvider()
+                .from(
+                    
KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransformConfiguration
+                        .builder()
+                        .setTopic(options.getKafkaTopic() + 
"-schema-transform")
+                        
.setBootstrapServers(options.getKafkaBootstrapServerAddresses())
+                        .setFormat(format)
+                        .build())
+                .buildTransform());
+
+    PAssert.that(
+            PCollectionRowTuple.empty(readPipeline)
+                .apply(
+                    "Read from unbounded Kafka",
+                    // A timeout of 30s for local, container-based tests, and 
2 minutes for
+                    // real-kafka tests.
+                    new KafkaReadSchemaTransformProvider(
+                            true, options.isWithTestcontainers() ? 30 : 120)
+                        .from(
+                            KafkaReadSchemaTransformConfiguration.builder()
+                                .setDataFormat(format)
+                                .setAutoOffsetResetConfig("earliest")
+                                .setSchema(schemaDefinition)
+                                .setTopic(options.getKafkaTopic() + 
"-schema-transform")
+                                
.setBootstrapServers(options.getKafkaBootstrapServerAddresses())
+                                .build())
+                        .buildTransform())
+                .get("output"))
+        .containsInAnyOrder(
+            LongStream.range(0L, 1000L)
+                .<Row>mapToObj(
+                    numb ->
+                        Row.withSchema(KAFKA_TOPIC_SCHEMA)
+                            .addValue(Long.toString(numb)) // Name
+                            
.addValue(Long.valueOf(Long.valueOf(numb).hashCode())) // User ID
+                            .addValue(Long.valueOf(numb).intValue()) // Age
+                            .addValue(numb % 2 == 0) // ageIsEven
+                            .addValue(new Random(numb).nextDouble())
+                            .addValue(
+                                Lists.newArrayList(
+                                    Long.toString(numb + 1),
+                                    Long.toString(numb + 2))) // childrenNames
+                            .build())
+                .collect(Collectors.toList()));
+
+    PipelineResult writeResult = writePipeline.run();
+    writeResult.waitUntilFinish();

Review Comment:
   Shall we assert pipeline status (not fail)? Same for the read pipeline below.



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java:
##########
@@ -68,15 +81,22 @@ public List<String> inputCollectionNames() {
 
   @Override
   public List<String> outputCollectionNames() {
-    return Lists.newArrayList("OUTPUT");
+    return Lists.newArrayList("output");

Review Comment:
   Just wondering the purpose of UPPER to lower case change. Is it some cleanup?



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