You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/15 08:12:32 UTC

[GitHub] [flink] danny0405 commented on a change in pull request #12152: [FLINK-17149][debezium][canal] Introduce Debezium and Canal changelog format

danny0405 commented on a change in pull request #12152:
URL: https://github.com/apache/flink/pull/12152#discussion_r425633198



##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.json.canal;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.ArrayData;
+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 Canal JSON 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>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="https://github.com/alibaba/canal">Alibaba Canal</a>
+ */
+public class CanalJsonDeserializationSchema implements DeserializationSchema<RowData> {
+	private static final long serialVersionUID = 1L;
+
+	private static final String OP_INSERT = "INSERT";
+	private static final String OP_UPDATE = "UPDATE";
+	private static final String OP_DELETE = "DELETE";
+
+	/** The deserializer to deserialize Debezium JSON data. */
+	private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+	/** TypeInformation of the produced {@link RowData}. **/
+	private final TypeInformation<RowData> resultTypeInfo;
+
+	/** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
+	private final boolean ignoreParseErrors;
+
+	/** Number of fields. */
+	private final int fieldCount;
+
+	public CanalJsonDeserializationSchema(
+			RowType rowType,
+			TypeInformation<RowData> resultTypeInfo,
+			boolean ignoreParseErrors) {
+		this.resultTypeInfo = resultTypeInfo;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.fieldCount = rowType.getFieldCount();
+		this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+			createJsonRowType(fromLogicalToDataType(rowType)),
+			// the result type is never used, so it's fine to pass in Canal's result type
+			resultTypeInfo,
+			false, // ignoreParseErrors already contains the functionality of failOnMissingField
+			ignoreParseErrors);
+
+	}
+
+	@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 {
+		RowData row = jsonDeserializer.deserialize(message);
+		String type = row.getString(2).toString(); // "type" field
+		if (OP_INSERT.equals(type)) {
+			// "data" field is an array of row, contains inserted rows
+			ArrayData data = row.getArray(0);
+			for (int i = 0; i < data.size(); i++) {
+				RowData insert = data.getRow(i, fieldCount);
+				insert.setRowKind(RowKind.INSERT);
+				out.collect(insert);
+			}
+		} else if (OP_UPDATE.equals(type)) {
+			// "data" field is an array of row, contains new rows
+			ArrayData data = row.getArray(0);
+			// "old" field is an array of row, contains old values
+			ArrayData old = row.getArray(1);
+			for (int i = 0; i < data.size(); i++) {
+				// the underlying JSON deserialization schema always produce GenericRowData.
+				GenericRowData after = (GenericRowData) data.getRow(i, fieldCount);
+				GenericRowData before = (GenericRowData) old.getRow(i, fieldCount);
+				for (int f = 0; f < fieldCount; f++) {
+					if (before.isNullAt(f)) {
+						// not null fields in "old" (before) means the fields are changed
+						// null/empty fields in "old" (before) means the fields not not changed
+						// so we just copy the not changed fields into before
+						before.setField(f, after.getField(f));
+					}
+				}
+				before.setRowKind(RowKind.UPDATE_BEFORE);
+				after.setRowKind(RowKind.UPDATE_AFTER);
+				out.collect(before);
+				out.collect(after);
+			}
+		} else if (OP_DELETE.equals(type)) {
+			// "data" field is an array of row, contains deleted rows
+			ArrayData data = row.getArray(0);
+			for (int i = 0; i < data.size(); i++) {
+				RowData insert = data.getRow(i, fieldCount);
+				insert.setRowKind(RowKind.DELETE);
+				out.collect(insert);
+			}
+		} else {
+			if (!ignoreParseErrors) {
+				throw new IOException(format("Failed to deserialize Canal JSON '%s'.", new String(message)));
+			}

Review comment:
       Give more detailed exception for an intentionally thrown exception.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.json.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.json.JsonRowDataDeserializationSchema;
+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 JSON 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>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="https://debezium.io/">Debezium</a>
+ */
+@Internal
+public final class DebeziumJsonDeserializationSchema implements DeserializationSchema<RowData> {
+	private static final long serialVersionUID = 1L;
+
+	private static final String OP_CREATE = "c";
+	private static final String OP_UPDATE = "u";
+	private static final String OP_DELETE = "d";
+
+	/** The deserializer to deserialize Debezium JSON data. */
+	private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+	/** TypeInformation of the produced {@link RowData}. **/
+	private final TypeInformation<RowData> resultTypeInfo;
+
+	/**
+	 * Flag indicating whether the Debezium JSON data contains schema part or not.
+	 * When Debezium Kafka Connect enables "value.converter.schemas.enable", the JSON
+	 * will contain "schema" information, but we just ignore "schema" and extract data
+	 * from "payload".
+	 */
+	private final boolean schemaInclude;
+
+	/** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
+	private final boolean ignoreParseErrors;
+
+	public DebeziumJsonDeserializationSchema(
+			RowType rowType,
+			TypeInformation<RowData> resultTypeInfo,
+			boolean schemaInclude,
+			boolean ignoreParseErrors) {
+		this.resultTypeInfo = resultTypeInfo;
+		this.schemaInclude = schemaInclude;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+			createJsonRowType(fromLogicalToDataType(rowType), schemaInclude),
+			// the result type is never used, so it's fine to pass in Debezium's result type
+			resultTypeInfo,
+			false, // ignoreParseErrors already contains the functionality of failOnMissingField
+			ignoreParseErrors);
+	}
+
+	@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 {
+		GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message);
+		GenericRowData payload;
+		if (schemaInclude) {
+			payload = (GenericRowData) row.getField(0);
+		} else {
+			payload = row;
+		}
+
+		GenericRowData before = (GenericRowData) payload.getField(0);
+		GenericRowData after = (GenericRowData) payload.getField(1);
+		String op = payload.getField(2).toString();
+		if (OP_CREATE.equals(op) && after != null) {
+			after.setRowKind(RowKind.INSERT);
+			out.collect(after);
+		} else if (OP_UPDATE.equals(op) && (before != null && after != null)) {
+			before.setRowKind(RowKind.UPDATE_BEFORE);
+			after.setRowKind(RowKind.UPDATE_AFTER);
+			out.collect(before);
+			out.collect(after);
+		} else if (OP_DELETE.equals(op) && before != null) {
+			before.setRowKind(RowKind.DELETE);
+			out.collect(before);
+		} else {
+			if (!ignoreParseErrors) {
+				throw new IOException(format("Failed to deserialize Debezium JSON '%s'.", new String(message)));
+			}

Review comment:
       Give more detailed message about why the deserialization fails.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumFormatFactory.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.json.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.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.ScanFormat;
+import org.apache.flink.table.connector.format.SinkFormat;
+import org.apache.flink.table.connector.source.ScanTableSource;
+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.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Format factory for providing configured instances of Debezium JSON to RowData {@link DeserializationSchema}.
+ */
+public class DebeziumFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {
+
+	public static final String IDENTIFIER = "debezium-json";

Review comment:
       `DebeziumJsonFormatFactory ` or maybe we can make the underlying format configurable.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.json.canal;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.ArrayData;
+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 Canal JSON 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>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="https://github.com/alibaba/canal">Alibaba Canal</a>
+ */
+public class CanalJsonDeserializationSchema implements DeserializationSchema<RowData> {
+	private static final long serialVersionUID = 1L;
+
+	private static final String OP_INSERT = "INSERT";
+	private static final String OP_UPDATE = "UPDATE";
+	private static final String OP_DELETE = "DELETE";
+
+	/** The deserializer to deserialize Debezium JSON data. */
+	private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+	/** TypeInformation of the produced {@link RowData}. **/
+	private final TypeInformation<RowData> resultTypeInfo;
+
+	/** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
+	private final boolean ignoreParseErrors;
+
+	/** Number of fields. */
+	private final int fieldCount;
+
+	public CanalJsonDeserializationSchema(
+			RowType rowType,
+			TypeInformation<RowData> resultTypeInfo,
+			boolean ignoreParseErrors) {
+		this.resultTypeInfo = resultTypeInfo;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.fieldCount = rowType.getFieldCount();
+		this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+			createJsonRowType(fromLogicalToDataType(rowType)),
+			// the result type is never used, so it's fine to pass in Canal's result type
+			resultTypeInfo,
+			false, // ignoreParseErrors already contains the functionality of failOnMissingField
+			ignoreParseErrors);
+
+	}
+
+	@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 {
+		RowData row = jsonDeserializer.deserialize(message);
+		String type = row.getString(2).toString(); // "type" field
+		if (OP_INSERT.equals(type)) {
+			// "data" field is an array of row, contains inserted rows

Review comment:
       Replace the if else clause with switch case seems better.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.json.canal;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.ArrayData;
+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 Canal JSON 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>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="https://github.com/alibaba/canal">Alibaba Canal</a>
+ */
+public class CanalJsonDeserializationSchema implements DeserializationSchema<RowData> {
+	private static final long serialVersionUID = 1L;
+
+	private static final String OP_INSERT = "INSERT";
+	private static final String OP_UPDATE = "UPDATE";
+	private static final String OP_DELETE = "DELETE";
+
+	/** The deserializer to deserialize Debezium JSON data. */
+	private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+	/** TypeInformation of the produced {@link RowData}. **/
+	private final TypeInformation<RowData> resultTypeInfo;
+
+	/** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
+	private final boolean ignoreParseErrors;
+
+	/** Number of fields. */
+	private final int fieldCount;
+
+	public CanalJsonDeserializationSchema(
+			RowType rowType,
+			TypeInformation<RowData> resultTypeInfo,
+			boolean ignoreParseErrors) {
+		this.resultTypeInfo = resultTypeInfo;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.fieldCount = rowType.getFieldCount();
+		this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+			createJsonRowType(fromLogicalToDataType(rowType)),
+			// the result type is never used, so it's fine to pass in Canal's result type
+			resultTypeInfo,
+			false, // ignoreParseErrors already contains the functionality of failOnMissingField
+			ignoreParseErrors);
+
+	}
+
+	@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 {
+		RowData row = jsonDeserializer.deserialize(message);
+		String type = row.getString(2).toString(); // "type" field
+		if (OP_INSERT.equals(type)) {
+			// "data" field is an array of row, contains inserted rows
+			ArrayData data = row.getArray(0);
+			for (int i = 0; i < data.size(); i++) {
+				RowData insert = data.getRow(i, fieldCount);
+				insert.setRowKind(RowKind.INSERT);
+				out.collect(insert);
+			}
+		} else if (OP_UPDATE.equals(type)) {
+			// "data" field is an array of row, contains new rows
+			ArrayData data = row.getArray(0);
+			// "old" field is an array of row, contains old values
+			ArrayData old = row.getArray(1);
+			for (int i = 0; i < data.size(); i++) {
+				// the underlying JSON deserialization schema always produce GenericRowData.
+				GenericRowData after = (GenericRowData) data.getRow(i, fieldCount);
+				GenericRowData before = (GenericRowData) old.getRow(i, fieldCount);
+				for (int f = 0; f < fieldCount; f++) {
+					if (before.isNullAt(f)) {
+						// not null fields in "old" (before) means the fields are changed
+						// null/empty fields in "old" (before) means the fields not not changed
+						// so we just copy the not changed fields into before

Review comment:
       `not not changed` => `are not changed`

##########
File path: flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.json.canal;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.table.api.DataTypes.FIELD;
+import static org.apache.flink.table.api.DataTypes.FLOAT;
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link CanalJsonDeserializationSchema}.
+ */
+public class CanalJsonDeserializationSchemaTest {
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	private static final RowType SCHEMA = (RowType) ROW(
+		FIELD("id", INT().notNull()),
+		FIELD("name", STRING()),
+		FIELD("description", STRING()),
+		FIELD("weight", FLOAT())
+	).getLogicalType();
+
+	@Test
+	public void testDeserialization() throws Exception {
+		List<String> lines = readLines("canal-data.txt");
+		CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema(
+			SCHEMA,
+			new RowDataTypeInfo(SCHEMA),
+			false);
+
+		SimpleCollector collector = new SimpleCollector();
+		for (String line : lines) {
+			deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector);
+		}
+
+		// Debezium captures change data (`debezium-data-schema-include.txt`) on the `product` table:
+		//

Review comment:
       Canal ?

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalFormatFactory.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.json.canal;
+
+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.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.ScanFormat;
+import org.apache.flink.table.connector.format.SinkFormat;
+import org.apache.flink.table.connector.source.ScanTableSource;
+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.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Format factory for providing configured instances of Canal JSON to RowData {@link DeserializationSchema}.
+ */
+public class CanalFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {
+
+	public static final String IDENTIFIER = "canal-json";

Review comment:
       `CanalJsonFormatFactory` ? or makes the underlying format configurable because Canal default format is Protobuf




----------------------------------------------------------------
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:
users@infra.apache.org