WRITE_NULL_STRING_LITERAL =
+ ConfigOptions.key("write-null-string-literal")
+ .stringType()
+ .defaultValue("")
+ .withDescription(
+ "When serializing to protobuf data, this is the optional config to specify the string literal in protobuf's array/map in case of null values."
+ + "By default empty string is used.");
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenArrayDeserializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenArrayDeserializer.java
new file mode 100644
index 0000000000000..65bc760b0a33a
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenArrayDeserializer.java
@@ -0,0 +1,87 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+
+/** Deserializer to convert proto array type object to flink array type data. */
+public class PbCodegenArrayDeserializer implements PbCodegenDeserializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final LogicalType elementType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenArrayDeserializer(
+ Descriptors.FieldDescriptor fd,
+ LogicalType elementType,
+ PbFormatContext formatContext) {
+ this.fd = fd;
+ this.elementType = elementType;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String pbObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of pbObjectCode represents a general List object,
+ // it should be converted to ArrayData of flink internal type as resultVariable.
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ String protoTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(fd, false, formatContext.getOuterPrefix());
+ String listPbVar = "list" + uid;
+ String flinkArrVar = "newArr" + uid;
+ String flinkArrEleVar = "subReturnVar" + uid;
+ String iVar = "i" + uid;
+ String subPbObjVar = "subObj" + uid;
+
+ appender.appendLine("List<" + protoTypeStr + "> " + listPbVar + "=" + pbObjectCode);
+ appender.appendLine(
+ "Object[] " + flinkArrVar + "= new " + "Object[" + listPbVar + ".size()]");
+ appender.begin(
+ "for(int " + iVar + "=0;" + iVar + " < " + listPbVar + ".size(); " + iVar + "++){");
+ appender.appendLine("Object " + flinkArrEleVar + " = null");
+ appender.appendLine(
+ protoTypeStr
+ + " "
+ + subPbObjVar
+ + " = ("
+ + protoTypeStr
+ + ")"
+ + listPbVar
+ + ".get("
+ + iVar
+ + ")");
+ PbCodegenDeserializer codegenDes =
+ PbCodegenDeserializeFactory.getPbCodegenDes(fd, elementType, formatContext);
+ String code = codegenDes.codegen(flinkArrEleVar, subPbObjVar, appender.currentIndent());
+ appender.appendSegment(code);
+ appender.appendLine(flinkArrVar + "[" + iVar + "]=" + flinkArrEleVar + "");
+ appender.end("}");
+ appender.appendLine(resultVar + " = new GenericArrayData(" + flinkArrVar + ")");
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializeFactory.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializeFactory.java
new file mode 100644
index 0000000000000..ddc372ff6bec7
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializeFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+
+/** Codegen factory class which return {@link PbCodegenDeserializer} of different data type. */
+public class PbCodegenDeserializeFactory {
+ public static PbCodegenDeserializer getPbCodegenDes(
+ Descriptors.FieldDescriptor fd, LogicalType type, PbFormatContext formatContext)
+ throws PbCodegenException {
+ // We do not use FieldDescriptor to check because there's no way to get
+ // element field descriptor of array type.
+ if (type instanceof RowType) {
+ return new PbCodegenRowDeserializer(fd.getMessageType(), (RowType) type, formatContext);
+ } else if (PbFormatUtils.isSimpleType(type)) {
+ return new PbCodegenSimpleDeserializer(fd, type);
+ } else if (type instanceof ArrayType) {
+ return new PbCodegenArrayDeserializer(
+ fd, ((ArrayType) type).getElementType(), formatContext);
+ } else if (type instanceof MapType) {
+ return new PbCodegenMapDeserializer(fd, (MapType) type, formatContext);
+ } else {
+ throw new PbCodegenException("Do not support flink type: " + type);
+ }
+ }
+
+ public static PbCodegenDeserializer getPbCodegenTopRowDes(
+ Descriptors.Descriptor descriptor, RowType rowType, PbFormatContext formatContext) {
+ return new PbCodegenRowDeserializer(descriptor, rowType, formatContext);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializer.java
new file mode 100644
index 0000000000000..05cec9e778d16
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenDeserializer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+
+/**
+ * {@link PbCodegenDeserializer} is responsible for converting protobuf object to flink internal
+ * object by codegen process. The codegen procedure could be considered as
+ *
+ * {@code returnVarName = codegen(pbStr) }
+ *
+ */
+public interface PbCodegenDeserializer {
+ /**
+ * @param resultVar the final var name that is calculated by codegen. This var name will be used
+ * by outsider codegen environment. {@code resultVariable} should be flink object
+ * @param pbObjectCode may be a variable or expression. Current codegen environment can use this
+ * literal name directly to access the input. {@code pbObject} should be a protobuf object
+ * @return The java code generated
+ */
+ String codegen(String resultVar, String pbObjectCode, int indent) throws PbCodegenException;
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenMapDeserializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenMapDeserializer.java
new file mode 100644
index 0000000000000..439332e29e72b
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenMapDeserializer.java
@@ -0,0 +1,115 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+
+import com.google.protobuf.Descriptors;
+
+/** Deserializer to convert proto map type object to flink map type data. */
+public class PbCodegenMapDeserializer implements PbCodegenDeserializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final MapType mapType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenMapDeserializer(
+ Descriptors.FieldDescriptor fd, MapType mapType, PbFormatContext formatContext) {
+ this.fd = fd;
+ this.mapType = mapType;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String pbObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of pbObjectCode is a general Map object,
+ // it should be converted to MapData of flink internal type as resultVariable
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+
+ LogicalType keyType = mapType.getKeyType();
+ LogicalType valueType = mapType.getValueType();
+ Descriptors.FieldDescriptor keyFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_KEY_NAME);
+ Descriptors.FieldDescriptor valueFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_VALUE_NAME);
+
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ String pbKeyTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(keyFd, false, formatContext.getOuterPrefix());
+ String pbValueTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(valueFd, false, formatContext.getOuterPrefix());
+ String pbMapVar = "pbMap" + uid;
+ String pbMapEntryVar = "pbEntry" + uid;
+ String resultDataMapVar = "resultDataMap" + uid;
+ String flinkKeyVar = "keyDataVar" + uid;
+ String flinkValueVar = "valueDataVar" + uid;
+
+ appender.appendLine(
+ "Map<"
+ + pbKeyTypeStr
+ + ","
+ + pbValueTypeStr
+ + "> "
+ + pbMapVar
+ + " = "
+ + pbObjectCode
+ + ";");
+ appender.appendLine("Map " + resultDataMapVar + " = new HashMap()");
+ appender.begin(
+ "for(Map.Entry<"
+ + pbKeyTypeStr
+ + ","
+ + pbValueTypeStr
+ + "> "
+ + pbMapEntryVar
+ + ": "
+ + pbMapVar
+ + ".entrySet()){");
+ appender.appendLine("Object " + flinkKeyVar + "= null");
+ appender.appendLine("Object " + flinkValueVar + "= null");
+ PbCodegenDeserializer keyDes =
+ PbCodegenDeserializeFactory.getPbCodegenDes(keyFd, keyType, formatContext);
+ PbCodegenDeserializer valueDes =
+ PbCodegenDeserializeFactory.getPbCodegenDes(valueFd, valueType, formatContext);
+ String keyGenCode =
+ keyDes.codegen(
+ flinkKeyVar,
+ "((" + pbKeyTypeStr + ")" + pbMapEntryVar + ".getKey())",
+ appender.currentIndent());
+ appender.appendSegment(keyGenCode);
+ String valueGenCode =
+ valueDes.codegen(
+ flinkValueVar,
+ "((" + pbValueTypeStr + ")" + pbMapEntryVar + ".getValue())",
+ appender.currentIndent());
+ appender.appendSegment(valueGenCode);
+ appender.appendLine(resultDataMapVar + ".put(" + flinkKeyVar + ", " + flinkValueVar + ")");
+ appender.end("}");
+ appender.appendLine(resultVar + " = new GenericMapData(" + resultDataMapVar + ")");
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenRowDeserializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenRowDeserializer.java
new file mode 100644
index 0000000000000..3a5b311316628
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenRowDeserializer.java
@@ -0,0 +1,126 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+
+/** Deserializer to convert proto message type object to flink row type data. */
+public class PbCodegenRowDeserializer implements PbCodegenDeserializer {
+ private final Descriptor descriptor;
+ private final RowType rowType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenRowDeserializer(
+ Descriptor descriptor, RowType rowType, PbFormatContext formatContext) {
+ this.rowType = rowType;
+ this.descriptor = descriptor;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String pbObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of pbObjectCode is a general pb object,
+ // it should be converted to RowData of flink internal type as resultVariable
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ String pbMessageVar = "message" + uid;
+ String flinkRowDataVar = "rowData" + uid;
+
+ int fieldSize = rowType.getFieldNames().size();
+ String pbMessageTypeStr =
+ PbFormatUtils.getFullJavaName(descriptor, formatContext.getOuterPrefix());
+ appender.appendLine(pbMessageTypeStr + " " + pbMessageVar + " = " + pbObjectCode);
+ appender.appendLine(
+ "GenericRowData " + flinkRowDataVar + " = new GenericRowData(" + fieldSize + ")");
+ int index = 0;
+ for (String fieldName : rowType.getFieldNames()) {
+ int subUid = varUid.getAndIncrement();
+ String flinkRowEleVar = "elementDataVar" + subUid;
+
+ LogicalType subType = rowType.getTypeAt(rowType.getFieldIndex(fieldName));
+ FieldDescriptor elementFd = descriptor.findFieldByName(fieldName);
+ String strongCamelFieldName = PbFormatUtils.getStrongCamelCaseJsonName(fieldName);
+ PbCodegenDeserializer codegen =
+ PbCodegenDeserializeFactory.getPbCodegenDes(elementFd, subType, formatContext);
+ appender.appendLine("Object " + flinkRowEleVar + " = null");
+ if (!formatContext.getPbFormatConfig().isReadDefaultValues()) {
+ // only works in syntax=proto2 and readDefaultValues=false
+ // readDefaultValues must be true in pb3 mode
+ String isMessageElementNonEmptyCode =
+ isMessageElementNonEmptyCode(
+ pbMessageVar,
+ strongCamelFieldName,
+ PbFormatUtils.isRepeatedType(subType));
+ appender.begin("if(" + isMessageElementNonEmptyCode + "){");
+ }
+ String pbGetMessageElementCode =
+ pbGetMessageElementCode(
+ pbMessageVar,
+ strongCamelFieldName,
+ elementFd,
+ PbFormatUtils.isArrayType(subType));
+ String code =
+ codegen.codegen(
+ flinkRowEleVar, pbGetMessageElementCode, appender.currentIndent());
+ appender.appendSegment(code);
+ if (!formatContext.getPbFormatConfig().isReadDefaultValues()) {
+ appender.end("}");
+ }
+ appender.appendLine(
+ flinkRowDataVar + ".setField(" + index + ", " + flinkRowEleVar + ")");
+ index += 1;
+ }
+ appender.appendLine(resultVar + " = " + flinkRowDataVar);
+ return appender.code();
+ }
+
+ private String pbGetMessageElementCode(
+ String message, String fieldName, FieldDescriptor fd, boolean isList) {
+ if (fd.isMapField()) {
+ // map
+ return message + ".get" + fieldName + "Map()";
+ } else if (isList) {
+ // list
+ return message + ".get" + fieldName + "List()";
+ } else {
+ return message + ".get" + fieldName + "()";
+ }
+ }
+
+ private String isMessageElementNonEmptyCode(
+ String message, String fieldName, boolean isListOrMap) {
+ if (isListOrMap) {
+ return message + ".get" + fieldName + "Count() > 0";
+ } else {
+ // proto syntax class do not have hasName() interface
+ return message + ".has" + fieldName + "()";
+ }
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java
new file mode 100644
index 0000000000000..146d6b0220341
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java
@@ -0,0 +1,86 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import com.google.protobuf.Descriptors;
+
+/** Deserializer to convert proto simple type object to flink simple type data. */
+public class PbCodegenSimpleDeserializer implements PbCodegenDeserializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final LogicalType logicalType;
+
+ public PbCodegenSimpleDeserializer(Descriptors.FieldDescriptor fd, LogicalType logicalType) {
+ this.fd = fd;
+ this.logicalType = logicalType;
+ }
+
+ @Override
+ public String codegen(String resultVar, String pbObjectCode, int indent)
+ throws PbCodegenException {
+ // the type of pbObjectCode must not be primitive type,
+ // it should convert to internal flink row type like StringData.
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ switch (fd.getJavaType()) {
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ case BOOLEAN:
+ appender.appendLine(resultVar + " = " + pbObjectCode);
+ break;
+ case BYTE_STRING:
+ appender.appendLine(resultVar + " = " + pbObjectCode + ".toByteArray()");
+ break;
+ case STRING:
+ appender.appendLine(
+ resultVar
+ + " = BinaryStringData.fromString("
+ + pbObjectCode
+ + ".toString())");
+ break;
+ case ENUM:
+ if (logicalType.getTypeRoot() == LogicalTypeRoot.CHAR
+ || logicalType.getTypeRoot() == LogicalTypeRoot.VARCHAR) {
+ appender.appendLine(
+ resultVar
+ + " = BinaryStringData.fromString("
+ + pbObjectCode
+ + ".toString())");
+ } else if (logicalType.getTypeRoot() == LogicalTypeRoot.TINYINT
+ || logicalType.getTypeRoot() == LogicalTypeRoot.SMALLINT
+ || logicalType.getTypeRoot() == LogicalTypeRoot.INTEGER
+ || logicalType.getTypeRoot() == LogicalTypeRoot.BIGINT) {
+ appender.appendLine(resultVar + " = " + pbObjectCode + ".getNumber()");
+ } else {
+ throw new PbCodegenException(
+ "Illegal type for protobuf enum, only char/vachar/int/bigint is supported");
+ }
+ break;
+ default:
+ throw new PbCodegenException(
+ "Unsupported protobuf simple type: " + fd.getJavaType());
+ }
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
new file mode 100644
index 0000000000000..5d3872713d2c1
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
@@ -0,0 +1,107 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.formats.protobuf.util.PbSchemaValidationUtils;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from Protobuf to Flink types.
+ *
+ * Deserializes a byte[]
message as a protobuf object and reads the specified
+ * fields.
+ *
+ *
Failures during deserialization are forwarded as wrapped IOExceptions.
+ */
+public class PbRowDataDeserializationSchema implements DeserializationSchema {
+ private static final long serialVersionUID = 1L;
+
+ private final RowType rowType;
+ private final TypeInformation resultTypeInfo;
+ private final PbFormatConfig formatConfig;
+ private transient ProtoToRowConverter protoToRowConverter;
+
+ public PbRowDataDeserializationSchema(
+ RowType rowType, TypeInformation resultTypeInfo, PbFormatConfig formatConfig) {
+ checkNotNull(rowType, "rowType cannot be null");
+ this.rowType = rowType;
+ this.resultTypeInfo = resultTypeInfo;
+ this.formatConfig = formatConfig;
+ // do it in client side to report error in the first place
+ PbSchemaValidationUtils.validate(
+ PbFormatUtils.getDescriptor(formatConfig.getMessageClassName()), rowType);
+ // this step is only used to validate codegen in client side in the first place
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ protoToRowConverter = new ProtoToRowConverter(rowType, formatConfig);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ try {
+ return protoToRowConverter.convertProtoBinaryToRow(message);
+ } catch (Throwable t) {
+ if (formatConfig.isIgnoreParseErrors()) {
+ return null;
+ }
+ throw new IOException("Failed to deserialize PB object.", t);
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return this.resultTypeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ PbRowDataDeserializationSchema that = (PbRowDataDeserializationSchema) o;
+ return Objects.equals(rowType, that.rowType)
+ && Objects.equals(resultTypeInfo, that.resultTypeInfo)
+ && Objects.equals(formatConfig, that.formatConfig);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType, resultTypeInfo, formatConfig);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/ProtoToRowConverter.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/ProtoToRowConverter.java
new file mode 100644
index 0000000000000..0ecbbaef19e31
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/ProtoToRowConverter.java
@@ -0,0 +1,134 @@
+/*
+ * 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.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FileDescriptor.Syntax;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link ProtoToRowConverter} can convert binary protobuf message data to flink row data by codegen
+ * process.
+ */
+public class ProtoToRowConverter {
+ private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+ private final Method parseFromMethod;
+ private final Method decodeMethod;
+
+ public ProtoToRowConverter(RowType rowType, PbFormatConfig formatConfig)
+ throws PbCodegenException {
+ try {
+ String outerPrefix =
+ PbFormatUtils.getOuterProtoPrefix(formatConfig.getMessageClassName());
+ Descriptors.Descriptor descriptor =
+ PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+ Class> messageClass =
+ Class.forName(
+ formatConfig.getMessageClassName(),
+ true,
+ Thread.currentThread().getContextClassLoader());
+ String fullMessageClassName = PbFormatUtils.getFullJavaName(descriptor, outerPrefix);
+ if (descriptor.getFile().getSyntax() == Syntax.PROTO3) {
+ // pb3 always read default values
+ formatConfig =
+ new PbFormatConfig(
+ formatConfig.getMessageClassName(),
+ formatConfig.isIgnoreParseErrors(),
+ true,
+ formatConfig.getWriteNullStringLiterals());
+ }
+ PbCodegenAppender codegenAppender = new PbCodegenAppender();
+ PbFormatContext pbFormatContext = new PbFormatContext(outerPrefix, formatConfig);
+ String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+ String generatedClassName = "GeneratedProtoToRow_" + uuid;
+ String generatedPackageName = ProtoToRowConverter.class.getPackage().getName();
+ codegenAppender.appendLine("package " + generatedPackageName);
+ codegenAppender.appendLine("import " + RowData.class.getName());
+ codegenAppender.appendLine("import " + ArrayData.class.getName());
+ codegenAppender.appendLine("import " + BinaryStringData.class.getName());
+ codegenAppender.appendLine("import " + GenericRowData.class.getName());
+ codegenAppender.appendLine("import " + GenericMapData.class.getName());
+ codegenAppender.appendLine("import " + GenericArrayData.class.getName());
+ codegenAppender.appendLine("import " + ArrayList.class.getName());
+ codegenAppender.appendLine("import " + List.class.getName());
+ codegenAppender.appendLine("import " + Map.class.getName());
+ codegenAppender.appendLine("import " + HashMap.class.getName());
+ codegenAppender.appendLine("import " + ByteString.class.getName());
+
+ codegenAppender.appendSegment("public class " + generatedClassName + "{");
+ codegenAppender.appendSegment(
+ "public static RowData "
+ + PbConstant.GENERATED_DECODE_METHOD
+ + "("
+ + fullMessageClassName
+ + " message){");
+ codegenAppender.appendLine("RowData rowData=null");
+ PbCodegenDeserializer codegenDes =
+ PbCodegenDeserializeFactory.getPbCodegenTopRowDes(
+ descriptor, rowType, pbFormatContext);
+ String genCode = codegenDes.codegen("rowData", "message", 0);
+ codegenAppender.appendSegment(genCode);
+ codegenAppender.appendLine("return rowData");
+ codegenAppender.appendSegment("}");
+ codegenAppender.appendSegment("}");
+
+ String printCode = codegenAppender.printWithLineNumber();
+ LOG.debug("Protobuf decode codegen: \n" + printCode);
+ Class generatedClass =
+ PbCodegenUtils.compileClass(
+ Thread.currentThread().getContextClassLoader(),
+ generatedPackageName + "." + generatedClassName,
+ codegenAppender.code());
+ decodeMethod =
+ generatedClass.getMethod(PbConstant.GENERATED_DECODE_METHOD, messageClass);
+ parseFromMethod = messageClass.getMethod(PbConstant.PB_METHOD_PARSE_FROM, byte[].class);
+ } catch (Exception ex) {
+ throw new PbCodegenException(ex);
+ }
+ }
+
+ public RowData convertProtoBinaryToRow(byte[] data) throws Exception {
+ Object messageObj = parseFromMethod.invoke(null, data);
+ return (RowData) decodeMethod.invoke(null, messageObj);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenArraySerializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenArraySerializer.java
new file mode 100644
index 0000000000000..9d242d96e437c
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenArraySerializer.java
@@ -0,0 +1,89 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+
+/** Serializer to convert flink array type data to proto array type object. */
+public class PbCodegenArraySerializer implements PbCodegenSerializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final LogicalType elementType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenArraySerializer(
+ Descriptors.FieldDescriptor fd,
+ LogicalType elementType,
+ PbFormatContext formatContext) {
+ this.fd = fd;
+ this.elementType = elementType;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String flinkObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of flinkObjectCode is a ArrayData of flink,
+ // it should be converted to array of protobuf as resultVariable.
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ String protoTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(fd, false, formatContext.getOuterPrefix());
+ String pbListVar = "pbList" + uid;
+ String flinkArrayDataVar = "arrData" + uid;
+ String pbElementVar = "elementPbVar" + uid;
+ String iVar = "i" + uid;
+
+ appender.appendLine("ArrayData " + flinkArrayDataVar + " = " + flinkObjectCode);
+ appender.appendLine("List<" + protoTypeStr + "> " + pbListVar + "= new ArrayList()");
+ appender.begin(
+ "for(int "
+ + iVar
+ + "=0;"
+ + iVar
+ + " < "
+ + flinkArrayDataVar
+ + ".size(); "
+ + iVar
+ + "++){");
+ String convertFlinkArrayElementToPbCode =
+ PbCodegenUtils.convertFlinkArrayElementToPbWithDefaultValueCode(
+ flinkArrayDataVar,
+ iVar,
+ pbElementVar,
+ fd,
+ elementType,
+ formatContext,
+ appender.currentIndent());
+ appender.appendSegment(convertFlinkArrayElementToPbCode);
+ // add pb element to result list
+ appender.appendLine(pbListVar + ".add( " + pbElementVar + ")");
+ // end for
+ appender.end("}");
+ appender.appendLine(resultVar + " = " + pbListVar);
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenMapSerializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenMapSerializer.java
new file mode 100644
index 0000000000000..c0c9e383a0087
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenMapSerializer.java
@@ -0,0 +1,126 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+
+import com.google.protobuf.Descriptors;
+
+/** Serializer to convert flink map type data to proto map type object. */
+public class PbCodegenMapSerializer implements PbCodegenSerializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final MapType mapType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenMapSerializer(
+ Descriptors.FieldDescriptor fd, MapType mapType, PbFormatContext formatContext) {
+ this.fd = fd;
+ this.mapType = mapType;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String flinkObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of flinkObjectCode is a MapData of flink,
+ // it should be converted to map of protobuf as resultVariable.
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ LogicalType keyType = mapType.getKeyType();
+ LogicalType valueType = mapType.getValueType();
+ Descriptors.FieldDescriptor keyFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_KEY_NAME);
+ Descriptors.FieldDescriptor valueFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_VALUE_NAME);
+
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ String keyProtoTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(keyFd, false, formatContext.getOuterPrefix());
+ String valueProtoTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(valueFd, false, formatContext.getOuterPrefix());
+
+ String flinkKeyArrDataVar = "keyArrData" + uid;
+ String flinkValueArrDataVar = "valueArrData" + uid;
+ String iVar = "i" + uid;
+ String pbMapVar = "resultPbMap" + uid;
+ String keyPbVar = "keyPbVar" + uid;
+ String valuePbVar = "valuePbVar" + uid;
+
+ appender.appendLine(
+ "ArrayData " + flinkKeyArrDataVar + " = " + flinkObjectCode + ".keyArray()");
+ appender.appendLine(
+ "ArrayData " + flinkValueArrDataVar + " = " + flinkObjectCode + ".valueArray()");
+
+ appender.appendLine(
+ "Map<"
+ + keyProtoTypeStr
+ + ", "
+ + valueProtoTypeStr
+ + "> "
+ + pbMapVar
+ + " = new HashMap()");
+ appender.begin(
+ "for(int "
+ + iVar
+ + " = 0; "
+ + iVar
+ + " < "
+ + flinkKeyArrDataVar
+ + ".size(); "
+ + iVar
+ + "++){");
+
+ // process key
+ String convertFlinkKeyArrayElementToPbCode =
+ PbCodegenUtils.convertFlinkArrayElementToPbWithDefaultValueCode(
+ flinkKeyArrDataVar,
+ iVar,
+ keyPbVar,
+ keyFd,
+ keyType,
+ formatContext,
+ appender.currentIndent());
+ appender.appendSegment(convertFlinkKeyArrayElementToPbCode);
+
+ // process value
+ String convertFlinkValueArrayElementToPbCode =
+ PbCodegenUtils.convertFlinkArrayElementToPbWithDefaultValueCode(
+ flinkValueArrDataVar,
+ iVar,
+ valuePbVar,
+ valueFd,
+ valueType,
+ formatContext,
+ appender.currentIndent());
+ appender.appendSegment(convertFlinkValueArrayElementToPbCode);
+
+ appender.appendLine(pbMapVar + ".put(" + keyPbVar + ", " + valuePbVar + ")");
+ appender.end("}");
+
+ appender.appendLine(resultVar + " = " + pbMapVar);
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
new file mode 100644
index 0000000000000..5f085a779e6ea
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
@@ -0,0 +1,128 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+
+/** Serializer to convert flink row type data to proto row type object. */
+public class PbCodegenRowSerializer implements PbCodegenSerializer {
+ private final Descriptors.Descriptor descriptor;
+ private final RowType rowType;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenRowSerializer(
+ Descriptors.Descriptor descriptor, RowType rowType, PbFormatContext formatContext) {
+ this.rowType = rowType;
+ this.descriptor = descriptor;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String flinkObjectCode, int indent)
+ throws PbCodegenException {
+ // The type of flinkObjectCode is a RowData of flink,
+ // it should be converted to object of protobuf as resultVariable.
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ String flinkRowDataVar = "rowData" + uid;
+ String pbMessageTypeStr =
+ PbFormatUtils.getFullJavaName(descriptor, formatContext.getOuterPrefix());
+ String messageBuilderVar = "messageBuilder" + uid;
+ appender.appendLine("RowData " + flinkRowDataVar + " = " + flinkObjectCode);
+ appender.appendLine(
+ pbMessageTypeStr
+ + ".Builder "
+ + messageBuilderVar
+ + " = "
+ + pbMessageTypeStr
+ + ".newBuilder()");
+ int index = 0;
+ for (String fieldName : rowType.getFieldNames()) {
+ Descriptors.FieldDescriptor elementFd = descriptor.findFieldByName(fieldName);
+ LogicalType subType = rowType.getTypeAt(rowType.getFieldIndex(fieldName));
+ int subUid = varUid.getAndIncrement();
+ String elementPbVar = "elementPbVar" + subUid;
+ String elementPbTypeStr;
+ if (elementFd.isMapField()) {
+ elementPbTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(
+ elementFd, false, formatContext.getOuterPrefix());
+ } else {
+ elementPbTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(
+ elementFd,
+ PbFormatUtils.isArrayType(subType),
+ formatContext.getOuterPrefix());
+ }
+ String strongCamelFieldName = PbFormatUtils.getStrongCamelCaseJsonName(fieldName);
+
+ // Only set non-null element of flink row to proto object. The real value in proto
+ // result depends on protobuf implementation.
+ appender.begin("if(!" + flinkRowDataVar + ".isNullAt(" + index + ")){");
+ appender.appendLine(elementPbTypeStr + " " + elementPbVar);
+ String flinkRowElementCode =
+ PbCodegenUtils.flinkContainerElementCode(flinkRowDataVar, index + "", subType);
+ PbCodegenSerializer codegen =
+ PbCodegenSerializeFactory.getPbCodegenSer(elementFd, subType, formatContext);
+ String code =
+ codegen.codegen(elementPbVar, flinkRowElementCode, appender.currentIndent());
+ appender.appendSegment(code);
+ if (subType.getTypeRoot() == LogicalTypeRoot.ARRAY) {
+ appender.appendLine(
+ messageBuilderVar
+ + ".addAll"
+ + strongCamelFieldName
+ + "("
+ + elementPbVar
+ + ")");
+ } else if (subType.getTypeRoot() == LogicalTypeRoot.MAP) {
+ appender.appendLine(
+ messageBuilderVar
+ + ".putAll"
+ + strongCamelFieldName
+ + "("
+ + elementPbVar
+ + ")");
+ } else {
+ appender.appendLine(
+ messageBuilderVar
+ + ".set"
+ + strongCamelFieldName
+ + "("
+ + elementPbVar
+ + ")");
+ }
+ appender.end("}");
+ index += 1;
+ }
+ appender.appendLine(resultVar + " = " + messageBuilderVar + ".build()");
+ return appender.code();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializeFactory.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializeFactory.java
new file mode 100644
index 0000000000000..4f90c0e15612b
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializeFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+
+/** Codegen factory class which return {@link PbCodegenSerializer} of different data type. */
+public class PbCodegenSerializeFactory {
+ public static PbCodegenSerializer getPbCodegenSer(
+ Descriptors.FieldDescriptor fd, LogicalType type, PbFormatContext formatContext)
+ throws PbCodegenException {
+ if (type instanceof RowType) {
+ return new PbCodegenRowSerializer(fd.getMessageType(), (RowType) type, formatContext);
+ } else if (PbFormatUtils.isSimpleType(type)) {
+ return new PbCodegenSimpleSerializer(fd, type, formatContext);
+ } else if (type instanceof ArrayType) {
+ return new PbCodegenArraySerializer(
+ fd, ((ArrayType) type).getElementType(), formatContext);
+ } else if (type instanceof MapType) {
+ return new PbCodegenMapSerializer(fd, (MapType) type, formatContext);
+ } else {
+ throw new PbCodegenException("Do not support flink data type: " + type);
+ }
+ }
+
+ public static PbCodegenSerializer getPbCodegenTopRowSer(
+ Descriptors.Descriptor descriptor, RowType rowType, PbFormatContext formatContext) {
+ return new PbCodegenRowSerializer(descriptor, rowType, formatContext);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializer.java
new file mode 100644
index 0000000000000..79f32f38f6416
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSerializer.java
@@ -0,0 +1,40 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+
+/**
+ * {@link PbCodegenSerializer} is responsible for converting flink internal data object to protobuf
+ * object by codegen process. The codegen procedure could be considered as
+ *
+ * {@code resultVariableCode = codegen(flinkObjectCode) }
+ *
+ */
+public interface PbCodegenSerializer {
+ /**
+ * @param resultVar the final var name that is calculated by codegen. This var name will be used
+ * by outsider codegen environment. {@code resultVariable} should be protobuf object
+ * @param flinkObjectCode may be a variable or expression. Current codegen environment can use
+ * this literal name directly to access the input. {@code flinkObject} should be a flink
+ * internal object.
+ * @return The java code generated
+ */
+ String codegen(String resultVar, String flinkObjectCode, int indent) throws PbCodegenException;
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSimpleSerializer.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSimpleSerializer.java
new file mode 100644
index 0000000000000..29d4bc9ebd89d
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSimpleSerializer.java
@@ -0,0 +1,121 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+/** Serializer to convert flink simple type data to proto simple type object. */
+public class PbCodegenSimpleSerializer implements PbCodegenSerializer {
+ private final Descriptors.FieldDescriptor fd;
+ private final LogicalType type;
+ private final PbFormatContext formatContext;
+
+ public PbCodegenSimpleSerializer(
+ Descriptors.FieldDescriptor fd, LogicalType type, PbFormatContext formatContext) {
+ this.fd = fd;
+ this.type = type;
+ this.formatContext = formatContext;
+ }
+
+ @Override
+ public String codegen(String resultVar, String flinkObjectCode, int indent)
+ throws PbCodegenException {
+ // the real value of flinkObjectCode may be String, Integer,
+ // Long, Double, Float, Boolean, byte[].
+ // The type of flinkObject is simple data type of flink, and flinkObject must not be null.
+ // it should be converted to protobuf simple data as resultVariable.
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ switch (type.getTypeRoot()) {
+ case FLOAT:
+ case DOUBLE:
+ case BOOLEAN:
+ appender.appendLine(resultVar + " = " + flinkObjectCode);
+ return appender.code();
+ case BIGINT:
+ case INTEGER:
+ case SMALLINT:
+ case TINYINT:
+ if (fd.getJavaType() == JavaType.ENUM) {
+ String enumTypeStr =
+ PbFormatUtils.getFullJavaName(
+ fd.getEnumType(), formatContext.getOuterPrefix());
+ appender.appendLine(
+ resultVar
+ + " = "
+ + enumTypeStr
+ + ".forNumber((int)"
+ + flinkObjectCode
+ + ")");
+ // choose the first enum element as default value if such value is invalid enum
+ appender.begin("if(null == " + resultVar + "){");
+ appender.appendLine(resultVar + " = " + enumTypeStr + ".values()[0]");
+ appender.end("}");
+ } else {
+ appender.appendLine(resultVar + " = " + flinkObjectCode);
+ }
+ return appender.code();
+ case VARCHAR:
+ case CHAR:
+ int uid = PbCodegenVarId.getInstance().getAndIncrement();
+ String fromVar = "fromVar" + uid;
+ appender.appendLine("String " + fromVar);
+ appender.appendLine(fromVar + " = " + flinkObjectCode + ".toString()");
+ if (fd.getJavaType() == JavaType.ENUM) {
+ String enumValueDescVar = "enumValueDesc" + uid;
+ String enumTypeStr =
+ PbFormatUtils.getFullJavaName(
+ fd.getEnumType(), formatContext.getOuterPrefix());
+ appender.appendLine(
+ "Descriptors.EnumValueDescriptor "
+ + enumValueDescVar
+ + "="
+ + enumTypeStr
+ + ".getDescriptor().findValueByName("
+ + fromVar
+ + ")");
+ appender.begin("if(null == " + enumValueDescVar + "){");
+ // choose the first enum element as default value if such value is invalid enum
+ appender.appendLine(resultVar + " = " + enumTypeStr + ".values()[0]");
+ appender.end("}");
+ appender.begin("else{");
+ // choose the exact enum value
+ appender.appendLine(
+ resultVar + " = " + enumTypeStr + ".valueOf(" + enumValueDescVar + ")");
+ appender.end("}");
+ } else {
+ appender.appendLine(resultVar + " = " + fromVar);
+ }
+ return appender.code();
+ case VARBINARY:
+ case BINARY:
+ appender.appendLine(resultVar + " = ByteString.copyFrom(" + flinkObjectCode + ")");
+ return appender.code();
+ default:
+ throw new PbCodegenException("Unsupported data type in schema: " + type);
+ }
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbRowDataSerializationSchema.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbRowDataSerializationSchema.java
new file mode 100644
index 0000000000000..24484dd71b47e
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbRowDataSerializationSchema.java
@@ -0,0 +1,66 @@
+/*
+ * 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.serialize;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.formats.protobuf.util.PbSchemaValidationUtils;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.google.protobuf.Descriptors;
+
+/**
+ * Serialization schema from Flink to Protobuf types.
+ *
+ * Serializes a {@link RowData } to protobuf binary data.
+ *
+ *
Failures during deserialization are forwarded as wrapped {@link FlinkRuntimeException}.
+ */
+public class PbRowDataSerializationSchema implements SerializationSchema {
+ public static final long serialVersionUID = 1L;
+
+ private final RowType rowType;
+ private final PbFormatConfig pbFormatConfig;
+ private transient RowToProtoConverter rowToProtoConverter;
+
+ public PbRowDataSerializationSchema(RowType rowType, PbFormatConfig pbFormatConfig) {
+ this.rowType = rowType;
+ this.pbFormatConfig = pbFormatConfig;
+ Descriptors.Descriptor descriptor =
+ PbFormatUtils.getDescriptor(pbFormatConfig.getMessageClassName());
+ PbSchemaValidationUtils.validate(descriptor, rowType);
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ rowToProtoConverter = new RowToProtoConverter(rowType, pbFormatConfig);
+ }
+
+ @Override
+ public byte[] serialize(RowData element) {
+ try {
+ return rowToProtoConverter.convertRowToProtoBinary(element);
+ } catch (Exception e) {
+ throw new FlinkRuntimeException(e);
+ }
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
new file mode 100644
index 0000000000000..349049da02d8f
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.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.flink.formats.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.formats.protobuf.util.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.util.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+ private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+ private final Method encodeMethod;
+
+ public RowToProtoConverter(RowType rowType, PbFormatConfig formatConfig)
+ throws PbCodegenException {
+ try {
+ String outerPrefix =
+ PbFormatUtils.getOuterProtoPrefix(formatConfig.getMessageClassName());
+ PbFormatContext formatContext = new PbFormatContext(outerPrefix, formatConfig);
+ Descriptors.Descriptor descriptor =
+ PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+
+ PbCodegenAppender codegenAppender = new PbCodegenAppender(0);
+ String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+ String generatedClassName = "GeneratedRowToProto_" + uuid;
+ String generatedPackageName = RowToProtoConverter.class.getPackage().getName();
+ codegenAppender.appendLine("package " + generatedPackageName);
+ codegenAppender.appendLine("import " + AbstractMessage.class.getName());
+ codegenAppender.appendLine("import " + Descriptors.class.getName());
+ codegenAppender.appendLine("import " + RowData.class.getName());
+ codegenAppender.appendLine("import " + ArrayData.class.getName());
+ codegenAppender.appendLine("import " + StringData.class.getName());
+ codegenAppender.appendLine("import " + ByteString.class.getName());
+ codegenAppender.appendLine("import " + List.class.getName());
+ codegenAppender.appendLine("import " + ArrayList.class.getName());
+ codegenAppender.appendLine("import " + Map.class.getName());
+ codegenAppender.appendLine("import " + HashMap.class.getName());
+
+ codegenAppender.begin("public class " + generatedClassName + "{");
+ codegenAppender.begin(
+ "public static AbstractMessage "
+ + PbConstant.GENERATED_ENCODE_METHOD
+ + "(RowData rowData){");
+ codegenAppender.appendLine("AbstractMessage message = null");
+ PbCodegenSerializer codegenSer =
+ PbCodegenSerializeFactory.getPbCodegenTopRowSer(
+ descriptor, rowType, formatContext);
+ String genCode =
+ codegenSer.codegen("message", "rowData", codegenAppender.currentIndent());
+ codegenAppender.appendSegment(genCode);
+ codegenAppender.appendLine("return message");
+ codegenAppender.end("}");
+ codegenAppender.end("}");
+
+ String printCode = codegenAppender.printWithLineNumber();
+ LOG.debug("Protobuf encode codegen: \n" + printCode);
+ Class generatedClass =
+ PbCodegenUtils.compileClass(
+ Thread.currentThread().getContextClassLoader(),
+ generatedPackageName + "." + generatedClassName,
+ codegenAppender.code());
+ encodeMethod =
+ generatedClass.getMethod(PbConstant.GENERATED_ENCODE_METHOD, RowData.class);
+ } catch (Exception ex) {
+ throw new PbCodegenException(ex);
+ }
+ }
+
+ public byte[] convertRowToProtoBinary(RowData rowData) throws Exception {
+ AbstractMessage message = (AbstractMessage) encodeMethod.invoke(null, rowData);
+ return message.toByteArray();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenAppender.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenAppender.java
new file mode 100644
index 0000000000000..915844bc7c3b1
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenAppender.java
@@ -0,0 +1,85 @@
+/*
+ * 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.util;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+/** Helper class which do code fragment concat. */
+public class PbCodegenAppender {
+ private static final int DEFAULT_INDENT = 2;
+ private final StringBuilder sb;
+ private int indent = 0;
+
+ public PbCodegenAppender() {
+ sb = new StringBuilder();
+ this.indent = 0;
+ }
+
+ public PbCodegenAppender(int indent) {
+ sb = new StringBuilder();
+ this.indent = indent;
+ }
+
+ public void begin() {
+ indent += DEFAULT_INDENT;
+ }
+
+ public void begin(String code) {
+ sb.append(indent()).append(code).append("\n");
+ begin();
+ }
+
+ public void end() {
+ indent -= DEFAULT_INDENT;
+ }
+
+ public void end(String code) {
+ end();
+ sb.append(indent()).append(code).append("\n");
+ }
+
+ public int currentIndent() {
+ return indent;
+ }
+
+ private String indent() {
+ return Strings.repeat(" ", indent);
+ }
+
+ public void appendLine(String code) {
+ sb.append(indent()).append(code).append(";\n");
+ }
+
+ public void appendSegment(String code) {
+ sb.append(code).append("\n");
+ }
+
+ public String code() {
+ return sb.toString();
+ }
+
+ public String printWithLineNumber() {
+ StringBuilder newSb = new StringBuilder();
+ String[] lines = sb.toString().split("\n");
+ for (int i = 0; i < lines.length; i++) {
+ newSb.append("Line ").append(i + 1).append(": ").append(lines[i]).append("\n");
+ }
+ return newSb.toString();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenUtils.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenUtils.java
new file mode 100644
index 0000000000000..9cf262ee4be56
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenUtils.java
@@ -0,0 +1,270 @@
+/*
+ * 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.util;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatContext;
+import org.apache.flink.formats.protobuf.serialize.PbCodegenSerializeFactory;
+import org.apache.flink.formats.protobuf.serialize.PbCodegenSerializer;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import org.codehaus.janino.SimpleCompiler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Codegen utils only used in protobuf format. */
+public class PbCodegenUtils {
+ private static final Logger LOG = LoggerFactory.getLogger(PbCodegenUtils.class);
+
+ /**
+ * @param flinkContainerCode code phrase which represent flink container type like row/array in
+ * codegen sections
+ * @param index the index number in flink container type
+ * @param eleType the element type
+ */
+ public static String flinkContainerElementCode(
+ String flinkContainerCode, String index, LogicalType eleType) {
+ switch (eleType.getTypeRoot()) {
+ case INTEGER:
+ return flinkContainerCode + ".getInt(" + index + ")";
+ case BIGINT:
+ return flinkContainerCode + ".getLong(" + index + ")";
+ case FLOAT:
+ return flinkContainerCode + ".getFloat(" + index + ")";
+ case DOUBLE:
+ return flinkContainerCode + ".getDouble(" + index + ")";
+ case BOOLEAN:
+ return flinkContainerCode + ".getBoolean(" + index + ")";
+ case VARCHAR:
+ case CHAR:
+ return flinkContainerCode + ".getString(" + index + ")";
+ case VARBINARY:
+ case BINARY:
+ return flinkContainerCode + ".getBinary(" + index + ")";
+ case ROW:
+ int size = eleType.getChildren().size();
+ return flinkContainerCode + ".getRow(" + index + ", " + size + ")";
+ case MAP:
+ return flinkContainerCode + ".getMap(" + index + ")";
+ case ARRAY:
+ return flinkContainerCode + ".getArray(" + index + ")";
+ default:
+ throw new IllegalArgumentException("Unsupported data type in schema: " + eleType);
+ }
+ }
+
+ /**
+ * Get java type str from {@link FieldDescriptor} which directly fetched from protobuf object.
+ *
+ * @return The returned code phrase will be used as java type str in codegen sections.
+ * @throws PbCodegenException
+ */
+ public static String getTypeStrFromProto(FieldDescriptor fd, boolean isList, String outerPrefix)
+ throws PbCodegenException {
+ String typeStr;
+ switch (fd.getJavaType()) {
+ case MESSAGE:
+ if (fd.isMapField()) {
+ // map
+ FieldDescriptor keyFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_KEY_NAME);
+ FieldDescriptor valueFd =
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_VALUE_NAME);
+ // key and value cannot be repeated
+ String keyTypeStr = getTypeStrFromProto(keyFd, false, outerPrefix);
+ String valueTypeStr = getTypeStrFromProto(valueFd, false, outerPrefix);
+ typeStr = "Map<" + keyTypeStr + "," + valueTypeStr + ">";
+ } else {
+ // simple message
+ typeStr = PbFormatUtils.getFullJavaName(fd.getMessageType(), outerPrefix);
+ }
+ break;
+ case INT:
+ typeStr = "Integer";
+ break;
+ case LONG:
+ typeStr = "Long";
+ break;
+ case STRING:
+ typeStr = "String";
+ break;
+ case ENUM:
+ typeStr = PbFormatUtils.getFullJavaName(fd.getEnumType(), outerPrefix);
+ break;
+ case FLOAT:
+ typeStr = "Float";
+ break;
+ case DOUBLE:
+ typeStr = "Double";
+ break;
+ case BYTE_STRING:
+ typeStr = "ByteString";
+ break;
+ case BOOLEAN:
+ typeStr = "Boolean";
+ break;
+ default:
+ throw new PbCodegenException("do not support field type: " + fd.getJavaType());
+ }
+ if (isList) {
+ return "List<" + typeStr + ">";
+ } else {
+ return typeStr;
+ }
+ }
+
+ /**
+ * Get java type str from {@link LogicalType} which directly fetched from flink type.
+ *
+ * @return The returned code phrase will be used as java type str in codegen sections.
+ */
+ public static String getTypeStrFromLogicType(LogicalType type) {
+ switch (type.getTypeRoot()) {
+ case INTEGER:
+ return "int";
+ case BIGINT:
+ return "long";
+ case FLOAT:
+ return "float";
+ case DOUBLE:
+ return "double";
+ case BOOLEAN:
+ return "boolean";
+ case VARCHAR:
+ case CHAR:
+ return "StringData";
+ case VARBINARY:
+ case BINARY:
+ return "byte[]";
+ case ROW:
+ return "RowData";
+ case MAP:
+ return "MapData";
+ case ARRAY:
+ return "ArrayData";
+ default:
+ throw new IllegalArgumentException("Unsupported data type in schema: " + type);
+ }
+ }
+
+ /**
+ * Get protobuf default value from {@link FieldDescriptor}.
+ *
+ * @return The java code phrase which represents default value calculation.
+ */
+ public static String pbDefaultValueCode(
+ FieldDescriptor fieldDescriptor, PbFormatContext pbFormatContext)
+ throws PbCodegenException {
+ String outerPrefix = pbFormatContext.getOuterPrefix();
+ String nullLiteral = pbFormatContext.getPbFormatConfig().getWriteNullStringLiterals();
+ switch (fieldDescriptor.getJavaType()) {
+ case MESSAGE:
+ return PbFormatUtils.getFullJavaName(fieldDescriptor.getMessageType(), outerPrefix)
+ + ".getDefaultInstance()";
+ case INT:
+ return "0";
+ case LONG:
+ return "0L";
+ case STRING:
+ return "\"" + nullLiteral + "\"";
+ case ENUM:
+ return PbFormatUtils.getFullJavaName(fieldDescriptor.getEnumType(), outerPrefix)
+ + ".values()[0]";
+ case FLOAT:
+ return "0.0f";
+ case DOUBLE:
+ return "0.0d";
+ case BYTE_STRING:
+ return "ByteString.EMPTY";
+ case BOOLEAN:
+ return "false";
+ default:
+ throw new PbCodegenException(
+ "do not support field type: " + fieldDescriptor.getJavaType());
+ }
+ }
+
+ /**
+ * This method will be called from serializer of flink array/map type because flink contains
+ * both array/map type in array format. Map/Array cannot contain null value in pb object then we
+ * must do conversion in case of null values in map/array type.
+ *
+ * @param flinkArrDataVar code phrase represent arrayData of arr type or keyData/valueData in
+ * map type.
+ * @param iVar the index in arrDataVar
+ * @param resultPbVar the returned pb variable name in codegen.
+ * @param elementPbFd {@link FieldDescriptor} of element type in proto object
+ * @param elementDataType {@link LogicalType} of element type in flink object
+ * @return The java code segment which represents field value retrieval.
+ */
+ public static String convertFlinkArrayElementToPbWithDefaultValueCode(
+ String flinkArrDataVar,
+ String iVar,
+ String resultPbVar,
+ FieldDescriptor elementPbFd,
+ LogicalType elementDataType,
+ PbFormatContext pbFormatContext,
+ int indent)
+ throws PbCodegenException {
+ PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+ int uid = varUid.getAndIncrement();
+ String flinkElementVar = "elementVar" + uid;
+ PbCodegenAppender appender = new PbCodegenAppender(indent);
+ String protoTypeStr =
+ PbCodegenUtils.getTypeStrFromProto(
+ elementPbFd, false, pbFormatContext.getOuterPrefix());
+ String dataTypeStr = PbCodegenUtils.getTypeStrFromLogicType(elementDataType);
+ appender.appendLine(protoTypeStr + " " + resultPbVar);
+ appender.begin("if(" + flinkArrDataVar + ".isNullAt(" + iVar + ")){");
+ appender.appendLine(
+ resultPbVar
+ + "="
+ + PbCodegenUtils.pbDefaultValueCode(elementPbFd, pbFormatContext));
+ appender.end("}else{");
+ appender.begin();
+ appender.appendLine(dataTypeStr + " " + flinkElementVar);
+ String flinkContainerElementCode =
+ PbCodegenUtils.flinkContainerElementCode(flinkArrDataVar, iVar, elementDataType);
+ appender.appendLine(flinkElementVar + " = " + flinkContainerElementCode);
+ PbCodegenSerializer codegenSer =
+ PbCodegenSerializeFactory.getPbCodegenSer(
+ elementPbFd, elementDataType, pbFormatContext);
+ String code = codegenSer.codegen(resultPbVar, flinkElementVar, appender.currentIndent());
+ appender.appendSegment(code);
+ appender.end("}");
+ return appender.code();
+ }
+
+ public static Class compileClass(ClassLoader classloader, String className, String code)
+ throws ClassNotFoundException {
+ SimpleCompiler simpleCompiler = new SimpleCompiler();
+ simpleCompiler.setParentClassLoader(classloader);
+ try {
+ simpleCompiler.cook(code);
+ } catch (Throwable t) {
+ LOG.error("Protobuf codegen compile error: \n" + code);
+ throw new InvalidProgramException(
+ "Program cannot be compiled. This is a bug. Please file an issue.", t);
+ }
+ return simpleCompiler.getClassLoader().loadClass(className);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenVarId.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenVarId.java
new file mode 100644
index 0000000000000..d7a9a19339ac6
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbCodegenVarId.java
@@ -0,0 +1,40 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Singleton class for generating variable suffix number globally to avoid conflict in codegen
+ * sections. It can be only used in protobuf format code.
+ */
+public class PbCodegenVarId {
+ private static final PbCodegenVarId varUid = new PbCodegenVarId();
+ private final AtomicInteger atomicInteger = new AtomicInteger();
+
+ private PbCodegenVarId() {}
+
+ public static PbCodegenVarId getInstance() {
+ return varUid;
+ }
+
+ public int getAndIncrement() {
+ return atomicInteger.getAndIncrement();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbFormatUtils.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbFormatUtils.java
new file mode 100644
index 0000000000000..84cd35c98cf83
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbFormatUtils.java
@@ -0,0 +1,105 @@
+/*
+ * 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.util;
+
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.ProtobufInternalUtils;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+ public static String getFullJavaName(Descriptors.Descriptor descriptor, String outerProtoName) {
+ if (null != descriptor.getContainingType()) {
+ // nested type
+ String parentJavaFullName =
+ getFullJavaName(descriptor.getContainingType(), outerProtoName);
+ return parentJavaFullName + "." + descriptor.getName();
+ } else {
+ // top level message
+ return outerProtoName + descriptor.getName();
+ }
+ }
+
+ public static String getFullJavaName(
+ Descriptors.EnumDescriptor enumDescriptor, String outerProtoName) {
+ if (null != enumDescriptor.getContainingType()) {
+ return getFullJavaName(enumDescriptor.getContainingType(), outerProtoName)
+ + "."
+ + enumDescriptor.getName();
+ } else {
+ return outerProtoName + enumDescriptor.getName();
+ }
+ }
+
+ public static boolean isSimpleType(LogicalType type) {
+ switch (type.getTypeRoot()) {
+ case BOOLEAN:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ case CHAR:
+ case VARCHAR:
+ case BINARY:
+ case VARBINARY:
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ public static String getStrongCamelCaseJsonName(String name) {
+ return ProtobufInternalUtils.underScoreToCamelCase(name, true);
+ }
+
+ public static String getOuterProtoPrefix(String name) {
+ name = name.replace('$', '.');
+ int index = name.lastIndexOf('.');
+ if (index != -1) {
+ // include dot
+ return name.substring(0, index + 1);
+ } else {
+ return "";
+ }
+ }
+
+ public static Descriptors.Descriptor getDescriptor(String className) {
+ try {
+ Class> pbClass =
+ Class.forName(className, true, Thread.currentThread().getContextClassLoader());
+ return (Descriptors.Descriptor)
+ pbClass.getMethod(PbConstant.PB_METHOD_GET_DESCRIPTOR).invoke(null);
+ } catch (Exception e) {
+ throw new IllegalArgumentException(
+ String.format("get %s descriptors error!", className), e);
+ }
+ }
+
+ public static boolean isRepeatedType(LogicalType type) {
+ return type instanceof MapType || type instanceof ArrayType;
+ }
+
+ public static boolean isArrayType(LogicalType type) {
+ return type instanceof ArrayType;
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbSchemaValidationUtils.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbSchemaValidationUtils.java
new file mode 100644
index 0000000000000..2bb177638540c
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbSchemaValidationUtils.java
@@ -0,0 +1,167 @@
+/*
+ * 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.util;
+
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.EnumMap;
+import java.util.EnumSet;
+
+/** Validation class to verify protobuf definition and flink schema. */
+public class PbSchemaValidationUtils {
+
+ private static final EnumMap> TYPE_MATCH_MAP =
+ new EnumMap<>(JavaType.class);
+
+ static {
+ TYPE_MATCH_MAP.put(JavaType.BOOLEAN, EnumSet.of(LogicalTypeRoot.BOOLEAN));
+ TYPE_MATCH_MAP.put(
+ JavaType.BYTE_STRING,
+ EnumSet.of(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+ TYPE_MATCH_MAP.put(JavaType.DOUBLE, EnumSet.of(LogicalTypeRoot.DOUBLE));
+ TYPE_MATCH_MAP.put(JavaType.FLOAT, EnumSet.of(LogicalTypeRoot.FLOAT));
+ TYPE_MATCH_MAP.put(
+ JavaType.ENUM,
+ EnumSet.of(
+ LogicalTypeRoot.VARCHAR,
+ LogicalTypeRoot.CHAR,
+ LogicalTypeRoot.TINYINT,
+ LogicalTypeRoot.SMALLINT,
+ LogicalTypeRoot.INTEGER,
+ LogicalTypeRoot.BIGINT));
+ TYPE_MATCH_MAP.put(
+ JavaType.STRING, EnumSet.of(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+ TYPE_MATCH_MAP.put(JavaType.INT, EnumSet.of(LogicalTypeRoot.INTEGER));
+ TYPE_MATCH_MAP.put(JavaType.LONG, EnumSet.of(LogicalTypeRoot.BIGINT));
+ }
+
+ public static void validate(Descriptors.Descriptor descriptor, RowType rowType) {
+ validateTypeMatch(descriptor, rowType);
+ }
+
+ /**
+ * Validate type match of row type.
+ *
+ * @param descriptor the {@link Descriptors.Descriptor} of the protobuf object.
+ * @param rowType the corresponding {@link RowType} to the {@link Descriptors.Descriptor}
+ */
+ private static void validateTypeMatch(Descriptors.Descriptor descriptor, RowType rowType) {
+ rowType.getFields()
+ .forEach(
+ rowField -> {
+ FieldDescriptor fieldDescriptor =
+ descriptor.findFieldByName(rowField.getName());
+ if (null != fieldDescriptor) {
+ validateTypeMatch(fieldDescriptor, rowField.getType());
+ } else {
+ throw new ValidationException(
+ "Column "
+ + rowField.getName()
+ + " does not exists in definition of proto class.");
+ }
+ });
+ }
+
+ /**
+ * Validate type match of general type.
+ *
+ * @param fd the {@link Descriptors.Descriptor} of the protobuf object.
+ * @param logicalType the corresponding {@link LogicalType} to the {@link FieldDescriptor}
+ */
+ private static void validateTypeMatch(FieldDescriptor fd, LogicalType logicalType) {
+ if (!fd.isRepeated()) {
+ if (fd.getJavaType() != JavaType.MESSAGE) {
+ // simple type
+ validateSimpleType(fd, logicalType.getTypeRoot());
+ } else {
+ // message type
+ if (!(logicalType instanceof RowType)) {
+ throw new ValidationException(
+ "Unexpected LogicalType: " + logicalType + ". It should be RowType");
+ }
+ validateTypeMatch(fd.getMessageType(), (RowType) logicalType);
+ }
+ } else {
+ if (fd.isMapField()) {
+ // map type
+ if (!(logicalType instanceof MapType)) {
+ throw new ValidationException(
+ "Unexpected LogicalType: " + logicalType + ". It should be MapType");
+ }
+ MapType mapType = (MapType) logicalType;
+ validateSimpleType(
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_KEY_NAME),
+ mapType.getKeyType().getTypeRoot());
+ validateTypeMatch(
+ fd.getMessageType().findFieldByName(PbConstant.PB_MAP_VALUE_NAME),
+ mapType.getValueType());
+ } else {
+ // array type
+ if (!(logicalType instanceof ArrayType)) {
+ throw new ValidationException(
+ "Unexpected LogicalType: " + logicalType + ". It should be ArrayType");
+ }
+ ArrayType arrayType = (ArrayType) logicalType;
+ if (fd.getJavaType() == JavaType.MESSAGE) {
+ // array message type
+ LogicalType elementType = arrayType.getElementType();
+ if (!(elementType instanceof RowType)) {
+ throw new ValidationException(
+ "Unexpected logicalType: "
+ + elementType
+ + ". It should be RowType");
+ }
+ validateTypeMatch(fd.getMessageType(), (RowType) elementType);
+ } else {
+ // array simple type
+ validateSimpleType(fd, arrayType.getElementType().getTypeRoot());
+ }
+ }
+ }
+ }
+
+ /**
+ * Only validate type match for simple type like int, long, string, boolean.
+ *
+ * @param fd {@link FieldDescriptor} in proto descriptor
+ * @param logicalTypeRoot {@link LogicalTypeRoot} of row element
+ */
+ private static void validateSimpleType(FieldDescriptor fd, LogicalTypeRoot logicalTypeRoot) {
+ if (!TYPE_MATCH_MAP.containsKey(fd.getJavaType())) {
+ throw new ValidationException("Unsupported protobuf java type: " + fd.getJavaType());
+ }
+ if (TYPE_MATCH_MAP.get(fd.getJavaType()).stream().noneMatch(x -> x == logicalTypeRoot)) {
+ throw new ValidationException(
+ "Protobuf field type does not match column type, "
+ + fd.getJavaType()
+ + "(protobuf) is not compatible of "
+ + logicalTypeRoot);
+ }
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbToRowTypeUtil.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbToRowTypeUtil.java
new file mode 100644
index 0000000000000..da8989b0ad9de
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/util/PbToRowTypeUtil.java
@@ -0,0 +1,110 @@
+/*
+ * 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.util;
+
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+/** Generate Row type information according to pb descriptors. */
+public class PbToRowTypeUtil {
+ public static RowType generateRowType(Descriptors.Descriptor root) {
+ return generateRowType(root, false);
+ }
+
+ public static RowType generateRowType(Descriptors.Descriptor root, boolean enumAsInt) {
+ int size = root.getFields().size();
+ LogicalType[] types = new LogicalType[size];
+ String[] rowFieldNames = new String[size];
+
+ for (int i = 0; i < size; i++) {
+ FieldDescriptor field = root.getFields().get(i);
+ rowFieldNames[i] = field.getName();
+ types[i] = generateFieldTypeInformation(field, enumAsInt);
+ }
+ return RowType.of(types, rowFieldNames);
+ }
+
+ private static LogicalType generateFieldTypeInformation(
+ FieldDescriptor field, boolean enumAsInt) {
+ JavaType fieldType = field.getJavaType();
+ LogicalType type;
+ if (fieldType.equals(JavaType.MESSAGE)) {
+ if (field.isMapField()) {
+ MapType mapType =
+ new MapType(
+ generateFieldTypeInformation(
+ field.getMessageType()
+ .findFieldByName(PbConstant.PB_MAP_KEY_NAME),
+ enumAsInt),
+ generateFieldTypeInformation(
+ field.getMessageType()
+ .findFieldByName(PbConstant.PB_MAP_VALUE_NAME),
+ enumAsInt));
+ return mapType;
+ } else if (field.isRepeated()) {
+ return new ArrayType(generateRowType(field.getMessageType()));
+ } else {
+ return generateRowType(field.getMessageType());
+ }
+ } else {
+ if (fieldType.equals(JavaType.STRING)) {
+ type = new VarCharType(Integer.MAX_VALUE);
+ } else if (fieldType.equals(JavaType.LONG)) {
+ type = new BigIntType();
+ } else if (fieldType.equals(JavaType.BOOLEAN)) {
+ type = new BooleanType();
+ } else if (fieldType.equals(JavaType.INT)) {
+ type = new IntType();
+ } else if (fieldType.equals(JavaType.DOUBLE)) {
+ type = new DoubleType();
+ } else if (fieldType.equals(JavaType.FLOAT)) {
+ type = new FloatType();
+ } else if (fieldType.equals(JavaType.ENUM)) {
+ if (enumAsInt) {
+ type = new IntType();
+ } else {
+ type = new VarCharType(Integer.MAX_VALUE);
+ }
+ } else if (fieldType.equals(JavaType.BYTE_STRING)) {
+ type = new VarBinaryType(Integer.MAX_VALUE);
+ } else {
+ throw new ValidationException("unsupported field type: " + fieldType);
+ }
+ if (field.isRepeated()) {
+ return new ArrayType(type);
+ }
+ return type;
+ }
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 0000000000000..48f9681e27882
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,15 @@
+# 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.
+org.apache.flink.formats.protobuf.PbFormatFactory
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapProtoToRowTest.java
new file mode 100644
index 0000000000000..44c4588d7abff
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapProtoToRowTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.MapTest;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of proto map data to flink internal data. */
+public class MapProtoToRowTest {
+ @Test
+ public void testMessage() throws Exception {
+ MapTest.InnerMessageTest innerMessageTest =
+ MapTest.InnerMessageTest.newBuilder().setA(1).setB(2).build();
+ MapTest mapTest =
+ MapTest.newBuilder()
+ .setA(1)
+ .putMap1("a", "b")
+ .putMap1("c", "d")
+ .putMap2("f", innerMessageTest)
+ .putMap3("e", ByteString.copyFrom(new byte[] {1, 2, 3}))
+ .build();
+ RowData row = ProtobufTestHelper.pbBytesToRow(MapTest.class, mapTest.toByteArray());
+
+ MapData map1 = row.getMap(1);
+ assertEquals("a", map1.keyArray().getString(0).toString());
+ assertEquals("b", map1.valueArray().getString(0).toString());
+ assertEquals("c", map1.keyArray().getString(1).toString());
+ assertEquals("d", map1.valueArray().getString(1).toString());
+
+ MapData map2 = row.getMap(2);
+ assertEquals("f", map2.keyArray().getString(0).toString());
+ RowData rowData2 = map2.valueArray().getRow(0, 2);
+
+ assertEquals(1, rowData2.getInt(0));
+ assertEquals(2L, rowData2.getLong(1));
+
+ MapData map3 = row.getMap(3);
+ assertEquals("e", map3.keyArray().getString(0).toString());
+ assertArrayEquals(new byte[] {1, 2, 3}, map3.valueArray().getBinary(0));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapRowToProtoTest.java
new file mode 100644
index 0000000000000..ec120adc828c4
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MapRowToProtoTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.MapTest;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of flink internal map data to proto data. */
+public class MapRowToProtoTest {
+ @Test
+ public void testSimple() throws Exception {
+ Map map1 = new HashMap<>();
+ map1.put(StringData.fromString("a"), StringData.fromString("b"));
+ Map map2 = new HashMap<>();
+ map2.put(StringData.fromString("c"), GenericRowData.of(1, 2L));
+ Map map3 = new HashMap<>();
+ map3.put(StringData.fromString("e"), new byte[] {1, 2, 3});
+ RowData row =
+ GenericRowData.of(
+ 1,
+ new GenericMapData(map1),
+ new GenericMapData(map2),
+ new GenericMapData(map3));
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, MapTest.class);
+
+ MapTest mapTest = MapTest.parseFrom(bytes);
+ assertEquals(1, mapTest.getA());
+ assertEquals("b", mapTest.getMap1Map().get("a"));
+ MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("c");
+ assertEquals(1, innerMessageTest.getA());
+ assertEquals(2L, innerMessageTest.getB());
+ assertEquals(ByteString.copyFrom(new byte[] {1, 2, 3}), mapTest.getMap3Map().get("e"));
+ }
+
+ @Test
+ public void testNull() throws Exception {
+ RowData row = GenericRowData.of(1, null, null, null);
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, MapTest.class);
+ MapTest mapTest = MapTest.parseFrom(bytes);
+ Map map = mapTest.getMap1Map();
+ assertEquals(0, map.size());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaNoOuterNoMultiProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaNoOuterNoMultiProtoToRowTest.java
new file mode 100644
index 0000000000000..32269393e17c6
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaNoOuterNoMultiProtoToRowTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema;
+import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema;
+import org.apache.flink.formats.protobuf.testproto.TestSimpleNoouterNomulti;
+import org.apache.flink.formats.protobuf.util.PbToRowTypeUtil;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.junit.Test;
+
+/**
+ * Test class for below case
+ *
+ *
+ * syntax = "proto2";
+ * package org.apache.flink.formats.protobuf.testproto;
+ * option java_package = "org.apache.flink.formats.protobuf.testproto";
+ * message SimpleTestNoouterNomulti {
+ *
+ *
+ * It is valid proto definition.
+ */
+public class MetaNoOuterNoMultiProtoToRowTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowType rowType =
+ PbToRowTypeUtil.generateRowType(
+ TestSimpleNoouterNomulti.SimpleTestNoouterNomulti.getDescriptor());
+ PbFormatConfig formatConfig =
+ new PbFormatConfig(
+ TestSimpleNoouterNomulti.SimpleTestNoouterNomulti.class.getName(),
+ false,
+ false,
+ "");
+ new PbRowDataDeserializationSchema(rowType, InternalTypeInfo.of(rowType), formatConfig)
+ .open(null);
+
+ new PbRowDataSerializationSchema(rowType, formatConfig).open(null);
+ // validation success
+ }
+
+ @Test
+ public void testOuterClassName() throws Exception {
+ RowType rowType =
+ PbToRowTypeUtil.generateRowType(
+ TestSimpleNoouterNomulti.SimpleTestNoouterNomulti.getDescriptor());
+ PbFormatConfig formatConfig =
+ new PbFormatConfig(
+ TestSimpleNoouterNomulti.SimpleTestNoouterNomulti.class.getName(),
+ false,
+ false,
+ "");
+ new PbRowDataDeserializationSchema(rowType, InternalTypeInfo.of(rowType), formatConfig)
+ .open(null);
+ new PbRowDataSerializationSchema(rowType, formatConfig).open(null);
+ // validation success
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterMultiTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterMultiTest.java
new file mode 100644
index 0000000000000..932964804e291
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterMultiTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema;
+import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema;
+import org.apache.flink.formats.protobuf.testproto.SimpleTestOuterMulti;
+import org.apache.flink.formats.protobuf.util.PbToRowTypeUtil;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.junit.Test;
+
+/**
+ * Test class for below case
+ *
+ *
+ * syntax = "proto2";
+ * package org.apache.flink.formats.protobuf.testproto;
+ * option java_package = "org.apache.flink.formats.protobuf.testproto";
+ * option java_outer_classname = "SimpleTestOuterMultiProto";
+ * option java_multiple_files = true;
+ * message SimpleTestOuterMulti {
+ *
+ *
+ * It is valid proto definition.
+ */
+public class MetaOuterMultiTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowType rowType = PbToRowTypeUtil.generateRowType(SimpleTestOuterMulti.getDescriptor());
+ PbFormatConfig formatConfig =
+ new PbFormatConfig(SimpleTestOuterMulti.class.getName(), false, false, "");
+ new PbRowDataDeserializationSchema(rowType, InternalTypeInfo.of(rowType), formatConfig)
+ .open(null);
+
+ new PbRowDataSerializationSchema(rowType, formatConfig).open(null);
+ // validation success
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterNoMultiTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterNoMultiTest.java
new file mode 100644
index 0000000000000..7e70a5561b52c
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MetaOuterNoMultiTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema;
+import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema;
+import org.apache.flink.formats.protobuf.testproto.SimpleTestOuterNomultiProto;
+import org.apache.flink.formats.protobuf.util.PbToRowTypeUtil;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.junit.Test;
+
+/**
+ * Test class for below case
+ *
+ *
+ * syntax = "proto2";
+ * package org.apache.flink.formats.protobuf.testproto;
+ * option java_package = "org.apache.flink.formats.protobuf.testproto";
+ * option java_outer_classname = "SimpleTestOuterNomultiProto";
+ * message SimpleTestOuterNomulti {
+ *
+ *
+ * It is valid proto definition.
+ */
+public class MetaOuterNoMultiTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowType rowType =
+ PbToRowTypeUtil.generateRowType(
+ SimpleTestOuterNomultiProto.SimpleTestOuterNomulti.getDescriptor());
+ PbFormatConfig formatConfig =
+ new PbFormatConfig(
+ SimpleTestOuterNomultiProto.SimpleTestOuterNomulti.class.getName(),
+ false,
+ false,
+ "");
+ new PbRowDataDeserializationSchema(rowType, InternalTypeInfo.of(rowType), formatConfig)
+ .open(null);
+
+ new PbRowDataSerializationSchema(rowType, formatConfig).open(null);
+ // validation success
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageProtoToRowTest.java
new file mode 100644
index 0000000000000..bea0c3be378a3
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageProtoToRowTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.MultipleLevelMessageTest;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/** Test conversion of multiple level of proto nested message data to flink internal data. */
+public class MultiLevelMessageProtoToRowTest {
+ @Test
+ public void testMessage() throws Exception {
+ MultipleLevelMessageTest.InnerMessageTest1.InnerMessageTest2 innerMessageTest2 =
+ MultipleLevelMessageTest.InnerMessageTest1.InnerMessageTest2.newBuilder()
+ .setA(1)
+ .setB(2L)
+ .build();
+ MultipleLevelMessageTest.InnerMessageTest1 innerMessageTest =
+ MultipleLevelMessageTest.InnerMessageTest1.newBuilder()
+ .setC(false)
+ .setA(innerMessageTest2)
+ .build();
+ MultipleLevelMessageTest multipleLevelMessageTest =
+ MultipleLevelMessageTest.newBuilder().setD(innerMessageTest).setA(1).build();
+
+ RowData row =
+ ProtobufTestHelper.pbBytesToRow(
+ MultipleLevelMessageTest.class, multipleLevelMessageTest.toByteArray());
+
+ assertEquals(4, row.getArity());
+ RowData subRow = (RowData) row.getRow(3, 2);
+ assertFalse(subRow.getBoolean(1));
+
+ RowData subSubRow = (RowData) subRow.getRow(0, 2);
+ assertEquals(1, subSubRow.getInt(0));
+ assertEquals(2L, subSubRow.getLong(1));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageRowToProtoTest.java
new file mode 100644
index 0000000000000..97c672f1a58d6
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/MultiLevelMessageRowToProtoTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.MultipleLevelMessageTest;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/** Test conversion of flink internal nested row data to proto data. */
+public class MultiLevelMessageRowToProtoTest {
+ @Test
+ public void testMultipleLevelMessage() throws Exception {
+ RowData subSubRow = GenericRowData.of(1, 2L);
+ RowData subRow = GenericRowData.of(subSubRow, false);
+ RowData row = GenericRowData.of(1, 2L, false, subRow);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, MultipleLevelMessageTest.class);
+ MultipleLevelMessageTest test = MultipleLevelMessageTest.parseFrom(bytes);
+
+ assertFalse(test.getD().getC());
+ assertEquals(1, test.getD().getA().getA());
+ assertEquals(2L, test.getD().getA().getB());
+ assertEquals(1, test.getA());
+ }
+
+ @Test
+ public void testNull() throws Exception {
+ RowData row = GenericRowData.of(1, 2L, false, null);
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, MultipleLevelMessageTest.class);
+
+ MultipleLevelMessageTest test = MultipleLevelMessageTest.parseFrom(bytes);
+
+ MultipleLevelMessageTest.InnerMessageTest1 empty =
+ MultipleLevelMessageTest.InnerMessageTest1.newBuilder().build();
+ assertEquals(empty, test.getD());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NoJavaPackageProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NoJavaPackageProtoToRowTest.java
new file mode 100644
index 0000000000000..b5499b8210319
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NoJavaPackageProtoToRowTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.proto.SimpleTestNoJavaPackage;
+
+import org.junit.Test;
+
+/** test no java_package. */
+public class NoJavaPackageProtoToRowTest {
+ @Test
+ public void testMessage() throws Exception {
+ SimpleTestNoJavaPackage simple = SimpleTestNoJavaPackage.newBuilder().build();
+ ProtobufTestHelper.pbBytesToRow(SimpleTestNoJavaPackage.class, simple.toByteArray());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NullValueToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NullValueToProtoTest.java
new file mode 100644
index 0000000000000..bb9ef49e57936
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/NullValueToProtoTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.NullTest;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+
+import static org.apache.flink.formats.protobuf.ProtobufTestHelper.mapOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test conversion of null values from flink internal data to proto data. Proto data does not permit
+ * null values in array/map data.
+ */
+public class NullValueToProtoTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ // string
+ new GenericMapData(
+ mapOf(
+ StringData.fromString("key"),
+ null,
+ StringData.fromString(""),
+ StringData.fromString("value"))),
+ // int32
+ new GenericMapData(mapOf(0, 1, 1, null)),
+ // int64
+ new GenericMapData(mapOf(0L, 1L, 1L, null)),
+ // boolean
+ new GenericMapData(mapOf(false, true, true, null)),
+ // float
+ new GenericMapData(mapOf(StringData.fromString("key"), null)),
+ // double
+ new GenericMapData(mapOf(StringData.fromString("key"), null)),
+ // enum
+ new GenericMapData(mapOf(StringData.fromString("key"), null)),
+ // message
+ new GenericMapData(mapOf(StringData.fromString("key"), null)),
+ // bytes
+ new GenericMapData(mapOf(StringData.fromString("key"), null)),
+ // string
+ new GenericArrayData(new Object[] {null}),
+ // int
+ new GenericArrayData(new Object[] {null}),
+ // long
+ new GenericArrayData(new Object[] {null}),
+ // boolean
+ new GenericArrayData(new Object[] {null}),
+ // float
+ new GenericArrayData(new Object[] {null}),
+ // double
+ new GenericArrayData(new Object[] {null}),
+ // enum
+ new GenericArrayData(new Object[] {null}),
+ // message, cannot be null
+ new GenericArrayData(new Object[] {null}),
+ // bytes, cannot be null
+ new GenericArrayData(new Object[] {null}));
+ byte[] bytes =
+ ProtobufTestHelper.rowToPbBytes(
+ row,
+ NullTest.class,
+ new PbFormatConfig(NullTest.class.getName(), false, false, ""),
+ false);
+ NullTest nullTest = NullTest.parseFrom(bytes);
+ // string map
+ assertEquals(2, nullTest.getStringMapCount());
+ assertTrue(nullTest.getStringMapMap().containsKey(""));
+ assertTrue(nullTest.getStringMapMap().containsKey("key"));
+ assertEquals("value", nullTest.getStringMapMap().get(""));
+ assertEquals("", nullTest.getStringMapMap().get("key"));
+ // int32 map
+ assertEquals(2, nullTest.getIntMapCount());
+ assertTrue(nullTest.getIntMapMap().containsKey(0));
+ assertTrue(nullTest.getIntMapMap().containsKey(1));
+ assertEquals(Integer.valueOf(1), nullTest.getIntMapMap().get(0));
+ assertEquals(Integer.valueOf(0), nullTest.getIntMapMap().get(1));
+ // int64 map
+ assertEquals(2, nullTest.getIntMapCount());
+ assertTrue(nullTest.getLongMapMap().containsKey(0L));
+ assertTrue(nullTest.getLongMapMap().containsKey(1L));
+ assertEquals(Long.valueOf(1L), nullTest.getLongMapMap().get(0L));
+ assertEquals(Long.valueOf(0L), nullTest.getLongMapMap().get(1L));
+ // bool map
+ assertEquals(2, nullTest.getBooleanMapCount());
+ assertTrue(nullTest.getBooleanMapMap().containsKey(false));
+ assertTrue(nullTest.getBooleanMapMap().containsKey(true));
+ assertEquals(Boolean.TRUE, nullTest.getBooleanMapMap().get(false));
+ assertEquals(Boolean.FALSE, nullTest.getBooleanMapMap().get(true));
+ // float map
+ assertEquals(1, nullTest.getFloatMapCount());
+ assertEquals(Float.valueOf(0.0f), nullTest.getFloatMapMap().get("key"));
+ // double map
+ assertEquals(1, nullTest.getDoubleMapCount());
+ assertEquals(Double.valueOf(0.0), nullTest.getDoubleMapMap().get("key"));
+ // enum map
+ assertEquals(1, nullTest.getEnumMapCount());
+ assertEquals(NullTest.Corpus.UNIVERSAL, nullTest.getEnumMapMap().get("key"));
+ // message map
+ assertEquals(1, nullTest.getMessageMapCount());
+ assertEquals(
+ NullTest.InnerMessageTest.getDefaultInstance(),
+ nullTest.getMessageMapMap().get("key"));
+ // bytes map
+ assertEquals(1, nullTest.getBytesMapCount());
+ assertEquals(ByteString.EMPTY, nullTest.getBytesMapMap().get("key"));
+
+ // string array
+ assertEquals(1, nullTest.getStringArrayCount());
+ assertEquals("", nullTest.getStringArrayList().get(0));
+ // int array
+ assertEquals(1, nullTest.getIntArrayCount());
+ assertEquals(Integer.valueOf(0), nullTest.getIntArrayList().get(0));
+ // long array
+ assertEquals(1, nullTest.getLongArrayCount());
+ assertEquals(Long.valueOf(0L), nullTest.getLongArrayList().get(0));
+ // float array
+ assertEquals(1, nullTest.getFloatArrayCount());
+ assertEquals(Float.valueOf(0), nullTest.getFloatArrayList().get(0));
+ // double array
+ assertEquals(1, nullTest.getDoubleArrayCount());
+ assertEquals(Double.valueOf(0), nullTest.getDoubleArrayList().get(0));
+ // boolean array
+ assertEquals(1, nullTest.getBooleanArrayCount());
+ assertEquals(Boolean.FALSE, nullTest.getBooleanArrayList().get(0));
+ // enum array
+ assertEquals(1, nullTest.getEnumArrayCount());
+ assertEquals(NullTest.Corpus.UNIVERSAL, nullTest.getEnumArrayList().get(0));
+ // message array
+ assertEquals(1, nullTest.getMessageArrayCount());
+ assertEquals(
+ NullTest.InnerMessageTest.getDefaultInstance(),
+ nullTest.getMessageArrayList().get(0));
+ // bytes array
+ assertEquals(1, nullTest.getBytesArrayCount());
+ assertEquals(ByteString.EMPTY, nullTest.getBytesArrayList().get(0));
+ }
+
+ @Test
+ public void testNullStringLiteral() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ // string
+ new GenericMapData(
+ mapOf(
+ StringData.fromString("key"),
+ null,
+ null,
+ StringData.fromString("value"))),
+ // int32
+ null,
+ // int64
+ null,
+ // boolean
+ null,
+ // float
+ null,
+ // double
+ null,
+ // enum
+ null,
+ // message
+ null,
+ // bytes
+ null,
+ // string
+ null,
+ // int
+ null,
+ // long
+ null,
+ // boolean
+ null,
+ // float
+ null,
+ // double
+ null,
+ // enum
+ null,
+ // message, cannot be null
+ null,
+ // bytes, cannot be null
+ null);
+ byte[] bytes =
+ ProtobufTestHelper.rowToPbBytes(
+ row,
+ NullTest.class,
+ new PbFormatConfig(NullTest.class.getName(), false, false, "NULL"),
+ false);
+ NullTest nullTest = NullTest.parseFrom(bytes);
+ assertEquals("NULL", nullTest.getStringMapMap().get("key"));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofProtoToRowTest.java
new file mode 100644
index 0000000000000..04d29c42eb8fc
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofProtoToRowTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.OneofTest;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Test conversion of proto one_of data to flink internal data. */
+public class OneofProtoToRowTest {
+ @Test
+ public void testSimple() throws Exception {
+ OneofTest oneofTest = OneofTest.newBuilder().setA(1).setB(2).build();
+ RowData row = ProtobufTestHelper.pbBytesToRow(OneofTest.class, oneofTest.toByteArray());
+ assertTrue(row.isNullAt(0));
+ assertEquals(2, row.getInt(1));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofRowToProtoTest.java
new file mode 100644
index 0000000000000..3b63e7f695d64
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/OneofRowToProtoTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.OneofTest;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/** Test conversion of flink internal map data to one_of proto data. */
+public class OneofRowToProtoTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowData row = GenericRowData.of(1, 2);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, OneofTest.class);
+ OneofTest oneofTest = OneofTest.parseFrom(bytes);
+ assertFalse(oneofTest.hasA());
+ assertEquals(2, oneofTest.getB());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/Pb3ToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/Pb3ToRowTest.java
new file mode 100644
index 0000000000000..960b215ad4c7d
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/Pb3ToRowTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.Pb3Test;
+import org.apache.flink.formats.protobuf.testproto.Pb3Test.Corpus;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Test conversion of proto3 data to flink internal data. Default values after conversion is tested
+ * especially.
+ */
+public class Pb3ToRowTest {
+ @Test
+ public void testDeserialization() throws Exception {
+ Pb3Test.InnerMessageTest innerMessageTest =
+ Pb3Test.InnerMessageTest.newBuilder().setA(1).setB(2).build();
+ Pb3Test mapTest =
+ Pb3Test.newBuilder()
+ .setA(1)
+ .setB(2L)
+ .setC("haha")
+ .setD(1.1f)
+ .setE(1.2)
+ .setF(Corpus.IMAGES)
+ .setG(innerMessageTest)
+ .addH(innerMessageTest)
+ .setI(ByteString.copyFrom(new byte[] {100}))
+ .putMap1("a", "b")
+ .putMap1("c", "d")
+ .putMap2("f", innerMessageTest)
+ .build();
+
+ RowData row = ProtobufTestHelper.pbBytesToRow(Pb3Test.class, mapTest.toByteArray());
+
+ assertEquals(1, row.getInt(0));
+ assertEquals(2L, row.getLong(1));
+ assertEquals("haha", row.getString(2).toString());
+ assertEquals(Float.valueOf(1.1f), Float.valueOf(row.getFloat(3)));
+ assertEquals(Double.valueOf(1.2), Double.valueOf(row.getDouble(4)));
+ assertEquals("IMAGES", row.getString(5).toString());
+
+ RowData rowData = row.getRow(6, 2);
+ assertEquals(1, rowData.getInt(0));
+ assertEquals(2L, rowData.getInt(1));
+
+ rowData = row.getArray(7).getRow(0, 2);
+ assertEquals(1, rowData.getInt(0));
+ assertEquals(2L, rowData.getInt(1));
+
+ assertEquals(100, row.getBinary(8)[0]);
+
+ MapData map1 = row.getMap(9);
+ assertEquals("a", map1.keyArray().getString(0).toString());
+ assertEquals("b", map1.valueArray().getString(0).toString());
+ assertEquals("c", map1.keyArray().getString(1).toString());
+ assertEquals("d", map1.valueArray().getString(1).toString());
+
+ MapData map2 = row.getMap(10);
+ assertEquals("f", map2.keyArray().getString(0).toString());
+ rowData = map2.valueArray().getRow(0, 2);
+
+ assertEquals(1, rowData.getInt(0));
+ assertEquals(2L, rowData.getLong(1));
+ }
+
+ @Test
+ public void testReadDefaultValues() throws Exception {
+ Pb3Test pb3Test = Pb3Test.newBuilder().build();
+ RowData row = ProtobufTestHelper.pbBytesToRow(Pb3Test.class, pb3Test.toByteArray());
+
+ assertFalse(row.isNullAt(0));
+ assertFalse(row.isNullAt(1));
+ assertFalse(row.isNullAt(2));
+ assertFalse(row.isNullAt(3));
+ assertFalse(row.isNullAt(4));
+ assertFalse(row.isNullAt(5));
+ assertFalse(row.isNullAt(6));
+ assertFalse(row.isNullAt(7));
+ assertFalse(row.isNullAt(8));
+ assertFalse(row.isNullAt(9));
+ assertFalse(row.isNullAt(10));
+
+ assertEquals(0, row.getInt(0));
+ assertEquals(0L, row.getLong(1));
+ assertEquals("", row.getString(2).toString());
+ assertEquals(Float.valueOf(0.0f), Float.valueOf(row.getFloat(3)));
+ assertEquals(Double.valueOf(0.0d), Double.valueOf(row.getDouble(4)));
+ assertEquals("UNIVERSAL", row.getString(5).toString());
+
+ RowData rowData = row.getRow(6, 2);
+ assertEquals(0, rowData.getInt(0));
+ assertEquals(0L, rowData.getLong(1));
+
+ assertEquals(0, row.getArray(7).size());
+
+ assertEquals(0, row.getBinary(8).length);
+
+ assertEquals(0, row.getMap(9).size());
+ assertEquals(0, row.getMap(10).size());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufSQLITCaseTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufSQLITCaseTest.java
new file mode 100644
index 0000000000000..f4f279b2af2ac
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufSQLITCaseTest.java
@@ -0,0 +1,331 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.table.TestProtobufTestStore;
+import org.apache.flink.formats.protobuf.testproto.MapTest;
+import org.apache.flink.formats.protobuf.testproto.Pb3Test;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+/** Integration SQL test for protobuf. */
+public class ProtobufSQLITCaseTest extends BatchTestBase {
+
+ private MapTest getProtoTestObject() {
+ MapTest.InnerMessageTest innerMessageTest =
+ MapTest.InnerMessageTest.newBuilder().setA(1).setB(2).build();
+ MapTest mapTest =
+ MapTest.newBuilder()
+ .setA(1)
+ .putMap1("a", "b")
+ .putMap1("c", "d")
+ .putMap2("f", innerMessageTest)
+ .build();
+ return mapTest;
+ }
+
+ @Test
+ public void testSource() {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(getProtoTestObject().toByteArray());
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest'"
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ Row row = result.collect().next();
+ assertEquals(1, (int) row.getField(0));
+ Map map1 = (Map) row.getField(1);
+ assertEquals("b", map1.get("a"));
+ assertEquals("d", map1.get("c"));
+ Map map2 = (Map) row.getField(2);
+ Row innerRow = map2.get("f");
+ assertEquals(1, innerRow.getField(0));
+ assertEquals(2L, innerRow.getField(1));
+ }
+
+ @Test
+ public void testSourceNotIgnoreParseError() throws InterruptedException {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ // pass an incompatible bytes
+ TestProtobufTestStore.sourcePbInputs.add(new byte[] {127, 127, 127, 127, 127});
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest'"
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ try {
+ result.await();
+ } catch (Exception ex) {
+ return;
+ }
+ fail("executeSql should raise exception");
+ }
+
+ @Test
+ public void testSourceIgnoreParseError() throws InterruptedException, ExecutionException {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ // pass an incompatible bytes
+ TestProtobufTestStore.sourcePbInputs.add(new byte[] {127, 127, 127, 127, 127});
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest',"
+ + " 'protobuf.ignore-parse-errors' = 'true'"
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ CloseableIterator iterator = result.collect();
+ assertFalse(iterator.hasNext());
+ }
+
+ @Test
+ public void testSourceWithDefaultValueOfPb2WhenTrue() {
+ MapTest mapTest = MapTest.newBuilder().build();
+
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(mapTest.toByteArray());
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ + " 'protobuf.read-default-values' = 'true' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ Row row = result.collect().next();
+ assertEquals(0, (int) row.getField(0));
+ }
+
+ @Test
+ public void testSourceWithDefaultValueOfPb2WhenFalse() {
+ MapTest mapTest = MapTest.newBuilder().build();
+
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(mapTest.toByteArray());
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ + " 'protobuf.read-default-values' = 'false' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ Row row = result.collect().next();
+ assertNull(row.getField(0));
+ }
+
+ @Test
+ public void testSourceWithDefaultValueOfPb3WhenTrue() {
+ Pb3Test pb3Test = Pb3Test.newBuilder().build();
+
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(pb3Test.toByteArray());
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int,"
+ + " b bigint,"
+ + " c string,"
+ + " d float"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.Pb3Test', "
+ + " 'protobuf.read-default-values' = 'true' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ Row row = result.collect().next();
+ assertEquals(0, (int) row.getField(0));
+ }
+
+ @Test
+ public void testSourceWithDefaultValueOfPb3WhenFalse() {
+ Pb3Test pb3Test = Pb3Test.newBuilder().build();
+
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(pb3Test.toByteArray());
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_source ( "
+ + " a int,"
+ + " b bigint,"
+ + " c string,"
+ + " d float"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.Pb3Test', "
+ + " 'protobuf.read-default-values' = 'false' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult result = tEnv().executeSql("select * from bigdata_source");
+ Row row = result.collect().next();
+ assertEquals(0, (int) row.getField(0));
+ }
+
+ @Test
+ public void testSink() throws Exception {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(getProtoTestObject().toByteArray());
+ TestProtobufTestStore.sinkResults.clear();
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_sink ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest'"
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult tableResult =
+ tEnv().executeSql(
+ "insert into bigdata_sink select 1, map['a', 'b', 'c', 'd'], map['f', row(1,cast(2 as bigint))] ");
+ tableResult.await();
+
+ byte[] bytes = TestProtobufTestStore.sinkResults.get(0);
+ MapTest mapTest = MapTest.parseFrom(bytes);
+ assertEquals(1, mapTest.getA());
+ assertEquals("b", mapTest.getMap1Map().get("a"));
+ assertEquals("d", mapTest.getMap1Map().get("c"));
+ MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("f");
+ assertEquals(1, innerMessageTest.getA());
+ assertEquals(2L, innerMessageTest.getB());
+ }
+
+ @Test
+ public void testSinkWithNullLiteral() throws Exception {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(getProtoTestObject().toByteArray());
+ TestProtobufTestStore.sinkResults.clear();
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_sink ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ + " 'protobuf.write-null-string-literal' = 'NULL' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult tableResult =
+ tEnv().executeSql(
+ "insert into bigdata_sink select 1, map['a', null], map['b', cast(null as row)]");
+ tableResult.await();
+
+ byte[] bytes = TestProtobufTestStore.sinkResults.get(0);
+ MapTest mapTest = MapTest.parseFrom(bytes);
+ assertEquals(1, mapTest.getA());
+ assertEquals("NULL", mapTest.getMap1Map().get("a"));
+ MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("b");
+ assertEquals(MapTest.InnerMessageTest.getDefaultInstance(), innerMessageTest);
+ }
+
+ @Test
+ public void testSinkWithNullLiteralWithEscape() throws Exception {
+ TestProtobufTestStore.sourcePbInputs.clear();
+ TestProtobufTestStore.sourcePbInputs.add(getProtoTestObject().toByteArray());
+ TestProtobufTestStore.sinkResults.clear();
+
+ env().setParallelism(1);
+ String sql =
+ "create table bigdata_sink ( "
+ + " a int, "
+ + " map1 map,"
+ + " map2 map>"
+ + ") with ("
+ + " 'connector' = 'protobuf-test-connector', "
+ + " 'format' = 'protobuf', "
+ + " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ + " 'protobuf.write-null-string-literal' = '\\\"NULL\\\"' "
+ + ")";
+ tEnv().executeSql(sql);
+ TableResult tableResult =
+ tEnv().executeSql(
+ "insert into bigdata_sink select 1, map['a', null], map['b', cast(null as row)]");
+ tableResult.await();
+
+ byte[] bytes = TestProtobufTestStore.sinkResults.get(0);
+ MapTest mapTest = MapTest.parseFrom(bytes);
+ assertEquals(1, mapTest.getA());
+ assertEquals("\"NULL\"", mapTest.getMap1Map().get("a"));
+ MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("b");
+ assertEquals(MapTest.InnerMessageTest.getDefaultInstance(), innerMessageTest);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufTestHelper.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufTestHelper.java
new file mode 100644
index 0000000000000..daaa8d68fcc02
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/ProtobufTestHelper.java
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema;
+import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema;
+import org.apache.flink.formats.protobuf.util.PbFormatUtils;
+import org.apache.flink.formats.protobuf.util.PbToRowTypeUtil;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+/** Test helper class for protobuf format. */
+public class ProtobufTestHelper {
+ public static RowData validateRow(RowData rowData, RowType rowType) throws Exception {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+ StreamTableEnvironment tableEnv =
+ StreamTableEnvironment.create(
+ env, EnvironmentSettings.newInstance().inStreamingMode().build());
+
+ DataType rowDataType = fromLogicalToDataType(rowType);
+ Row row =
+ (Row) DataFormatConverters.getConverterForDataType(rowDataType).toExternal(rowData);
+ TypeInformation rowTypeInfo =
+ (TypeInformation) TypeConversions.fromDataTypeToLegacyInfo(rowDataType);
+ DataStream rows = env.fromCollection(Collections.singletonList(row), rowTypeInfo);
+
+ Table table = tableEnv.fromDataStream(rows);
+ tableEnv.createTemporaryView("t", table);
+ table = tableEnv.sqlQuery("select * from t");
+ List resultRows =
+ tableEnv.toAppendStream(table, InternalTypeInfo.of(rowType)).executeAndCollect(1);
+ return resultRows.get(0);
+ }
+
+ public static byte[] rowToPbBytes(RowData row, Class messageClass) throws Exception {
+ return rowToPbBytes(row, messageClass, false);
+ }
+
+ public static byte[] rowToPbBytes(RowData row, Class messageClass, boolean enumAsInt)
+ throws Exception {
+ return rowToPbBytes(
+ row,
+ messageClass,
+ new PbFormatConfig(messageClass.getName(), false, false, ""),
+ enumAsInt);
+ }
+
+ public static byte[] rowToPbBytes(
+ RowData row, Class messageClass, PbFormatConfig formatConfig, boolean enumAsInt)
+ throws Exception {
+ RowType rowType =
+ PbToRowTypeUtil.generateRowType(
+ PbFormatUtils.getDescriptor(messageClass.getName()), enumAsInt);
+ row = validateRow(row, rowType);
+ PbRowDataSerializationSchema serializationSchema =
+ new PbRowDataSerializationSchema(rowType, formatConfig);
+ serializationSchema.open(null);
+ byte[] bytes = serializationSchema.serialize(row);
+ return bytes;
+ }
+
+ public static RowData pbBytesToRow(Class messageClass, byte[] bytes) throws Exception {
+ return pbBytesToRow(messageClass, bytes, false);
+ }
+
+ public static RowData pbBytesToRow(Class messageClass, byte[] bytes, boolean enumAsInt)
+ throws Exception {
+ return pbBytesToRow(
+ messageClass,
+ bytes,
+ new PbFormatConfig(messageClass.getName(), false, false, ""),
+ enumAsInt);
+ }
+
+ public static RowData pbBytesToRow(
+ Class messageClass, byte[] bytes, PbFormatConfig formatConfig, boolean enumAsInt)
+ throws Exception {
+ RowType rowType =
+ PbToRowTypeUtil.generateRowType(
+ PbFormatUtils.getDescriptor(messageClass.getName()), enumAsInt);
+ PbRowDataDeserializationSchema deserializationSchema =
+ new PbRowDataDeserializationSchema(
+ rowType, InternalTypeInfo.of(rowType), formatConfig);
+ deserializationSchema.open(null);
+ RowData row = deserializationSchema.deserialize(bytes);
+ return ProtobufTestHelper.validateRow(row, rowType);
+ }
+
+ public static Map mapOf(Object... keyValues) {
+ Map map = new HashMap<>();
+
+ for (int index = 0; index < keyValues.length / 2; index++) {
+ map.put((K) keyValues[index * 2], (V) keyValues[index * 2 + 1]);
+ }
+
+ return map;
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageProtoToRowTest.java
new file mode 100644
index 0000000000000..0c2ab740bf4fe
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageProtoToRowTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.RepeatedMessageTest;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of proto repeated message data to flink internal data. */
+public class RepeatedMessageProtoToRowTest {
+ @Test
+ public void testRepeatedMessage() throws Exception {
+ RepeatedMessageTest.InnerMessageTest innerMessageTest =
+ RepeatedMessageTest.InnerMessageTest.newBuilder().setA(1).setB(2L).build();
+
+ RepeatedMessageTest.InnerMessageTest innerMessageTest1 =
+ RepeatedMessageTest.InnerMessageTest.newBuilder().setA(3).setB(4L).build();
+
+ RepeatedMessageTest repeatedMessageTest =
+ RepeatedMessageTest.newBuilder()
+ .addD(innerMessageTest)
+ .addD(innerMessageTest1)
+ .build();
+
+ RowData row =
+ ProtobufTestHelper.pbBytesToRow(
+ RepeatedMessageTest.class, repeatedMessageTest.toByteArray());
+
+ ArrayData objs = row.getArray(0);
+ RowData subRow = objs.getRow(0, 2);
+ assertEquals(1, subRow.getInt(0));
+ assertEquals(2L, subRow.getLong(1));
+ subRow = objs.getRow(1, 2);
+ assertEquals(3, subRow.getInt(0));
+ assertEquals(4L, subRow.getLong(1));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageRowToProtoTest.java
new file mode 100644
index 0000000000000..af18a014591d3
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedMessageRowToProtoTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.RepeatedMessageTest;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of flink internal array of row to proto data. */
+public class RepeatedMessageRowToProtoTest {
+ @Test
+ public void testRepeatedMessage() throws Exception {
+ RowData subRow = GenericRowData.of(1, 2L);
+ RowData subRow2 = GenericRowData.of(3, 4L);
+ ArrayData tmp = new GenericArrayData(new Object[] {subRow, subRow2});
+ RowData row = GenericRowData.of(tmp);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, RepeatedMessageTest.class);
+ RepeatedMessageTest repeatedMessageTest = RepeatedMessageTest.parseFrom(bytes);
+
+ assertEquals(2, repeatedMessageTest.getDCount());
+
+ assertEquals(1, repeatedMessageTest.getD(0).getA());
+ assertEquals(2L, repeatedMessageTest.getD(0).getB());
+ assertEquals(3, repeatedMessageTest.getD(1).getA());
+ assertEquals(4L, repeatedMessageTest.getD(1).getB());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedProtoToRowTest.java
new file mode 100644
index 0000000000000..8d541db95969a
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedProtoToRowTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.RepeatedTest;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of proto list of primitive data to flink internal data. */
+public class RepeatedProtoToRowTest {
+ @Test
+ public void testRepeated() throws Exception {
+ RepeatedTest simple = RepeatedTest.newBuilder().setA(1).addB(1).addB(2).build();
+ RowData row = ProtobufTestHelper.pbBytesToRow(RepeatedTest.class, simple.toByteArray());
+
+ assertEquals(6, row.getArity());
+ assertEquals(1, row.getInt(0));
+ ArrayData arr = row.getArray(1);
+ assertEquals(2, arr.size());
+ assertEquals(1L, arr.getLong(0));
+ assertEquals(2L, arr.getLong(1));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedRowToProtoTest.java
new file mode 100644
index 0000000000000..4dd0cd9b97fb4
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/RepeatedRowToProtoTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.RepeatedTest;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test conversion of flink internal array of primitive data to proto data. */
+public class RepeatedRowToProtoTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ 1,
+ new GenericArrayData(new Object[] {1L, 2L, 3L}),
+ false,
+ 0.1f,
+ 0.01,
+ StringData.fromString("hello"));
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, RepeatedTest.class);
+ RepeatedTest repeatedTest = RepeatedTest.parseFrom(bytes);
+ assertEquals(3, repeatedTest.getBCount());
+ assertEquals(1L, repeatedTest.getB(0));
+ assertEquals(2L, repeatedTest.getB(1));
+ assertEquals(3L, repeatedTest.getB(2));
+ }
+
+ @Test
+ public void testEmptyArray() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ 1,
+ new GenericArrayData(new Object[] {}),
+ false,
+ 0.1f,
+ 0.01,
+ StringData.fromString("hello"));
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, RepeatedTest.class);
+ RepeatedTest repeatedTest = RepeatedTest.parseFrom(bytes);
+ assertEquals(0, repeatedTest.getBCount());
+ }
+
+ @Test
+ public void testNull() throws Exception {
+ RowData row = GenericRowData.of(1, null, false, 0.1f, 0.01, StringData.fromString("hello"));
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, RepeatedTest.class);
+ RepeatedTest repeatedTest = RepeatedTest.parseFrom(bytes);
+ assertEquals(0, repeatedTest.getBCount());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleProtoToRowTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleProtoToRowTest.java
new file mode 100644
index 0000000000000..f7cfa33110950
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleProtoToRowTest.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;
+
+import org.apache.flink.formats.protobuf.testproto.SimpleTest;
+import org.apache.flink.table.data.RowData;
+
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Test conversion of proto primitive data to flink internal data. */
+public class SimpleProtoToRowTest {
+ @Test
+ public void testSimple() throws Exception {
+ SimpleTest simple =
+ SimpleTest.newBuilder()
+ .setA(1)
+ .setB(2L)
+ .setC(false)
+ .setD(0.1f)
+ .setE(0.01)
+ .setF("haha")
+ .setG(ByteString.copyFrom(new byte[] {1}))
+ .setH(SimpleTest.Corpus.IMAGES)
+ .setFAbc7D(1) // test fieldNameToJsonName
+ .setVpr6S(2)
+ .build();
+
+ RowData row = ProtobufTestHelper.pbBytesToRow(SimpleTest.class, simple.toByteArray());
+
+ assertEquals(10, row.getArity());
+ assertEquals(1, row.getInt(0));
+ assertEquals(2L, row.getLong(1));
+ assertFalse(row.getBoolean(2));
+ assertEquals(Float.valueOf(0.1f), Float.valueOf(row.getFloat(3)));
+ assertEquals(Double.valueOf(0.01d), Double.valueOf(row.getDouble(4)));
+ assertEquals("haha", row.getString(5).toString());
+ assertEquals(1, (row.getBinary(6))[0]);
+ assertEquals("IMAGES", row.getString(7).toString());
+ assertEquals(1, row.getInt(8));
+ assertEquals(2, row.getInt(9));
+ }
+
+ @Test
+ public void testNotExistsValueIgnoringDefault() throws Exception {
+ SimpleTest simple =
+ SimpleTest.newBuilder()
+ .setB(2L)
+ .setC(false)
+ .setD(0.1f)
+ .setE(0.01)
+ .setF("haha")
+ .build();
+
+ RowData row = ProtobufTestHelper.pbBytesToRow(SimpleTest.class, simple.toByteArray());
+
+ assertTrue(row.isNullAt(0));
+ assertFalse(row.isNullAt(1));
+ }
+
+ @Test
+ public void testDefaultValues() throws Exception {
+ SimpleTest simple = SimpleTest.newBuilder().build();
+
+ RowData row =
+ ProtobufTestHelper.pbBytesToRow(
+ SimpleTest.class,
+ simple.toByteArray(),
+ new PbFormatConfig(SimpleTest.class.getName(), false, true, ""),
+ false);
+
+ assertFalse(row.isNullAt(0));
+ assertFalse(row.isNullAt(1));
+ assertFalse(row.isNullAt(2));
+ assertFalse(row.isNullAt(3));
+ assertFalse(row.isNullAt(4));
+ assertFalse(row.isNullAt(5));
+ assertFalse(row.isNullAt(6));
+ assertFalse(row.isNullAt(7));
+ assertEquals(10, row.getInt(0));
+ assertEquals(100L, row.getLong(1));
+ assertFalse(row.getBoolean(2));
+ assertEquals(0.0f, row.getFloat(3), 0.0001);
+ assertEquals(0.0d, row.getDouble(4), 0.0001);
+ assertEquals("f", row.getString(5).toString());
+ assertArrayEquals(ByteString.EMPTY.toByteArray(), row.getBinary(6));
+ assertEquals(SimpleTest.Corpus.UNIVERSAL.toString(), row.getString(7).toString());
+ }
+
+ @Test
+ public void testIntEnum() throws Exception {
+ SimpleTest simple = SimpleTest.newBuilder().setH(SimpleTest.Corpus.IMAGES).build();
+ RowData row = ProtobufTestHelper.pbBytesToRow(SimpleTest.class, simple.toByteArray(), true);
+ assertEquals(2, row.getInt(7));
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleRowToProtoTest.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleRowToProtoTest.java
new file mode 100644
index 0000000000000..eccee9308926d
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/SimpleRowToProtoTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.flink.formats.protobuf.testproto.SimpleTest;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Test conversion of flink internal primitive data to proto data. */
+public class SimpleRowToProtoTest {
+ @Test
+ public void testSimple() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ 1,
+ 2L,
+ false,
+ 0.1f,
+ 0.01,
+ StringData.fromString("hello"),
+ new byte[] {1},
+ StringData.fromString("IMAGES"),
+ 1,
+ 2);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, SimpleTest.class);
+ SimpleTest simpleTest = SimpleTest.parseFrom(bytes);
+ assertTrue(simpleTest.hasA());
+ assertEquals(1, simpleTest.getA());
+ assertEquals(2L, simpleTest.getB());
+ assertFalse(simpleTest.getC());
+ assertEquals(Float.valueOf(0.1f), Float.valueOf(simpleTest.getD()));
+ assertEquals(Double.valueOf(0.01d), Double.valueOf(simpleTest.getE()));
+ assertEquals("hello", simpleTest.getF());
+ assertEquals(1, simpleTest.getG().byteAt(0));
+ assertEquals(SimpleTest.Corpus.IMAGES, simpleTest.getH());
+ assertEquals(1, simpleTest.getFAbc7D());
+ }
+
+ @Test
+ public void testNull() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ null,
+ 2L,
+ false,
+ 0.1f,
+ 0.01,
+ StringData.fromString("hello"),
+ null,
+ null,
+ 1,
+ 2);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, SimpleTest.class);
+ SimpleTest simpleTest = SimpleTest.parseFrom(bytes);
+ assertFalse(simpleTest.hasA());
+ assertFalse(simpleTest.hasG());
+ assertFalse(simpleTest.hasH());
+ }
+
+ @Test
+ public void testEnumAsInt() throws Exception {
+ RowData row =
+ GenericRowData.of(
+ null, null, null, null, null, null, null, 2, // CORPUS: IMAGE
+ null, null);
+
+ byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, SimpleTest.class, true);
+ SimpleTest simpleTest = SimpleTest.parseFrom(bytes);
+ assertEquals(SimpleTest.Corpus.IMAGES, simpleTest.getH());
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java
new file mode 100644
index 0000000000000..728a4848e4908
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java
@@ -0,0 +1,45 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.table.data.RowData;
+
+/** Sink Function for protobuf table factory test. Must run in single parallelism. */
+public class TestProtobufSinkFunction extends RichSinkFunction {
+ private final SerializationSchema serializer;
+
+ public TestProtobufSinkFunction(SerializationSchema serializer) {
+ this.serializer = serializer;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ this.serializer.open(null);
+ }
+
+ @Override
+ public void invoke(RowData value, Context context) throws Exception {
+ byte[] bytes = serializer.serialize(value);
+ TestProtobufTestStore.sinkResults.add(bytes);
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java
new file mode 100644
index 0000000000000..e084a5f902ab5
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java
@@ -0,0 +1,52 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.table.data.RowData;
+
+/** Source Function for protobuf table factory test. */
+public class TestProtobufSourceFunction extends RichSourceFunction {
+ private final DeserializationSchema deserializer;
+
+ public TestProtobufSourceFunction(DeserializationSchema deserializer) {
+ this.deserializer = deserializer;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ this.deserializer.open(null);
+ }
+
+ @Override
+ public void run(SourceContext ctx) throws Exception {
+ for (byte[] bytes : TestProtobufTestStore.sourcePbInputs) {
+ RowData rowData = deserializer.deserialize(bytes);
+ if (rowData != null) {
+ ctx.collect(rowData);
+ }
+ }
+ }
+
+ @Override
+ public void cancel() {}
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableFactory.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableFactory.java
new file mode 100644
index 0000000000000..cc6ac82c35c0d
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableFactory.java
@@ -0,0 +1,84 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/** Test protobuf table factory. Only used in flink ptotobuf test module. */
+public class TestProtobufTableFactory
+ implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+ @Override
+ public DynamicTableSource createDynamicTableSource(Context context) {
+ FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+ // discover a suitable decoding format
+ final DecodingFormat> decodingFormat =
+ helper.discoverDecodingFormat(
+ DeserializationFormatFactory.class, FactoryUtil.FORMAT);
+
+ helper.validate();
+ final DataType producedDataType =
+ context.getCatalogTable().getSchema().toPhysicalRowDataType();
+ return new TestProtobufTableSource(decodingFormat, producedDataType);
+ }
+
+ @Override
+ public DynamicTableSink createDynamicTableSink(Context context) {
+ FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+ // discover a suitable decoding format
+ final EncodingFormat> decodingFormat =
+ helper.discoverEncodingFormat(SerializationFormatFactory.class, FactoryUtil.FORMAT);
+ final DataType dataType = context.getCatalogTable().getSchema().toPhysicalRowDataType();
+ return new TestProtobufTableSink(decodingFormat, dataType);
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return "protobuf-test-connector";
+ }
+
+ @Override
+ public Set> requiredOptions() {
+ Set> options = new HashSet<>();
+ options.add(FactoryUtil.FORMAT);
+ return options;
+ }
+
+ @Override
+ public Set> optionalOptions() {
+ Set> options = new HashSet<>();
+ return options;
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSink.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSink.java
new file mode 100644
index 0000000000000..54942db36a142
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSink.java
@@ -0,0 +1,61 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+
+/** Table sink for protobuf table factory test. */
+public class TestProtobufTableSink implements DynamicTableSink {
+ private final EncodingFormat> encodingFormat;
+ private final DataType dataType;
+
+ public TestProtobufTableSink(
+ EncodingFormat> encodingFormat, DataType dataType) {
+ this.encodingFormat = encodingFormat;
+ this.dataType = dataType;
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+ return ChangelogMode.insertOnly();
+ }
+
+ @Override
+ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+ final SerializationSchema serializer =
+ encodingFormat.createRuntimeEncoder(context, dataType);
+ return SinkFunctionProvider.of(new TestProtobufSinkFunction(serializer));
+ }
+
+ @Override
+ public DynamicTableSink copy() {
+ return new TestProtobufTableSink(encodingFormat, dataType);
+ }
+
+ @Override
+ public String asSummaryString() {
+ return TestProtobufTableSink.class.getName();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSource.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSource.java
new file mode 100644
index 0000000000000..1a706e02e23e3
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTableSource.java
@@ -0,0 +1,64 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+
+/** Table source for protobuf table factory test. */
+public class TestProtobufTableSource implements ScanTableSource {
+ private final DecodingFormat> decodingFormat;
+ private final DataType producedDataType;
+
+ public TestProtobufTableSource(
+ DecodingFormat> decodingFormat,
+ DataType producedDataType) {
+ this.decodingFormat = decodingFormat;
+ this.producedDataType = producedDataType;
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.insertOnly();
+ }
+
+ @Override
+ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
+ final DeserializationSchema deserializer =
+ decodingFormat.createRuntimeDecoder(runtimeProviderContext, producedDataType);
+
+ return SourceFunctionProvider.of(new TestProtobufSourceFunction(deserializer), true);
+ }
+
+ @Override
+ public DynamicTableSource copy() {
+ return new TestProtobufTableSource(this.decodingFormat, this.producedDataType);
+ }
+
+ @Override
+ public String asSummaryString() {
+ return TestProtobufTableSource.class.getName();
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTestStore.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTestStore.java
new file mode 100644
index 0000000000000..c79b274ba596a
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufTestStore.java
@@ -0,0 +1,28 @@
+/*
+ * 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.table;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** The test data store for protobuf SQL integration test only. */
+public class TestProtobufTestStore {
+ public static List sourcePbInputs = new ArrayList<>();
+ public static List sinkResults = new ArrayList<>();
+}
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_map.proto b/flink-formats/flink-protobuf/src/test/proto/test_map.proto
new file mode 100644
index 0000000000000..e22f771bfe317
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_map.proto
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message MapTest {
+ optional int32 a = 1;
+ map map1 = 2;
+ map map2 = 3;
+ map map3 = 4;
+
+
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_multiple_level_message.proto b/flink-formats/flink-protobuf/src/test/proto/test_multiple_level_message.proto
new file mode 100644
index 0000000000000..ba0c320d35f39
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_multiple_level_message.proto
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message MultipleLevelMessageTest {
+ optional int32 a = 1;
+ optional int64 b = 2;
+ optional bool c = 3;
+ optional InnerMessageTest1 d = 4;
+
+ message InnerMessageTest1{
+ optional InnerMessageTest2 a = 1;
+ optional bool c = 2;
+ message InnerMessageTest2{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+ }
+}
+
+
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_null.proto b/flink-formats/flink-protobuf/src/test/proto/test_null.proto
new file mode 100644
index 0000000000000..b17b15336b402
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_null.proto
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message NullTest {
+ map string_map = 1;
+ map int_map = 2;
+ map long_map = 3;
+ map boolean_map = 4;
+ map float_map = 5;
+ map double_map = 6;
+ map enum_map = 7;
+ map message_map = 8;
+ map bytes_map=9;
+
+ repeated string string_array = 10;
+ repeated int32 int_array = 11;
+ repeated int64 long_array = 12;
+ repeated bool boolean_array = 13;
+ repeated float float_array = 14;
+ repeated double double_array = 15;
+ repeated Corpus enum_array = 16;
+ repeated InnerMessageTest message_array = 17;
+ repeated bytes bytes_array = 18;
+
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+
+ enum Corpus {
+ UNIVERSAL = 0;
+ WEB = 1;
+ IMAGES = 2;
+ LOCAL = 3;
+ NEWS = 4;
+ PRODUCTS = 5;
+ VIDEO = 7;
+ }
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_oneof.proto b/flink-formats/flink-protobuf/src/test/proto/test_oneof.proto
new file mode 100644
index 0000000000000..814761b2cec97
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_oneof.proto
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message OneofTest {
+ oneof test_oneof{
+ int32 a = 1;
+ int32 b = 2;
+ }
+}
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_pb3.proto b/flink-formats/flink-protobuf/src/test/proto/test_pb3.proto
new file mode 100644
index 0000000000000..b21ca840f65bb
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_pb3.proto
@@ -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.
+ */
+
+syntax = "proto3";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message Pb3Test {
+ int32 a = 1;
+ int64 b = 2;
+ string c = 3;
+ float d = 4;
+ double e = 5;
+ Corpus f = 6;
+ InnerMessageTest g = 7;
+ repeated InnerMessageTest h = 8;
+ bytes i = 9;
+ map map1 = 10;
+ map map2 = 11;
+
+ message InnerMessageTest{
+ int32 a =1;
+ int64 b =2;
+ }
+
+ enum Corpus {
+ UNIVERSAL = 0;
+ WEB = 1;
+ IMAGES = 2;
+ LOCAL = 3;
+ NEWS = 4;
+ PRODUCTS = 5;
+ VIDEO = 7;
+ }
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_repeated.proto b/flink-formats/flink-protobuf/src/test/proto/test_repeated.proto
new file mode 100644
index 0000000000000..90cb9a58bed39
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_repeated.proto
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message RepeatedTest {
+ optional int32 a = 1;
+ repeated int64 b = 2;
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6;
+}
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_repeated_message.proto b/flink-formats/flink-protobuf/src/test/proto/test_repeated_message.proto
new file mode 100644
index 0000000000000..2a73a9a111b51
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_repeated_message.proto
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message RepeatedMessageTest {
+ repeated InnerMessageTest d = 4;
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_simple.proto b/flink-formats/flink-protobuf/src/test/proto/test_simple.proto
new file mode 100644
index 0000000000000..e5d01f740c754
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_simple.proto
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_multiple_files = true;
+
+message SimpleTest {
+ optional int32 a = 1 [default=10];
+ optional int64 b = 2 [default=100];
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6 [default="f"];;
+ optional bytes g = 7;
+ optional Corpus h = 8;
+ //this is must because protobuf have some field name parse bug if number is after "_".
+ optional int32 f_abc_7d = 9;
+ optional int32 vpr6s = 10;
+
+ enum Corpus {
+ UNIVERSAL = 0;
+ WEB = 1;
+ IMAGES = 2;
+ LOCAL = 3;
+ NEWS = 4;
+ PRODUCTS = 5;
+ VIDEO = 7;
+ }
+
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_simple_no_java_package.proto b/flink-formats/flink-protobuf/src/test/proto/test_simple_no_java_package.proto
new file mode 100644
index 0000000000000..334ec0d68e487
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_simple_no_java_package.proto
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_multiple_files = true;
+
+message SimpleTestNoJavaPackage {
+ optional int32 a = 1 [default=10];
+ optional int64 b = 2 [default=100];
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6 [default="f"];
+ optional bytes g = 7;
+ optional Corpus h = 8;
+ //this is must because protobuf have some field name parse bug if number is after "_".
+ optional int32 f_abc_7d = 9;
+
+ enum Corpus {
+ UNIVERSAL = 0;
+ WEB = 1;
+ IMAGES = 2;
+ LOCAL = 3;
+ NEWS = 4;
+ PRODUCTS = 5;
+ VIDEO = 7;
+ }
+}
+
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_simple_noouter_nomulti.proto b/flink-formats/flink-protobuf/src/test/proto/test_simple_noouter_nomulti.proto
new file mode 100644
index 0000000000000..379732d55033c
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_simple_noouter_nomulti.proto
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+
+message SimpleTestNoouterNomulti {
+ optional int32 a = 1;
+ optional int64 b = 2;
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6;
+ optional bytes g = 7;
+ optional InnerMessageTest h = 8;
+
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+}
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_multi.proto b/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_multi.proto
new file mode 100644
index 0000000000000..ca024b862d6c4
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_multi.proto
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_outer_classname = "SimpleTestOuterMultiProto";
+option java_multiple_files = true;
+
+message SimpleTestOuterMulti {
+ optional int32 a = 1;
+ optional int64 b = 2;
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6;
+ optional bytes g = 7;
+ optional InnerMessageTest h = 8;
+
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+}
+
diff --git a/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_nomulti.proto b/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_nomulti.proto
new file mode 100644
index 0000000000000..16459d27250e3
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/proto/test_simple_outer_nomulti.proto
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+syntax = "proto2";
+package org.apache.flink.formats.protobuf.proto;
+option java_package = "org.apache.flink.formats.protobuf.testproto";
+option java_outer_classname = "SimpleTestOuterNomultiProto";
+
+message SimpleTestOuterNomulti {
+ optional int32 a = 1;
+ optional int64 b = 2;
+ optional bool c = 3;
+ optional float d = 4;
+ optional double e = 5;
+ optional string f = 6;
+ optional bytes g = 7;
+ optional InnerMessageTest h = 8;
+
+ message InnerMessageTest{
+ optional int32 a =1;
+ optional int64 b =2;
+ }
+}
+
diff --git a/flink-formats/flink-protobuf/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-protobuf/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 0000000000000..bf058e02d2202
--- /dev/null
+++ b/flink-formats/flink-protobuf/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,15 @@
+# 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.
+org.apache.flink.formats.protobuf.table.TestProtobufTableFactory
diff --git a/flink-formats/flink-sql-protobuf/pom.xml b/flink-formats/flink-sql-protobuf/pom.xml
new file mode 100644
index 0000000000000..8fe8ab718dc3b
--- /dev/null
+++ b/flink-formats/flink-sql-protobuf/pom.xml
@@ -0,0 +1,69 @@
+
+
+
+
+ 4.0.0
+
+
+ org.apache.flink
+ flink-formats
+ 1.16-SNAPSHOT
+
+
+ flink-sql-protobuf
+ Flink : Formats : SQL Protobuf
+
+ jar
+
+
+
+ org.apache.flink
+ flink-protobuf
+ ${project.version}
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ shade-flink
+ package
+
+ shade
+
+
+
+
+ org.apache.flink:flink-protobuf
+ com.google.protobuf:protobuf-java
+
+
+
+
+
+
+
+
+
diff --git a/flink-formats/flink-sql-protobuf/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-protobuf/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000000000..8fd8106a6c2d5
--- /dev/null
+++ b/flink-formats/flink-sql-protobuf/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,10 @@
+flink-sql-protobuf
+Copyright 2014-2022 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+This project bundles the following dependencies under BSD-3 License (https://opensource.org/licenses/BSD-3-Clause).
+See bundled license files for details.
+
+- com.google.protobuf:protobuf-java:3.21.2
diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml
index a288b40331afb..74300b5106e60 100644
--- a/flink-formats/pom.xml
+++ b/flink-formats/pom.xml
@@ -46,6 +46,7 @@ under the License.
flink-sequence-file
flink-compress
flink-csv
+ flink-protobuf
flink-orc
flink-orc-nohive
flink-hadoop-bulk
@@ -88,6 +89,7 @@ under the License.
flink-sql-parquet
flink-sql-avro
flink-sql-avro-confluent-registry
+ flink-sql-protobuf