klam-shop commented on code in PR #24482:
URL: https://github.com/apache/flink/pull/24482#discussion_r1520345251


##########
flink-formats/flink-protobuf-confluent-registry/src/main/java/org/apache/flink/formats/protobuf/registry/confluent/SchemaRegistryCoder.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.flink.formats.protobuf.registry.confluent;
+
+import io.confluent.kafka.schemaregistry.ParsedSchema;
+import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
+import 
io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import static java.lang.String.format;
+
+/** Reads and Writes schema using Confluent Schema Registry protocol. */
+public class SchemaRegistryCoder {

Review Comment:
   Should we reuse the exissting `SchemaCoder` interface already in Flink 
avro-confluent format?
   
https://github.com/apache/flink/blob/2ec2a606a4f041bfa45569700f97df556f10779d/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/SchemaCoder.java#L32



##########
flink-formats/flink-protobuf-confluent-registry/src/main/java/org/apache/flink/formats/protobuf/registry/confluent/ProtoRegistryDeserializationSchema.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.flink.formats.protobuf.registry.confluent;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import 
org.apache.flink.formats.protobuf.registry.confluent.utils.MutableByteArrayInputStream;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.DynamicMessage;
+import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
+import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
+import org.apache.kafka.common.utils.ByteUtils;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A {@link DeserializationSchema} that deserializes {@link RowData} from 
Protobuf messages using
+ * Schema Registry protocol.
+ */
+public class ProtoRegistryDeserializationSchema implements 
DeserializationSchema<RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SchemaRegistryConfig schemaRegistryConfig;
+    private final RowType rowType;
+    private final TypeInformation<RowData> producedType;
+
+    /** Input stream to read message from. */
+    private transient MutableByteArrayInputStream inputStream;
+
+    private transient SchemaRegistryCoder schemaCoder;
+
+    private transient ProtoToRowDataConverters.ProtoToRowDataConverter 
runtimeConverter;
+    private transient Descriptor descriptor;
+
+    public ProtoRegistryDeserializationSchema(
+            SchemaRegistryConfig schemaRegistryConfig,
+            RowType rowType,
+            TypeInformation<RowData> producedType) {
+        this.schemaRegistryConfig = 
Preconditions.checkNotNull(schemaRegistryConfig);
+        this.rowType = Preconditions.checkNotNull(rowType);
+        this.producedType = Preconditions.checkNotNull(producedType);
+    }
+
+    @Override
+    public void open(InitializationContext context) throws Exception {
+        final SchemaRegistryClient schemaRegistryClient = 
schemaRegistryConfig.createClient();
+        this.schemaCoder =
+                new SchemaRegistryCoder(schemaRegistryConfig.getSchemaId(), 
schemaRegistryClient);
+        final ProtobufSchema schema =
+                (ProtobufSchema)
+                        
schemaRegistryClient.getSchemaById(schemaRegistryConfig.getSchemaId());
+        this.descriptor = schema.toDescriptor();
+        this.runtimeConverter = 
ProtoToRowDataConverters.createConverter(descriptor, rowType);
+        this.inputStream = new MutableByteArrayInputStream();
+    }
+
+    @Override
+    public RowData deserialize(byte[] message) throws IOException {
+        if (message == null) {
+            return null;
+        }
+        try {
+            inputStream.setBuffer(message);
+            schemaCoder.readSchema(inputStream);
+            // Not sure what the message indexes are, it is some Confluent 
Schema Registry Protobuf
+            // magic. Until we figure out what that is, let's skip it
+            skipMessageIndexes(inputStream);

Review Comment:
   I think this code should parse the Message Indexes in order to determine 
what Protobuf Message to use, in the case where the fetched schema defines 
multiple Messages. 
   
   (Note: it does make sense it for passing bytes to deserialize to 
DynamicMessage to skip the Message Indexes, but as above and below, we need 
them to determine which Protobuf Message to use.)
   
   The Message Indexes are part of the Wire format, described here:
   
https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format
   
   >The Protobuf serialization format appends a list of message indexes after 
the magic-byte and schema-id. So, the Protobuf serialization format is:
   > magic-byte, schema-id, message-indexes, protobuf-payload
   > where message-indexes is an array of indexes that corresponds to the 
message type (which may be nested). A single Schema Registry Protobuf entry may 
contain multiple Protobuf messages, some of which may have nested messages. 
**The role of message-indexes is to identify which Protobuf message in the 
Schema Registry entry to use.**
   



##########
flink-formats/flink-protobuf-confluent-registry/src/main/java/org/apache/flink/formats/protobuf/registry/confluent/ProtoRegistrySerializationSchema.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.flink.formats.protobuf.registry.confluent;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import com.google.protobuf.DynamicMessage;
+import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
+import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
+import org.apache.kafka.common.utils.ByteUtils;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * Serialization schema that serializes an object of Flink internal data 
structure with a Protobuf
+ * format.
+ *
+ * <p>Result <code>byte[]</code> messages can be deserialized using {@link
+ * ProtoRegistryDeserializationSchema}.
+ */
+public class ProtoRegistrySerializationSchema implements 
SerializationSchema<RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    /** RowType to generate the runtime converter. */
+    private final RowType rowType;
+
+    private final SchemaRegistryConfig schemaRegistryConfig;
+
+    /** The converter that converts internal data formats to JsonNode. */
+    private transient RowDataToProtoConverters.RowDataToProtoConverter 
runtimeConverter;
+
+    private transient SchemaRegistryCoder schemaCoder;
+    /** Output stream to write message to. */
+    private transient ByteArrayOutputStream arrayOutputStream;
+
+    public ProtoRegistrySerializationSchema(SchemaRegistryConfig 
registryConfig, RowType rowType) {
+        this.rowType = Preconditions.checkNotNull(rowType);
+        this.schemaRegistryConfig = Preconditions.checkNotNull(registryConfig);
+    }
+
+    @Override
+    public void open(InitializationContext context) throws Exception {
+        final SchemaRegistryClient schemaRegistryClient = 
schemaRegistryConfig.createClient();
+        this.schemaCoder =
+                new SchemaRegistryCoder(schemaRegistryConfig.getSchemaId(), 
schemaRegistryClient);
+        final ProtobufSchema schema =
+                (ProtobufSchema)
+                        
schemaRegistryClient.getSchemaById(schemaRegistryConfig.getSchemaId());
+        this.runtimeConverter =
+                RowDataToProtoConverters.createConverter(rowType, 
schema.toDescriptor());
+        this.arrayOutputStream = new ByteArrayOutputStream();
+    }
+
+    @Override
+    public byte[] serialize(RowData row) {
+        try {
+            final DynamicMessage converted = (DynamicMessage) 
runtimeConverter.convert(row);
+
+            arrayOutputStream.reset();
+            schemaCoder.writeSchema(arrayOutputStream);
+            final ByteBuffer buffer = writeMessageIndexes();
+            arrayOutputStream.write(buffer.array());
+            converted.writeTo(arrayOutputStream);
+            return arrayOutputStream.toByteArray();
+        } catch (Throwable t) {
+            throw new FlinkRuntimeException(String.format("Could not serialize 
row '%s'.", row), t);
+        }
+    }
+
+    private static ByteBuffer writeMessageIndexes() {
+        //write empty message indices for now
+        ByteBuffer buffer = ByteBuffer.allocate(ByteUtils.sizeOfVarint(0));
+        ByteUtils.writeVarint(0, buffer);
+        return buffer;

Review Comment:
   Similar to other comment, I think this code should handle Message Indexes 
properly to support schemas with multiple Messages. 
   
   Refer to the wire format for protobuf:
   
https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to