piotr-szuberski commented on a change in pull request #12838: URL: https://github.com/apache/beam/pull/12838#discussion_r515873264
########## File path: sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableProtoTest.java ########## @@ -0,0 +1,120 @@ +/* + * 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 static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +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 Schema TEST_SCHEMA = + Schema.builder() + .addInt64Field("f_long") + .addInt32Field("f_int") + .addDoubleField("f_double") + .addStringField("f_string") + .addArrayField("f_float_array", Schema.FieldType.FLOAT) + .build(); + + private static final Schema SHUFFLED_SCHEMA = + Schema.builder() + .addStringField("f_string") + .addInt32Field("f_int") + .addArrayField("f_float_array", Schema.FieldType.FLOAT) + .addDoubleField("f_double") + .addInt64Field("f_long") + .build(); + + @Test + public void testWithShuffledSchema() { + BeamKafkaTable kafkaTable = + new BeamKafkaProtoTable( + SHUFFLED_SCHEMA, "", ImmutableList.of(), KafkaMessages.TestMessage.class); + + 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(); + } + + @Test + public void testSchemasDoNotMatch() { + Schema schema = Schema.builder().addStringField("non_existing_field").build(); + + IllegalArgumentException e = + assertThrows( + IllegalArgumentException.class, + () -> + new BeamKafkaProtoTable( + schema, "", ImmutableList.of(), KafkaMessages.TestMessage.class)); + + assertThat( + e.getMessage(), + containsString("does not match schema inferred from protobuf class. Protobuf class: ")); + } + + @Override + protected BeamKafkaTable getBeamKafkaTable() { + return new BeamKafkaProtoTable( + TEST_SCHEMA, "", 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) throws IOException { + KafkaMessages.TestMessage message = + KafkaMessages.TestMessage.newBuilder() + .setFLong(i) + .setFInt(i) + .setFDouble(i) + .setFString("proto_value" + i) + .addFFloatArray((float) i) + .build(); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + message.writeDelimitedTo(out); Review comment: I thought that proto coder uses what is preferable, thanks for explanation! Done. ########## File path: sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java ########## @@ -115,6 +122,76 @@ public SchemaUserTypeCreator schemaTypeCreator(Class<?> targetClass, Schema sche return creator; } + public static <T extends Message> SimpleFunction<byte[], Row> getProtoBytesToRowFn( + Class<T> clazz) { + return new ProtoBytesToRowFn<>(clazz); + } + + public static class ProtoBytesToRowFn<T extends Message> extends SimpleFunction<byte[], Row> { + private final ProtoCoder<T> protoCoder; + private final SerializableFunction<T, Row> toRowFunction; + + public ProtoBytesToRowFn(Class<T> clazz) { + this.protoCoder = ProtoCoder.of(clazz); + this.toRowFunction = new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(clazz)); + } + + @Override + public Row apply(byte[] bytes) { + try { + InputStream inputStream = new ByteArrayInputStream(bytes); + T message = protoCoder.decode(inputStream); + return toRowFunction.apply(message); + } catch (IOException e) { + throw new IllegalArgumentException("Could not decode row from proto payload.", e); + } + } + } + + public static <T extends Message> SimpleFunction<Row, byte[]> getRowToProtoBytesFn( + Class<T> clazz) { + return new RowToProtoBytesFn<>(clazz); + } + + public static class RowToProtoBytesFn<T extends Message> extends SimpleFunction<Row, byte[]> { + private final ProtoCoder<T> protoCoder; + private final SerializableFunction<Row, T> toMessageFunction; + private final Class<T> clazz; + private final Schema protoSchema; + + public RowToProtoBytesFn(Class<T> clazz) { + ProtoMessageSchema messageSchema = new ProtoMessageSchema(); + TypeDescriptor<T> typeDescriptor = TypeDescriptor.of(clazz); + this.clazz = clazz; + this.protoCoder = ProtoCoder.of(typeDescriptor); + this.toMessageFunction = messageSchema.fromRowFunction(typeDescriptor); + this.protoSchema = messageSchema.schemaFor(typeDescriptor); + } + + @Override + public byte[] apply(Row row) { + if (!protoSchema.equivalent(row.getSchema())) { + row = switchFieldsOrder(row); + } + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + try { + Message message = toMessageFunction.apply(row); + protoCoder.encode(clazz.cast(message), outputStream); Review comment: Done. ########## File path: sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java ########## @@ -115,6 +122,76 @@ public SchemaUserTypeCreator schemaTypeCreator(Class<?> targetClass, Schema sche return creator; } + public static <T extends Message> SimpleFunction<byte[], Row> getProtoBytesToRowFn( + Class<T> clazz) { + return new ProtoBytesToRowFn<>(clazz); + } + + public static class ProtoBytesToRowFn<T extends Message> extends SimpleFunction<byte[], Row> { + private final ProtoCoder<T> protoCoder; + private final SerializableFunction<T, Row> toRowFunction; + + public ProtoBytesToRowFn(Class<T> clazz) { + this.protoCoder = ProtoCoder.of(clazz); + this.toRowFunction = new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(clazz)); + } + + @Override + public Row apply(byte[] bytes) { + try { + InputStream inputStream = new ByteArrayInputStream(bytes); + T message = protoCoder.decode(inputStream); + return toRowFunction.apply(message); + } catch (IOException e) { + throw new IllegalArgumentException("Could not decode row from proto payload.", e); + } + } + } + + public static <T extends Message> SimpleFunction<Row, byte[]> getRowToProtoBytesFn( + Class<T> clazz) { + return new RowToProtoBytesFn<>(clazz); + } + + public static class RowToProtoBytesFn<T extends Message> extends SimpleFunction<Row, byte[]> { Review comment: Done. ########## File path: sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java ########## @@ -115,6 +122,76 @@ public SchemaUserTypeCreator schemaTypeCreator(Class<?> targetClass, Schema sche return creator; } + public static <T extends Message> SimpleFunction<byte[], Row> getProtoBytesToRowFn( + Class<T> clazz) { + return new ProtoBytesToRowFn<>(clazz); + } + + public static class ProtoBytesToRowFn<T extends Message> extends SimpleFunction<byte[], Row> { + private final ProtoCoder<T> protoCoder; + private final SerializableFunction<T, Row> toRowFunction; + + public ProtoBytesToRowFn(Class<T> clazz) { + this.protoCoder = ProtoCoder.of(clazz); + this.toRowFunction = new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(clazz)); + } + + @Override + public Row apply(byte[] bytes) { + try { + InputStream inputStream = new ByteArrayInputStream(bytes); + T message = protoCoder.decode(inputStream); + return toRowFunction.apply(message); + } catch (IOException e) { + throw new IllegalArgumentException("Could not decode row from proto payload.", e); + } + } + } + + public static <T extends Message> SimpleFunction<Row, byte[]> getRowToProtoBytesFn( + Class<T> clazz) { + return new RowToProtoBytesFn<>(clazz); + } + + public static class RowToProtoBytesFn<T extends Message> extends SimpleFunction<Row, byte[]> { Review comment: The `get*Fn` wants to receive Class<T extends Message> which cannot be provided from BeamKafkaProtoTable as it's not allowed to import from `com.google.protobuf`. I'll move the suppresion to ProtoMessageSchema and add a validation if the provided class extends Message. If you know a cleaner solution then I'll gladly change it. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaProtoTable.java ########## @@ -0,0 +1,117 @@ +/* + * 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.util.List; +import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; +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 extends BeamKafkaTable { + private final Class<?> protoClass; + + public BeamKafkaProtoTable( + Schema messageSchema, String bootstrapServers, List<String> topics, Class<?> protoClass) { + super(inferAndVerifySchema(protoClass, messageSchema), bootstrapServers, topics); + this.protoClass = protoClass; + } + + @Override + public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> getPTransformForInput() { + return new ProtoRecorderDecoder(schema, protoClass); + } + + @Override + public PTransform<PCollection<Row>, PCollection<KV<byte[], byte[]>>> getPTransformForOutput() { + return new ProtoRecorderEncoder(protoClass); + } + + private static Schema inferAndVerifySchema(Class<?> protoClass, Schema messageSchema) { + Schema inferredSchema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(protoClass)); + if (!messageSchema.equivalent(inferredSchema)) { + throw new IllegalArgumentException( + String.format( + "Given message schema '%s' does not match schema inferred from protobuf class. Protobuf class: '%s' Inferred schema: '%s'", + messageSchema, protoClass.getCanonicalName(), inferredSchema)); + } + return inferredSchema; + } + + /** A PTransform to convert {@code KV<byte[], byte[]>} to {@link Row}. */ + private static class ProtoRecorderDecoder + extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> { + private final Schema schema; + private final Class<?> clazz; + + ProtoRecorderDecoder(Schema schema, Class<?> clazz) { + this.schema = schema; + this.clazz = clazz; + } + + @Override + public PCollection<Row> expand(PCollection<KV<byte[], byte[]>> input) { + // We are not allowed to use non-vendored protobuf Message here to extend the wildcard + @SuppressWarnings({"unchecked", "rawtypes"}) Review comment: Unfortunately they are. I didn't find a solution to get rid of that. I moved the class check to ProtoMessageSchema - more below. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaProtoTable.java ########## @@ -0,0 +1,117 @@ +/* + * 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.util.List; +import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; +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 extends BeamKafkaTable { + private final Class<?> protoClass; + + public BeamKafkaProtoTable( + Schema messageSchema, String bootstrapServers, List<String> topics, Class<?> protoClass) { + super(inferAndVerifySchema(protoClass, messageSchema), bootstrapServers, topics); + this.protoClass = protoClass; + } + + @Override + public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> getPTransformForInput() { + return new ProtoRecorderDecoder(schema, protoClass); + } + + @Override + public PTransform<PCollection<Row>, PCollection<KV<byte[], byte[]>>> getPTransformForOutput() { + return new ProtoRecorderEncoder(protoClass); + } + + private static Schema inferAndVerifySchema(Class<?> protoClass, Schema messageSchema) { + Schema inferredSchema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(protoClass)); + if (!messageSchema.equivalent(inferredSchema)) { + throw new IllegalArgumentException( + String.format( + "Given message schema '%s' does not match schema inferred from protobuf class. Protobuf class: '%s' Inferred schema: '%s'", + messageSchema, protoClass.getCanonicalName(), inferredSchema)); + } + return inferredSchema; + } + + /** A PTransform to convert {@code KV<byte[], byte[]>} to {@link Row}. */ + private static class ProtoRecorderDecoder + extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> { + private final Schema schema; + private final Class<?> clazz; + + ProtoRecorderDecoder(Schema schema, Class<?> clazz) { + this.schema = schema; + this.clazz = clazz; + } + + @Override + public PCollection<Row> expand(PCollection<KV<byte[], byte[]>> input) { + // We are not allowed to use non-vendored protobuf Message here to extend the wildcard + @SuppressWarnings({"unchecked", "rawtypes"}) Review comment: Unfortunately they are. I haven't found a solution to get rid of that. I moved the class check to ProtoMessageSchema - more below. ---------------------------------------------------------------- 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: [email protected]
