caozhen1937 commented on a change in pull request #13296: URL: https://github.com/apache/flink/pull/13296#discussion_r496341421
########## File path: flink-formats/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java ########## @@ -0,0 +1,208 @@ +/* + * 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.avro.registry.confluent.debezium; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Deserialization schema from Debezium Avro to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Debezium's schema definition and can extract the database data + * and convert into {@link RowData} with {@link RowKind}. + * <p> + * <p>Deserializes a <code>byte[]</code> message as a JSON object and reads + * the specified fields. + * <p> + * <p>Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see <a href="https://debezium.io/">Debezium</a> + */ +@Internal +public final class DebeziumAvroDeserializationSchema implements DeserializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + /** + * snapshot read + */ + private static final String OP_READ = "r"; + /** + * insert operation + */ + private static final String OP_CREATE = "c"; + /** + * update operation + */ + private static final String OP_UPDATE = "u"; + /** + * delete operation + */ + private static final String OP_DELETE = "d"; + + private static final String REPLICA_IDENTITY_EXCEPTION = "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** + * The deserializer to deserialize Debezium Avro data. + */ + private final AvroRowDataDeserializationSchema avroDeserializer; + + /** + * TypeInformation of the produced {@link RowData}. + **/ + private final TypeInformation<RowData> resultTypeInfo; + + /** + * Debezium Avro data rowType + */ + private final RowType rowType; + + + public DebeziumAvroDeserializationSchema( + RowType rowType, + TypeInformation<RowData> resultTypeInfo, + String schemaRegistryUrl) { + this.resultTypeInfo = resultTypeInfo; + this.rowType = rowType; + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroDeserializer = new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), + resultTypeInfo); + } + + public DebeziumAvroDeserializationSchema( + RowType rowType, + TypeInformation<RowData> resultTypeInfo, + AvroRowDataDeserializationSchema avroDeserializer) { Review comment: This constructor is used for unit testing because schemaRegistryUrl cannot be used to retrieve the schema. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
