You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/10/29 21:16:01 UTC

[GitHub] [beam] TheNeuralBit commented on a change in pull request #12838: [BEAM-10892] Add Proto support to Kafka Table Provider

TheNeuralBit commented on a change in pull request #12838:
URL: https://github.com/apache/beam/pull/12838#discussion_r514555185



##########
File path: sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderProtoIT.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.extensions.sql.meta.provider.kafka;
+
+import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+public class KafkaTableProviderProtoIT extends KafkaTableProviderIT {
+  private final SimpleFunction<Row, byte[]> toBytesFn =
+      ProtoMessageSchema.getRowToProtoBytesFn(KafkaMessages.ItMessage.class);
+
+  @Override
+  protected ProducerRecord<String, byte[]> generateProducerRecord(int i) {
+    return new ProducerRecord<>(
+        kafkaOptions.getKafkaTopic(), "k" + i, toBytesFn.apply(generateRow(i)));

Review comment:
       Please generate the proto directly rather than using `getRowToProtoBytesFn`

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaProtoTable.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.extensions.sql.meta.provider.kafka;
+
+import com.google.protobuf.Message;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
+import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+public class BeamKafkaProtoTable<ProtoT extends Message> extends BeamKafkaTable {
+  private final transient Class<ProtoT> protoClass;
+
+  public BeamKafkaProtoTable(
+      Schema beamSchema, String bootstrapServers, List<String> topics, Class<ProtoT> protoClass) {
+    super(beamSchema, bootstrapServers, topics);
+    this.protoClass = protoClass;

Review comment:
       Nice!
   
   I think we should still handle the situation where a schema is specified though, and raise an error if the schema inferred from the proto class is not [equivalent](https://github.com/apache/beam/blob/7828af87e55a1311fcd0a76070713b4eac8b2c4c/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L343) to the one the user specified.

##########
File path: sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableProtoTest.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.extensions.sql.meta.provider.kafka;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.List;
+import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.Test;
+
+public class BeamKafkaTableProtoTest extends BeamKafkaTableTest {
+  private static final ProtoCoder<KafkaMessages.TestMessage> PROTO_CODER =
+      ProtoCoder.of(KafkaMessages.TestMessage.class);
+
+  private static final Schema TEST_SCHEMA =
+      Schema.builder()
+          .addNullableField("f_long", Schema.FieldType.INT64)
+          .addNullableField("f_int", Schema.FieldType.INT32)
+          .addNullableField("f_double", Schema.FieldType.DOUBLE)
+          .addNullableField("f_string", Schema.FieldType.STRING)
+          .addNullableField("f_float_array", Schema.FieldType.array(Schema.FieldType.FLOAT))
+          .build();
+
+  private static final Schema SHUFFLED_SCHEMA =
+      Schema.builder()
+          .addNullableField("f_string", Schema.FieldType.STRING)
+          .addNullableField("f_int", Schema.FieldType.INT32)
+          .addNullableField("f_float_array", Schema.FieldType.array(Schema.FieldType.FLOAT))
+          .addNullableField("f_double", Schema.FieldType.DOUBLE)
+          .addNullableField("f_long", Schema.FieldType.INT64)
+          .build();
+
+  @Test
+  public void testWithShuffledSchema() {
+    BeamKafkaTable kafkaTable = getBeamKafkaTable();
+    PCollection<Row> result =
+        pipeline
+            .apply(Create.of(shuffledRow(1), shuffledRow(2)))
+            .apply(kafkaTable.getPTransformForOutput())
+            .apply(kafkaTable.getPTransformForInput());
+    PAssert.that(result).containsInAnyOrder(generateRow(1), generateRow(2));
+    pipeline.run();
+  }
+
+  @Override
+  protected BeamKafkaTable getBeamKafkaTable() {
+    return new BeamKafkaProtoTable("", ImmutableList.of(), KafkaMessages.TestMessage.class);
+  }
+
+  @Override
+  protected Row generateRow(int i) {
+    List<Object> values =
+        ImmutableList.of((long) i, i, (double) i, "proto_value" + i, ImmutableList.of((float) i));
+    return Row.withSchema(TEST_SCHEMA).addValues(values).build();
+  }
+
+  @Override
+  protected byte[] generateEncodedPayload(int i) {
+    KafkaMessages.TestMessage message =
+        KafkaMessages.TestMessage.newBuilder()
+            .setFLong(i)
+            .setFInt(i)
+            .setFDouble(i)
+            .setFString("proto_value" + i)
+            .addFFloatArray((float) i)
+            .build();
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    try {
+      PROTO_CODER.encode(message, out);

Review comment:
       I think you can just serialize the message rather than relying on `ProtoCoder`, you should just need something like `message.writeTo(out)`

##########
File path: sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderProtoIT.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.extensions.sql.meta.provider.kafka;
+
+import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+public class KafkaTableProviderProtoIT extends KafkaTableProviderIT {
+  private final SimpleFunction<Row, byte[]> toBytesFn =
+      ProtoMessageSchema.getRowToProtoBytesFn(KafkaMessages.ItMessage.class);
+
+  @Override
+  protected ProducerRecord<String, byte[]> generateProducerRecord(int i) {
+    return new ProducerRecord<>(
+        kafkaOptions.getKafkaTopic(), "k" + i, toBytesFn.apply(generateRow(i)));

Review comment:
       It looks like we do a similar thing in KafkaTableProviderAvroIT, that should also create the Avro direcly




----------------------------------------------------------------
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:
users@infra.apache.org