wuchong commented on a change in pull request #13296: URL: https://github.com/apache/flink/pull/13296#discussion_r490841725
########## 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 Review comment: Add `.` at the end. Please check the checkstyle agian. The compile build is failed. ########## 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> Review comment: Remove. ########## 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) { + this.rowType = rowType; + this.resultTypeInfo = resultTypeInfo; + this.avroDeserializer = avroDeserializer; + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector<RowData>) instead."); + } + + @Override + public void deserialize(byte[] message, Collector<RowData> out) throws IOException { + + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) avroDeserializer.deserialize(message); + + GenericRowData before = null; + int index = rowType.getFieldIndex("before"); Review comment: We should also know the index of before and after. Shouldn't get the index dynamically. See the implementation of debezium-json. ########## File path: flink-formats/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java ########## @@ -0,0 +1,141 @@ +/* + * 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.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +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.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; + +/** + * Format factory for providing configured instances of Debezium Avro to RowData {@link DeserializationSchema}. + */ +public class DebeziumAvroFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium-avro-confluent"; + + @SuppressWarnings("unchecked") + @Override + public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat( + DynamicTableFactory.Context context, Review comment: Add one more indent for method arguments. Please check other methods. ########## File path: flink-formats/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java ########## @@ -0,0 +1,129 @@ +/* + * 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.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; +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.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium Avro. + */ +public class DebeziumAvroSerializationSchema implements SerializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + /** + * insert operation + */ + private static final StringData OP_CREATE = StringData.fromString("c"); + /** + * delete operation + */ + private static final StringData OP_DELETE = StringData.fromString("d"); + + /** + * The deserializer to deserialize Debezium Avro data. + */ + private final AvroRowDataSerializationSchema avroSerializer; + + private transient GenericRowData reuse = new GenericRowData(2); + + public DebeziumAvroSerializationSchema( + RowType rowType, + String schemaRegistryUrl, + String schemaRegistrySubject) { + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroSerializer = new AvroRowDataSerializationSchema( + debeziumAvroRowType, + ConfluentRegistryAvroSerializationSchema.forGeneric( + schemaRegistrySubject, + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + RowDataToAvroConverters.createRowConverter(debeziumAvroRowType)); + } + + public DebeziumAvroSerializationSchema(AvroRowDataSerializationSchema avroSerializer) { + this.avroSerializer = avroSerializer; + } + + @Override + public void open(InitializationContext context) throws Exception { + } + + @Override + public byte[] serialize(RowData element) { + reuse.setField(0, element); + reuse.setField(1, rowKind2String(element.getRowKind())); + return avroSerializer.serialize(reuse); + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_CREATE; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumAvroSerializationSchema that = (DebeziumAvroSerializationSchema) o; + return Objects.equals(avroSerializer, that.avroSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(avroSerializer); + } + + public static RowType createDebeziumAvroRowType(DataType dataType) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + // and we don't need "before" , because can not support UPDATE_BEFORE,UPDATE_AFTER Review comment: We need "before", because the DELETE should encode data into "before". ########## File path: flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java ########## @@ -0,0 +1,138 @@ +/* + * 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.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +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.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static junit.framework.TestCase.assertEquals; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; + +/** + * Tests for {@link DebeziumAvroFormatFactory}. + */ +public class DebeziumAvroFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final String SUBJECT = "test-debezium-avro"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Test + public void testSeDeSchema() { + final Map<String, String> options = getAllOptions(); + String schemaRegistryUrl = options.get(DebeziumAvroFormatFactory.IDENTIFIER + "." + RegistryAvroOptions.SCHEMA_REGISTRY_URL.key()); + String schemaRegistrySubject = options.get(DebeziumAvroFormatFactory.IDENTIFIER + "." + RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT.key()); + + final DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + schemaRegistryUrl); Review comment: Use `REGISTRY_URL` instead of extract the value from options. This is used for asserting which should be a constant value. ########## File path: flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java ########## @@ -0,0 +1,138 @@ +/* + * 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.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +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.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static junit.framework.TestCase.assertEquals; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; + +/** + * Tests for {@link DebeziumAvroFormatFactory}. + */ +public class DebeziumAvroFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final String SUBJECT = "test-debezium-avro"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Test + public void testSeDeSchema() { + final Map<String, String> options = getAllOptions(); + String schemaRegistryUrl = options.get(DebeziumAvroFormatFactory.IDENTIFIER + "." + RegistryAvroOptions.SCHEMA_REGISTRY_URL.key()); + String schemaRegistrySubject = options.get(DebeziumAvroFormatFactory.IDENTIFIER + "." + RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT.key()); + + final DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + schemaRegistryUrl); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema<RowData> actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + + + final DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( + ROW_TYPE, + schemaRegistryUrl, + schemaRegistrySubject Review comment: Use `SUBJECT` instead. ########## 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: Why we need such a constructor? The `avroDeserializer` should be created by the debezium deserialization schema, otherwise it's error-prone. ---------------------------------------------------------------- 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]
