TheNeuralBit commented on a change in pull request #12838: URL: https://github.com/apache/beam/pull/12838#discussion_r493890826
########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaProtoTable.java ########## @@ -0,0 +1,114 @@ +/* + * 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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.apache.beam.sdk.coders.RowCoder; +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.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; + +public class BeamKafkaProtoTable extends BeamKafkaTable { + + public BeamKafkaProtoTable(Schema beamSchema, String bootstrapServers, List<String> topics) { + super(beamSchema, bootstrapServers, topics); + } + + @Override + public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> getPTransformForInput() { + return new ProtoRecorderDecoder(schema); + } + + @Override + public PTransform<PCollection<Row>, PCollection<KV<byte[], byte[]>>> getPTransformForOutput() { + return new ProtoRecorderEncoder(); + } + + /** A PTransform to convert {@code KV<byte[], byte[]>} to {@link Row}. */ + public static class ProtoRecorderDecoder + extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> { + private final Schema schema; + + public ProtoRecorderDecoder(Schema schema) { + this.schema = schema; + } + + @Override + public PCollection<Row> expand(PCollection<KV<byte[], byte[]>> input) { + return input + .apply( + "decodeProtoRecord", + ParDo.of( + new DoFn<KV<byte[], byte[]>, Row>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(parseProtoPayloadToRow(c.element().getValue(), schema)); + } + })) + .setRowSchema(schema); + } + + static Row parseProtoPayloadToRow(byte[] payload, Schema payloadSchema) { + try { + InputStream inputStream = new ByteArrayInputStream(payload); + RowCoder rowCoder = RowCoder.of(payloadSchema); + return rowCoder.decode(inputStream); + } catch (IOException e) { + throw new IllegalArgumentException("Could not decode row from proto payload.", e); + } + } + } + + /** A PTransform to convert {@link Row} to {@code KV<byte[], byte[]>}. */ + public static class ProtoRecorderEncoder + extends PTransform<PCollection<Row>, PCollection<KV<byte[], byte[]>>> { + + @Override + public PCollection<KV<byte[], byte[]>> expand(PCollection<Row> input) { + return input.apply( + "encodeProtoRecord", + ParDo.of( + new DoFn<Row, KV<byte[], byte[]>>() { + @ProcessElement + public void processElement(ProcessContext c) { + Row in = c.element(); + c.output(KV.of(new byte[] {}, encodeRowToProtoBytes(in))); + } + })); + } + + static byte[] encodeRowToProtoBytes(Row row) { + RowCoder rowCoder = RowCoder.of(row.getSchema()); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + try { + rowCoder.encode(row, outputStream); Review comment: This isn't actually encoding to protobuf, RowCoder has it's own serialization format, specified here: https://github.com/apache/beam/blob/3f71138fba908fc753438ba17d4b600bbf18de20/model/pipeline/src/main/proto/beam_runner_api.proto#L891-L903 Instead you'll want to look at using the [protobuf extensions](https://github.com/apache/beam/tree/master/sdks/java/extensions/protobuf) that @alexvanboxel added. That package has utilities for converting between protobuf types and beam Rows, in [ProtoDynamicMessageSchema.forSchema](https://github.com/apache/beam/blob/master/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDynamicMessageSchema.java). There's also [ProtoCoder](https://github.com/apache/beam/blob/master/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java) and [DynamicProtoCoder](https://github.com/apache/beam/blob/master/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoder.java) which can serialize to bytes. I think this may be more involved than adding Avro support, I'm not sure that we can encode to proto with just a Beam schema, we likely need a protobuf message descriptor as well, so there will need to be some way for the user to provide that. ---------------------------------------------------------------- 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]
