diff --git a/flink-formats/flink-protobuf/pom.xml b/flink-formats/flink-protobuf/pom.xml new file mode 100644 index 0000000000000..101a022f86064 --- /dev/null +++ b/flink-formats/flink-protobuf/pom.xml @@ -0,0 +1,146 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-formats + 1.16-SNAPSHOT + + + flink-protobuf + Flink : Formats : Protobuf + + jar + + + + 3.0.11 + + + + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + com.google.protobuf + protobuf-java + ${protoc.version} + + + + org.codehaus.janino + janino + + ${janino.version} + provided + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + test-jar + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + + + + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 + + + generate-sources + + run + + + ${protoc.version} + + src/test/proto + + + + java + none + target/test-proto-sources + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 3.0.0 + + + generate-sources + + add-test-source + + + + target/test-proto-sources + + + + + + + + diff --git a/flink-formats/flink-protobuf/src/main/java/com/google/protobuf/ProtobufInternalUtils.java b/flink-formats/flink-protobuf/src/main/java/com/google/protobuf/ProtobufInternalUtils.java new file mode 100644 index 0000000000000..1734ebefb0606 --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/com/google/protobuf/ProtobufInternalUtils.java @@ -0,0 +1,27 @@ +/* + * 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 com.google.protobuf; + +/** This class is to access internal method in protobuf package. */ +public class ProtobufInternalUtils { + /** convert underscore name to camel name. */ + public static String underScoreToCamelCase(String name, boolean capNext) { + return SchemaUtil.toCamelCase(name, capNext); + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbCodegenException.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbCodegenException.java new file mode 100644 index 0000000000000..772a8f24ecaca --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbCodegenException.java @@ -0,0 +1,36 @@ +/* + * 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; + +/** Exception represents codegen error in row and proto conversion which is probably a bug. */ +public class PbCodegenException extends Exception { + public PbCodegenException() {} + + public PbCodegenException(String message) { + super(message); + } + + public PbCodegenException(String message, Throwable cause) { + super(message, cause); + } + + public PbCodegenException(Throwable cause) { + super(cause); + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbConstant.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbConstant.java new file mode 100644 index 0000000000000..a886768534d2c --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbConstant.java @@ -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. + */ + +package org.apache.flink.formats.protobuf; + +/** Keeps protobuf constants separately. */ +public class PbConstant { + public static final String PB_METHOD_GET_DESCRIPTOR = "getDescriptor"; + public static final String PB_METHOD_PARSE_FROM = "parseFrom"; + public static final String GENERATED_DECODE_METHOD = "decode"; + public static final String GENERATED_ENCODE_METHOD = "encode"; + public static final String PB_MAP_KEY_NAME = "key"; + public static final String PB_MAP_VALUE_NAME = "value"; +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbDecodingFormat.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbDecodingFormat.java new file mode 100644 index 0000000000000..12e086f26353d --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbDecodingFormat.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; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema; +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.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +/** {@link DecodingFormat} for protobuf decoding. */ +public class PbDecodingFormat implements DecodingFormat> { + private final PbFormatConfig formatConfig; + + public PbDecodingFormat(PbFormatConfig formatConfig) { + this.formatConfig = formatConfig; + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return new PbRowDataDeserializationSchema(rowType, rowDataTypeInfo, formatConfig); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbEncodingFormat.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbEncodingFormat.java new file mode 100644 index 0000000000000..7744755f765db --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbEncodingFormat.java @@ -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. + */ + +package org.apache.flink.formats.protobuf; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema; +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.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +/** {@link EncodingFormat} for protobuf encoding. */ +public class PbEncodingFormat implements EncodingFormat> { + private final PbFormatConfig pbFormatConfig; + + public PbEncodingFormat(PbFormatConfig pbFormatConfig) { + this.pbFormatConfig = pbFormatConfig; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new PbRowDataSerializationSchema(rowType, pbFormatConfig); + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatConfig.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatConfig.java new file mode 100644 index 0000000000000..36dde1dd473c9 --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatConfig.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.protobuf; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.formats.protobuf.PbFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.protobuf.PbFormatOptions.READ_DEFAULT_VALUES; +import static org.apache.flink.formats.protobuf.PbFormatOptions.WRITE_NULL_STRING_LITERAL; + +/** Config of protobuf configs. */ +public class PbFormatConfig implements Serializable { + private static final long serialVersionUID = 1L; + + private final String messageClassName; + private final boolean ignoreParseErrors; + private final boolean readDefaultValues; + private final String writeNullStringLiterals; + + public PbFormatConfig( + String messageClassName, + boolean ignoreParseErrors, + boolean readDefaultValues, + String writeNullStringLiterals) { + this.messageClassName = messageClassName; + this.ignoreParseErrors = ignoreParseErrors; + this.readDefaultValues = readDefaultValues; + this.writeNullStringLiterals = writeNullStringLiterals; + } + + public String getMessageClassName() { + return messageClassName; + } + + public boolean isIgnoreParseErrors() { + return ignoreParseErrors; + } + + public boolean isReadDefaultValues() { + return readDefaultValues; + } + + public String getWriteNullStringLiterals() { + return writeNullStringLiterals; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PbFormatConfig that = (PbFormatConfig) o; + return ignoreParseErrors == that.ignoreParseErrors + && readDefaultValues == that.readDefaultValues + && Objects.equals(messageClassName, that.messageClassName) + && Objects.equals(writeNullStringLiterals, that.writeNullStringLiterals); + } + + @Override + public int hashCode() { + return Objects.hash( + messageClassName, ignoreParseErrors, readDefaultValues, writeNullStringLiterals); + } + + /** Builder of PbFormatConfig. */ + public static class PbFormatConfigBuilder { + private String messageClassName; + private boolean ignoreParseErrors = IGNORE_PARSE_ERRORS.defaultValue(); + private boolean readDefaultValues = READ_DEFAULT_VALUES.defaultValue(); + private String writeNullStringLiterals = WRITE_NULL_STRING_LITERAL.defaultValue(); + + public PbFormatConfigBuilder messageClassName(String messageClassName) { + this.messageClassName = messageClassName; + return this; + } + + public PbFormatConfigBuilder ignoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + + public PbFormatConfigBuilder readDefaultValues(boolean readDefaultValues) { + this.readDefaultValues = readDefaultValues; + return this; + } + + public PbFormatConfigBuilder writeNullStringLiterals(String writeNullStringLiterals) { + this.writeNullStringLiterals = writeNullStringLiterals; + return this; + } + + public PbFormatConfig build() { + return new PbFormatConfig( + messageClassName, + ignoreParseErrors, + readDefaultValues, + writeNullStringLiterals); + } + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatContext.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatContext.java new file mode 100644 index 0000000000000..06370f3fa6e17 --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatContext.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; + +/** store config and common information. */ +public class PbFormatContext { + private final String outerPrefix; + private final PbFormatConfig pbFormatConfig; + + public PbFormatContext(String outerPrefix, PbFormatConfig pbFormatConfig) { + this.outerPrefix = outerPrefix; + this.pbFormatConfig = pbFormatConfig; + } + + public String getOuterPrefix() { + return outerPrefix; + } + + public PbFormatConfig getPbFormatConfig() { + return pbFormatConfig; + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatFactory.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatFactory.java new file mode 100644 index 0000000000000..af6e88a81e4ab --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatFactory.java @@ -0,0 +1,94 @@ +/* + * 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.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; + +import java.util.HashSet; +import java.util.Set; + +/** + * Table format factory for providing configured instances of Protobuf to RowData {@link + * SerializationSchema} and {@link DeserializationSchema}. + */ +public class PbFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "protobuf"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + return new PbDecodingFormat(buildConfig(formatOptions)); + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + return new PbEncodingFormat(buildConfig(formatOptions)); + } + + private static PbFormatConfig buildConfig(ReadableConfig formatOptions) { + PbFormatConfig.PbFormatConfigBuilder configBuilder = + new PbFormatConfig.PbFormatConfigBuilder(); + configBuilder.messageClassName(formatOptions.get(PbFormatOptions.MESSAGE_CLASS_NAME)); + formatOptions + .getOptional(PbFormatOptions.IGNORE_PARSE_ERRORS) + .ifPresent(configBuilder::ignoreParseErrors); + formatOptions + .getOptional(PbFormatOptions.READ_DEFAULT_VALUES) + .ifPresent(configBuilder::readDefaultValues); + formatOptions + .getOptional(PbFormatOptions.WRITE_NULL_STRING_LITERAL) + .ifPresent(configBuilder::writeNullStringLiterals); + return configBuilder.build(); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + Set> result = new HashSet<>(); + result.add(PbFormatOptions.MESSAGE_CLASS_NAME); + return result; + } + + @Override + public Set> optionalOptions() { + Set> result = new HashSet<>(); + result.add(PbFormatOptions.IGNORE_PARSE_ERRORS); + result.add(PbFormatOptions.READ_DEFAULT_VALUES); + result.add(PbFormatOptions.WRITE_NULL_STRING_LITERAL); + return result; + } +} diff --git a/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatOptions.java b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatOptions.java new file mode 100644 index 0000000000000..1cf884f831422 --- /dev/null +++ b/flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatOptions.java @@ -0,0 +1,55 @@ +/* + * 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.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** This class holds configuration constants used by protobuf format. */ +public class PbFormatOptions { + public static final ConfigOption MESSAGE_CLASS_NAME = + ConfigOptions.key("message-class-name") + .stringType() + .noDefaultValue() + .withDescription( + "Required option to specify the full name of protobuf message class. The protobuf class " + + "must be located in the classpath both in client and task side"); + + public static final ConfigOption IGNORE_PARSE_ERRORS = + ConfigOptions.key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional read flag to skip rows with parse errors instead of failing; false by default."); + + public static final ConfigOption READ_DEFAULT_VALUES = + ConfigOptions.key("read-default-values") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to read as default values instead of null when some field does not exist in deserialization; default to false." + + "If proto syntax is proto3, this value will be set true forcibly because proto3's standard is to use default values."); + public static final ConfigOption 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