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


##########
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:
   good idea. done!



##########
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:
   great point. done



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