[FLINK-30859] Externalize confluent avro related code

[FLINK-30859] Fix NOTICE file to point to the right kafka-clients version

[FLINK-30859] Redirect confluent-avro deps to project.version

[FLINK-30859] port json debezium, rename flink-formats-kafka and cleanup poms

Co-Authored-By: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml
index fea104d..75ff2cc 100644
--- a/flink-confluent-schema-registry-e2e-tests/pom.xml
+++ b/flink-confluent-schema-registry-e2e-tests/pom.xml
@@ -20,9 +20,9 @@
 		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<parent>
-		<artifactId>flink-end-to-end-tests</artifactId>
 		<groupId>org.apache.flink</groupId>
-		<version>1.18-SNAPSHOT</version>
+		<artifactId>flink-connector-kafka-parent</artifactId>
+		<version>4.1-SNAPSHOT</version>
 	</parent>
 	<modelVersion>4.0.0</modelVersion>
 
@@ -44,10 +44,9 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 		</dependency>
-		<!-- https://mvnrepository.com/artifact/org.apache.flink/flink-connector-kafka -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-connector-kafka</artifactId>
@@ -56,7 +55,7 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-avro</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml
index 6a87c46..bd7c67a 100644
--- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml
+++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml
@@ -37,10 +37,6 @@
         </repository>
     </repositories>
 
-    <properties>
-        <confluent.version>7.2.2</confluent.version>
-    </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.flink</groupId>
@@ -95,7 +91,7 @@
             <!-- Used by maven-dependency-plugin -->
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-sql-avro-confluent-registry</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
 
@@ -198,7 +194,7 @@
                         <artifactItem>
                             <groupId>org.apache.flink</groupId>
                             <artifactId>flink-sql-avro-confluent-registry</artifactId>
-                            <version>${flink.version}</version>
+                            <version>${project.version}</version>
                             <destFileName>avro-confluent.jar</destFileName>
                             <type>jar</type>
                             <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml
index 5c11705..f754ae2 100644
--- a/flink-connector-kafka/pom.xml
+++ b/flink-connector-kafka/pom.xml
@@ -34,11 +34,6 @@
 
     <packaging>jar</packaging>
 
-    <properties>
-        <scala.binary.version>2.12</scala.binary.version>
-        <powermock.version>2.0.9</powermock.version>
-    </properties>
-
     <dependencies>
 
         <!-- Core -->
@@ -241,7 +236,7 @@
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-avro-confluent-registry</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
index 7ab0503..b0d6149 100644
--- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
+++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
@@ -866,7 +866,7 @@
 
     private SerializationSchema<RowData> createDebeziumAvroSerSchema(
             RowType rowType, String subject) {
-        return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null);
+        return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null, null);
     }
 
     // --------------------------------------------------------------------------------------------
diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml
index d366760..a74fe8f 100644
--- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml
+++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml
@@ -21,9 +21,9 @@
 		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<parent>
-		<artifactId>flink-formats</artifactId>
 		<groupId>org.apache.flink</groupId>
-		<version>1.18-SNAPSHOT</version>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
 	</parent>
 	<modelVersion>4.0.0</modelVersion>
 
@@ -31,11 +31,6 @@
 
 	<name>Flink : Formats : Avro confluent registry</name>
 
-	<properties>
-		<kafka.version>3.2.3</kafka.version>
-		<confluent.version>7.2.2</confluent.version>
-	</properties>
-
 	<repositories>
 		<repository>
 			<id>confluent</id>
@@ -98,18 +93,18 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-avro</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-api-java</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
@@ -118,7 +113,7 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 			<optional>true</optional>
 		</dependency>
@@ -126,7 +121,7 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-connector-files</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 			<optional>true</optional>
 		</dependency>
@@ -135,30 +130,36 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-api-java</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-runtime</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-avro</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
+		<dependency>
+			<groupId>org.powermock</groupId>
+			<artifactId>powermock-reflect</artifactId>
+			<version>${powermock.version}</version>
+			<scope>test</scope>
+		</dependency>
 
 		<!-- ArchUit test dependencies -->
 
@@ -175,4 +176,29 @@
 		</dependency>
 	</dependencies>
 
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-format-common</include>
+								</includes>
+							</artifactSet>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
 </project>
diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162
index a13a3ce..e69de29 100644
--- a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162
+++ b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162
@@ -1,6 +0,0 @@
-org.apache.flink.formats.json.JsonBatchFileSystemITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
\ No newline at end of file
diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml
index 686ad5d..147cf0e 100644
--- a/flink-formats-kafka/flink-json-debezium/pom.xml
+++ b/flink-formats-kafka/flink-json-debezium/pom.xml
@@ -24,63 +24,45 @@
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-formats</artifactId>
-		<version>1.18-SNAPSHOT</version>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
 	</parent>
 
-	<artifactId>flink-json</artifactId>
-	<name>Flink : Formats : Json</name>
+	<artifactId>flink-json-debezium</artifactId>
+	<name>Flink : Formats : Json Debezium</name>
 
 	<packaging>jar</packaging>
 
 	<dependencies>
-
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-format-common</artifactId>
-			<version>${project.version}</version>
+			<artifactId>flink-json</artifactId>
+			<version>${flink.version}</version>
 		</dependency>
 
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-jackson</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Table ecosystem and filesystem connector -->
-
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<optional>true</optional>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-files</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>provided</scope>
 			<optional>true</optional>
 		</dependency>
 
 		<!-- test dependencies -->
 
+		<!-- Required for integration test -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-files</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
 		<!-- JSON table descriptor testing -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
@@ -89,7 +71,7 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 
@@ -97,13 +79,13 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-api-scala-bridge_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
@@ -112,20 +94,13 @@
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-connector-test-utils</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<!-- JSON RowData schema test dependency -->
-		<dependency>
-			<groupId>org.scala-lang</groupId>
-			<artifactId>scala-compiler</artifactId>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java
deleted file mode 100644
index cc244b0..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.AbstractDeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.util.function.SerializableSupplier;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.IOException;
-
-/** DeserializationSchema that deserializes a JSON String. */
-@PublicEvolving
-public class JsonDeserializationSchema<T> extends AbstractDeserializationSchema<T> {
-
-    private static final long serialVersionUID = 1L;
-
-    private final Class<T> clazz;
-    private final SerializableSupplier<ObjectMapper> mapperFactory;
-    protected transient ObjectMapper mapper;
-
-    public JsonDeserializationSchema(Class<T> clazz) {
-        this(clazz, JacksonMapperFactory::createObjectMapper);
-    }
-
-    public JsonDeserializationSchema(TypeInformation<T> typeInformation) {
-        this(typeInformation, JacksonMapperFactory::createObjectMapper);
-    }
-
-    public JsonDeserializationSchema(
-            Class<T> clazz, SerializableSupplier<ObjectMapper> mapperFactory) {
-        super(clazz);
-        this.clazz = clazz;
-        this.mapperFactory = mapperFactory;
-    }
-
-    public JsonDeserializationSchema(
-            TypeInformation<T> typeInformation, SerializableSupplier<ObjectMapper> mapperFactory) {
-        super(typeInformation);
-        this.clazz = typeInformation.getTypeClass();
-        this.mapperFactory = mapperFactory;
-    }
-
-    @Override
-    public void open(InitializationContext context) {
-        mapper = mapperFactory.get();
-    }
-
-    @Override
-    public T deserialize(byte[] message) throws IOException {
-        return mapper.readValue(message, clazz);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java
deleted file mode 100644
index 74d8c53..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-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.formats.common.TimestampFormat;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
-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 java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
-import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD;
-import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS;
-import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_LITERAL;
-import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE;
-import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT;
-
-/**
- * Table format factory for providing configured instances of JSON to RowData {@link
- * SerializationSchema} and {@link DeserializationSchema}.
- */
-@Internal
-public class JsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {
-
-    public static final String IDENTIFIER = "json";
-
-    @Override
-    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        JsonFormatOptionsUtil.validateDecodingFormatOptions(formatOptions);
-
-        final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD);
-        final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
-        TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-
-        return new ProjectableDecodingFormat<DeserializationSchema<RowData>>() {
-            @Override
-            public DeserializationSchema<RowData> createRuntimeDecoder(
-                    DynamicTableSource.Context context,
-                    DataType physicalDataType,
-                    int[][] projections) {
-                final DataType producedDataType =
-                        Projection.of(projections).project(physicalDataType);
-                final RowType rowType = (RowType) producedDataType.getLogicalType();
-                final TypeInformation<RowData> rowDataTypeInfo =
-                        context.createTypeInformation(producedDataType);
-                return new JsonRowDataDeserializationSchema(
-                        rowType,
-                        rowDataTypeInfo,
-                        failOnMissingField,
-                        ignoreParseErrors,
-                        timestampOption);
-            }
-
-            @Override
-            public ChangelogMode getChangelogMode() {
-                return ChangelogMode.insertOnly();
-            }
-        };
-    }
-
-    @Override
-    public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        JsonFormatOptionsUtil.validateEncodingFormatOptions(formatOptions);
-
-        TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-        JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
-                JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
-        String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL);
-
-        final boolean encodeDecimalAsPlainNumber =
-                formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-
-        return new EncodingFormat<SerializationSchema<RowData>>() {
-            @Override
-            public SerializationSchema<RowData> createRuntimeEncoder(
-                    DynamicTableSink.Context context, DataType consumedDataType) {
-                final RowType rowType = (RowType) consumedDataType.getLogicalType();
-                return new JsonRowDataSerializationSchema(
-                        rowType,
-                        timestampOption,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-            }
-
-            @Override
-            public ChangelogMode getChangelogMode() {
-                return ChangelogMode.insertOnly();
-            }
-        };
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(FAIL_ON_MISSING_FIELD);
-        options.add(IGNORE_PARSE_ERRORS);
-        options.add(TIMESTAMP_FORMAT);
-        options.add(MAP_NULL_KEY_MODE);
-        options.add(MAP_NULL_KEY_LITERAL);
-        options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-        return options;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> forwardOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(TIMESTAMP_FORMAT);
-        options.add(MAP_NULL_KEY_MODE);
-        options.add(MAP_NULL_KEY_LITERAL);
-        options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-        return options;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java
deleted file mode 100644
index 74d567a..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-
-/** Options for the JSON format. */
-@PublicEvolving
-public class JsonFormatOptions {
-
-    public static final ConfigOption<Boolean> FAIL_ON_MISSING_FIELD =
-            ConfigOptions.key("fail-on-missing-field")
-                    .booleanType()
-                    .defaultValue(false)
-                    .withDescription(
-                            "Optional flag to specify whether to fail if a field is missing or not, false by default.");
-
-    public static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS =
-            ConfigOptions.key("ignore-parse-errors")
-                    .booleanType()
-                    .defaultValue(false)
-                    .withDescription(
-                            "Optional flag to skip fields and rows with parse errors instead of failing;\n"
-                                    + "fields are set to null in case of errors, false by default.");
-
-    public static final ConfigOption<String> MAP_NULL_KEY_MODE =
-            ConfigOptions.key("map-null-key.mode")
-                    .stringType()
-                    .defaultValue("FAIL")
-                    .withDescription(
-                            "Optional flag to control the handling mode when serializing null key for map data, FAIL by default."
-                                    + " Option DROP will drop null key entries for map data."
-                                    + " Option LITERAL will use 'map-null-key.literal' as key literal.");
-
-    public static final ConfigOption<String> MAP_NULL_KEY_LITERAL =
-            ConfigOptions.key("map-null-key.literal")
-                    .stringType()
-                    .defaultValue("null")
-                    .withDescription(
-                            "Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default.");
-
-    public static final ConfigOption<String> TIMESTAMP_FORMAT =
-            ConfigOptions.key("timestamp-format.standard")
-                    .stringType()
-                    .defaultValue("SQL")
-                    .withDescription(
-                            "Optional flag to specify timestamp format, SQL by default."
-                                    + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format."
-                                    + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format.");
-
-    public static final ConfigOption<Boolean> ENCODE_DECIMAL_AS_PLAIN_NUMBER =
-            ConfigOptions.key("encode.decimal-as-plain-number")
-                    .booleanType()
-                    .defaultValue(false)
-                    .withDescription(
-                            "Optional flag to specify whether to encode all decimals as plain numbers instead of possible scientific notations, false by default.");
-
-    // --------------------------------------------------------------------------------------------
-    // Enums
-    // --------------------------------------------------------------------------------------------
-
-    /** Handling mode for map data with null key. */
-    public enum MapNullKeyMode {
-        FAIL,
-        DROP,
-        LITERAL
-    }
-
-    private JsonFormatOptions() {}
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java
deleted file mode 100644
index 60a953d..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.api.ValidationException;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD;
-import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS;
-import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE;
-import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT;
-
-/** Utilities for {@link JsonFormatOptions}. */
-@Internal
-public class JsonFormatOptionsUtil {
-
-    // --------------------------------------------------------------------------------------------
-    // Option enumerations
-    // --------------------------------------------------------------------------------------------
-
-    public static final String SQL = "SQL";
-    public static final String ISO_8601 = "ISO-8601";
-
-    public static final Set<String> TIMESTAMP_FORMAT_ENUM =
-            new HashSet<>(Arrays.asList(SQL, ISO_8601));
-
-    // The handling mode of null key for map data
-    public static final String JSON_MAP_NULL_KEY_MODE_FAIL = "FAIL";
-    public static final String JSON_MAP_NULL_KEY_MODE_DROP = "DROP";
-    public static final String JSON_MAP_NULL_KEY_MODE_LITERAL = "LITERAL";
-
-    // --------------------------------------------------------------------------------------------
-    // Utilities
-    // --------------------------------------------------------------------------------------------
-
-    public static TimestampFormat getTimestampFormat(ReadableConfig config) {
-        String timestampFormat = config.get(TIMESTAMP_FORMAT);
-        switch (timestampFormat) {
-            case SQL:
-                return TimestampFormat.SQL;
-            case ISO_8601:
-                return TimestampFormat.ISO_8601;
-            default:
-                throw new TableException(
-                        String.format(
-                                "Unsupported timestamp format '%s'. Validator should have checked that.",
-                                timestampFormat));
-        }
-    }
-
-    /**
-     * Creates handling mode for null key map data.
-     *
-     * <p>See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, and {@link
-     * #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information.
-     */
-    public static JsonFormatOptions.MapNullKeyMode getMapNullKeyMode(ReadableConfig config) {
-        String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE);
-        switch (mapNullKeyMode.toUpperCase()) {
-            case JSON_MAP_NULL_KEY_MODE_FAIL:
-                return JsonFormatOptions.MapNullKeyMode.FAIL;
-            case JSON_MAP_NULL_KEY_MODE_DROP:
-                return JsonFormatOptions.MapNullKeyMode.DROP;
-            case JSON_MAP_NULL_KEY_MODE_LITERAL:
-                return JsonFormatOptions.MapNullKeyMode.LITERAL;
-            default:
-                throw new TableException(
-                        String.format(
-                                "Unsupported map null key handling mode '%s'. Validator should have checked that.",
-                                mapNullKeyMode));
-        }
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Validation
-    // --------------------------------------------------------------------------------------------
-
-    /** Validator for json decoding format. */
-    public static void validateDecodingFormatOptions(ReadableConfig tableOptions) {
-        boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD);
-        boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS);
-        if (ignoreParseErrors && failOnMissingField) {
-            throw new ValidationException(
-                    FAIL_ON_MISSING_FIELD.key()
-                            + " and "
-                            + IGNORE_PARSE_ERRORS.key()
-                            + " shouldn't both be true.");
-        }
-        validateTimestampFormat(tableOptions);
-    }
-
-    /** Validator for json encoding format. */
-    public static void validateEncodingFormatOptions(ReadableConfig tableOptions) {
-        // validator for {@link MAP_NULL_KEY_MODE}
-        Set<String> nullKeyModes =
-                Arrays.stream(JsonFormatOptions.MapNullKeyMode.values())
-                        .map(Objects::toString)
-                        .collect(Collectors.toSet());
-        if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())) {
-            throw new ValidationException(
-                    String.format(
-                            "Unsupported value '%s' for option %s. Supported values are %s.",
-                            tableOptions.get(MAP_NULL_KEY_MODE),
-                            MAP_NULL_KEY_MODE.key(),
-                            nullKeyModes));
-        }
-        validateTimestampFormat(tableOptions);
-    }
-
-    /** Validates timestamp format which value should be SQL or ISO-8601. */
-    static void validateTimestampFormat(ReadableConfig tableOptions) {
-        String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT);
-        if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)) {
-            throw new ValidationException(
-                    String.format(
-                            "Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].",
-                            timestampFormat, TIMESTAMP_FORMAT.key()));
-        }
-    }
-
-    private JsonFormatOptionsUtil() {}
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java
deleted file mode 100644
index 928a6f1..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-/**
- * DeserializationSchema that deserializes a JSON String into an ObjectNode.
- *
- * <p>Fields can be accessed by calling objectNode.get(&lt;name>).as(&lt;type>)
- *
- * @deprecated Use {@code new JsonDeserializationSchema(ObjectNode.class)} instead
- */
-@Deprecated
-public class JsonNodeDeserializationSchema extends JsonDeserializationSchema<ObjectNode> {
-
-    private static final long serialVersionUID = 2L;
-
-    public JsonNodeDeserializationSchema() {
-        super(ObjectNode.class);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
deleted file mode 100644
index 9a57bac..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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;
-
-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.common.TimestampFormat;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.json.JsonReadFeature;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.Objects;
-
-import static java.lang.String.format;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Deserialization schema from JSON to Flink Table/SQL internal data structure {@link RowData}.
- *
- * <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.
- */
-@Internal
-public class JsonRowDataDeserializationSchema implements DeserializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    /** Flag indicating whether to fail if a field is missing. */
-    private final boolean failOnMissingField;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    /** TypeInformation of the produced {@link RowData}. */
-    private final TypeInformation<RowData> resultTypeInfo;
-
-    /**
-     * Runtime converter that converts {@link JsonNode}s into objects of Flink SQL internal data
-     * structures.
-     */
-    private final JsonToRowDataConverters.JsonToRowDataConverter runtimeConverter;
-
-    /** Object mapper for parsing the JSON. */
-    private transient ObjectMapper objectMapper;
-
-    /** Timestamp format specification which is used to parse timestamp. */
-    private final TimestampFormat timestampFormat;
-
-    private final boolean hasDecimalType;
-
-    public JsonRowDataDeserializationSchema(
-            RowType rowType,
-            TypeInformation<RowData> resultTypeInfo,
-            boolean failOnMissingField,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        if (ignoreParseErrors && failOnMissingField) {
-            throw new IllegalArgumentException(
-                    "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled.");
-        }
-        this.resultTypeInfo = checkNotNull(resultTypeInfo);
-        this.failOnMissingField = failOnMissingField;
-        this.ignoreParseErrors = ignoreParseErrors;
-        this.runtimeConverter =
-                new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat)
-                        .createConverter(checkNotNull(rowType));
-        this.timestampFormat = timestampFormat;
-        this.hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType);
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        objectMapper =
-                JacksonMapperFactory.createObjectMapper()
-                        .configure(
-                                JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(),
-                                true);
-        if (hasDecimalType) {
-            objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);
-        }
-    }
-
-    @Override
-    public RowData deserialize(@Nullable byte[] message) throws IOException {
-        if (message == null) {
-            return null;
-        }
-        try {
-            return convertToRowData(deserializeToJsonNode(message));
-        } catch (Throwable t) {
-            if (ignoreParseErrors) {
-                return null;
-            }
-            throw new IOException(
-                    format("Failed to deserialize JSON '%s'.", new String(message)), t);
-        }
-    }
-
-    public JsonNode deserializeToJsonNode(byte[] message) throws IOException {
-        return objectMapper.readTree(message);
-    }
-
-    public RowData convertToRowData(JsonNode message) {
-        return (RowData) runtimeConverter.convert(message);
-    }
-
-    @Override
-    public boolean isEndOfStream(RowData nextElement) {
-        return false;
-    }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        return resultTypeInfo;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o;
-        return failOnMissingField == that.failOnMissingField
-                && ignoreParseErrors == that.ignoreParseErrors
-                && resultTypeInfo.equals(that.resultTypeInfo)
-                && timestampFormat.equals(that.timestampFormat);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java
deleted file mode 100644
index c8b7f73..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import java.util.Objects;
-
-/**
- * Serialization schema that serializes an object of Flink internal data structure into a JSON
- * bytes.
- *
- * <p>Serializes the input Flink object into a JSON string and converts it into <code>byte[]</code>.
- *
- * <p>Result <code>byte[]</code> messages can be deserialized using {@link
- * JsonRowDataDeserializationSchema}.
- */
-@Internal
-public class JsonRowDataSerializationSchema implements SerializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    /** RowType to generate the runtime converter. */
-    private final RowType rowType;
-
-    /** The converter that converts internal data formats to JsonNode. */
-    private final RowDataToJsonConverters.RowDataToJsonConverter runtimeConverter;
-
-    /** Object mapper that is used to create output JSON objects. */
-    private transient ObjectMapper mapper;
-
-    /** Reusable object node. */
-    private transient ObjectNode node;
-
-    /** Timestamp format specification which is used to parse timestamp. */
-    private final TimestampFormat timestampFormat;
-
-    /** The handling mode when serializing null keys for map data. */
-    private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode;
-
-    /** The string literal when handling mode for map null key LITERAL. */
-    private final String mapNullKeyLiteral;
-
-    /** Flag indicating whether to serialize all decimals as plain numbers. */
-    private final boolean encodeDecimalAsPlainNumber;
-
-    public JsonRowDataSerializationSchema(
-            RowType rowType,
-            TimestampFormat timestampFormat,
-            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
-            String mapNullKeyLiteral,
-            boolean encodeDecimalAsPlainNumber) {
-        this.rowType = rowType;
-        this.timestampFormat = timestampFormat;
-        this.mapNullKeyMode = mapNullKeyMode;
-        this.mapNullKeyLiteral = mapNullKeyLiteral;
-        this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber;
-        this.runtimeConverter =
-                new RowDataToJsonConverters(timestampFormat, mapNullKeyMode, mapNullKeyLiteral)
-                        .createConverter(rowType);
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        mapper =
-                JacksonMapperFactory.createObjectMapper()
-                        .configure(
-                                JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN,
-                                encodeDecimalAsPlainNumber);
-    }
-
-    @Override
-    public byte[] serialize(RowData row) {
-        if (node == null) {
-            node = mapper.createObjectNode();
-        }
-
-        try {
-            runtimeConverter.convert(mapper, node, row);
-            return mapper.writeValueAsBytes(node);
-        } catch (Throwable t) {
-            throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        JsonRowDataSerializationSchema that = (JsonRowDataSerializationSchema) o;
-        return rowType.equals(that.rowType)
-                && timestampFormat.equals(that.timestampFormat)
-                && mapNullKeyMode.equals(that.mapNullKeyMode)
-                && mapNullKeyLiteral.equals(that.mapNullKeyLiteral)
-                && encodeDecimalAsPlainNumber == that.encodeDecimalAsPlainNumber;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                rowType,
-                timestampFormat,
-                mapNullKeyMode,
-                mapNullKeyLiteral,
-                encodeDecimalAsPlainNumber);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java
deleted file mode 100644
index dd4a9bb..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java
+++ /dev/null
@@ -1,587 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.MapTypeInfo;
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.Array;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.ZoneOffset;
-import java.time.temporal.TemporalAccessor;
-import java.time.temporal.TemporalQueries;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
-import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT;
-import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Deserialization schema from JSON to Flink types.
- *
- * <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.
- *
- * @deprecated The format was developed for the Table API users and will not be maintained for
- *     DataStream API users anymore. Either use Table API or switch to Data Stream, defining your
- *     own {@link DeserializationSchema}.
- */
-@PublicEvolving
-@Deprecated
-public class JsonRowDeserializationSchema implements DeserializationSchema<Row> {
-
-    private static final long serialVersionUID = -228294330688809195L;
-
-    /** Type information describing the result type. */
-    private final RowTypeInfo typeInfo;
-
-    private boolean failOnMissingField;
-
-    private final boolean hasDecimalType;
-
-    /** Object mapper for parsing the JSON. */
-    private transient ObjectMapper objectMapper;
-
-    private DeserializationRuntimeConverter runtimeConverter;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    private JsonRowDeserializationSchema(
-            TypeInformation<Row> typeInfo, boolean failOnMissingField, boolean ignoreParseErrors) {
-        checkNotNull(typeInfo, "Type information");
-        checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
-        if (ignoreParseErrors && failOnMissingField) {
-            throw new IllegalArgumentException(
-                    "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true.");
-        }
-        this.typeInfo = (RowTypeInfo) typeInfo;
-        this.failOnMissingField = failOnMissingField;
-        this.runtimeConverter = createConverter(this.typeInfo);
-        this.ignoreParseErrors = ignoreParseErrors;
-        RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType();
-        hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL));
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        objectMapper = JacksonMapperFactory.createObjectMapper();
-        if (hasDecimalType) {
-            objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);
-        }
-    }
-
-    /** @deprecated Use the provided {@link Builder} instead. */
-    @Deprecated
-    public JsonRowDeserializationSchema(TypeInformation<Row> typeInfo) {
-        this(typeInfo, false, false);
-    }
-
-    /** @deprecated Use the provided {@link Builder} instead. */
-    @Deprecated
-    public JsonRowDeserializationSchema(String jsonSchema) {
-        this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false);
-    }
-
-    /** @deprecated Use the provided {@link Builder} instead. */
-    @Deprecated
-    public void setFailOnMissingField(boolean failOnMissingField) {
-        // TODO make this class immutable once we drop this method
-        this.failOnMissingField = failOnMissingField;
-        this.runtimeConverter = createConverter(this.typeInfo);
-    }
-
-    @Override
-    public Row deserialize(byte[] message) throws IOException {
-        try {
-            final JsonNode root = objectMapper.readTree(message);
-            return (Row) runtimeConverter.convert(objectMapper, root);
-        } catch (Throwable t) {
-            if (ignoreParseErrors) {
-                return null;
-            }
-            throw new IOException(
-                    format("Failed to deserialize JSON '%s'.", new String(message)), t);
-        }
-    }
-
-    @Override
-    public boolean isEndOfStream(Row nextElement) {
-        return false;
-    }
-
-    @Override
-    public TypeInformation<Row> getProducedType() {
-        return typeInfo;
-    }
-
-    /** Builder for {@link JsonRowDeserializationSchema}. */
-    public static class Builder {
-
-        private final RowTypeInfo typeInfo;
-        private boolean failOnMissingField = false;
-        private boolean ignoreParseErrors = false;
-
-        /**
-         * Creates a JSON deserialization schema for the given type information.
-         *
-         * @param typeInfo Type information describing the result type. The field names of {@link
-         *     Row} are used to parse the JSON properties.
-         */
-        public Builder(TypeInformation<Row> typeInfo) {
-            checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
-            this.typeInfo = (RowTypeInfo) typeInfo;
-        }
-
-        /**
-         * Creates a JSON deserialization schema for the given JSON schema.
-         *
-         * @param jsonSchema JSON schema describing the result type
-         * @see <a href="http://json-schema.org/">http://json-schema.org/</a>
-         */
-        public Builder(String jsonSchema) {
-            this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)));
-        }
-
-        /**
-         * Configures schema to fail if a JSON field is missing.
-         *
-         * <p>By default, a missing field is ignored and the field is set to null.
-         */
-        public Builder failOnMissingField() {
-            this.failOnMissingField = true;
-            return this;
-        }
-
-        /**
-         * Configures schema to fail when parsing json failed.
-         *
-         * <p>By default, an exception will be thrown when parsing json fails.
-         */
-        public Builder ignoreParseErrors() {
-            this.ignoreParseErrors = true;
-            return this;
-        }
-
-        public JsonRowDeserializationSchema build() {
-            return new JsonRowDeserializationSchema(
-                    typeInfo, failOnMissingField, ignoreParseErrors);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o;
-        return Objects.equals(typeInfo, that.typeInfo)
-                && Objects.equals(failOnMissingField, that.failOnMissingField)
-                && Objects.equals(ignoreParseErrors, that.ignoreParseErrors);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors);
-    }
-
-    /*
-    Runtime converter
-    */
-
-    /** Runtime converter that maps between {@link JsonNode}s and Java objects. */
-    @FunctionalInterface
-    private interface DeserializationRuntimeConverter extends Serializable {
-        Object convert(ObjectMapper mapper, JsonNode jsonNode);
-    }
-
-    private DeserializationRuntimeConverter createConverter(TypeInformation<?> typeInfo) {
-        DeserializationRuntimeConverter baseConverter =
-                createConverterForSimpleType(typeInfo)
-                        .orElseGet(
-                                () ->
-                                        createContainerConverter(typeInfo)
-                                                .orElseGet(
-                                                        () ->
-                                                                createFallbackConverter(
-                                                                        typeInfo.getTypeClass())));
-        return wrapIntoNullableConverter(baseConverter);
-    }
-
-    private DeserializationRuntimeConverter wrapIntoNullableConverter(
-            DeserializationRuntimeConverter converter) {
-        return (mapper, jsonNode) -> {
-            if (jsonNode.isNull()) {
-                return null;
-            }
-            try {
-                return converter.convert(mapper, jsonNode);
-            } catch (Throwable t) {
-                if (!ignoreParseErrors) {
-                    throw t;
-                }
-                return null;
-            }
-        };
-    }
-
-    private Optional<DeserializationRuntimeConverter> createContainerConverter(
-            TypeInformation<?> typeInfo) {
-        if (typeInfo instanceof RowTypeInfo) {
-            return Optional.of(createRowConverter((RowTypeInfo) typeInfo));
-        } else if (typeInfo instanceof ObjectArrayTypeInfo) {
-            return Optional.of(
-                    createObjectArrayConverter(
-                            ((ObjectArrayTypeInfo) typeInfo).getComponentInfo()));
-        } else if (typeInfo instanceof BasicArrayTypeInfo) {
-            return Optional.of(
-                    createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo()));
-        } else if (isPrimitiveByteArray(typeInfo)) {
-            return Optional.of(createByteArrayConverter());
-        } else if (typeInfo instanceof MapTypeInfo) {
-            MapTypeInfo<?, ?> mapTypeInfo = (MapTypeInfo<?, ?>) typeInfo;
-            return Optional.of(
-                    createMapConverter(
-                            mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo()));
-        } else {
-            return Optional.empty();
-        }
-    }
-
-    private DeserializationRuntimeConverter createMapConverter(
-            TypeInformation keyType, TypeInformation valueType) {
-        DeserializationRuntimeConverter valueConverter = createConverter(valueType);
-        DeserializationRuntimeConverter keyConverter = createConverter(keyType);
-        return (mapper, jsonNode) -> {
-            Iterator<Map.Entry<String, JsonNode>> fields = jsonNode.fields();
-            Map<Object, Object> result = new HashMap<>();
-            while (fields.hasNext()) {
-                Map.Entry<String, JsonNode> entry = fields.next();
-                Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey()));
-                Object value = valueConverter.convert(mapper, entry.getValue());
-                result.put(key, value);
-            }
-            return result;
-        };
-    }
-
-    private DeserializationRuntimeConverter createByteArrayConverter() {
-        return (mapper, jsonNode) -> {
-            try {
-                return jsonNode.binaryValue();
-            } catch (IOException e) {
-                throw new JsonParseException("Unable to deserialize byte array.", e);
-            }
-        };
-    }
-
-    private boolean isPrimitiveByteArray(TypeInformation<?> typeInfo) {
-        return typeInfo instanceof PrimitiveArrayTypeInfo
-                && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE;
-    }
-
-    private DeserializationRuntimeConverter createObjectArrayConverter(
-            TypeInformation elementTypeInfo) {
-        DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo);
-        return assembleArrayConverter(elementTypeInfo, elementConverter);
-    }
-
-    private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) {
-        List<DeserializationRuntimeConverter> fieldConverters =
-                Arrays.stream(typeInfo.getFieldTypes())
-                        .map(this::createConverter)
-                        .collect(Collectors.toList());
-
-        return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters);
-    }
-
-    private DeserializationRuntimeConverter createFallbackConverter(Class<?> valueType) {
-        return (mapper, jsonNode) -> {
-            // for types that were specified without JSON schema
-            // e.g. POJOs
-            try {
-                return mapper.treeToValue(jsonNode, valueType);
-            } catch (JsonProcessingException e) {
-                throw new JsonParseException(format("Could not convert node: %s", jsonNode), e);
-            }
-        };
-    }
-
-    private Optional<DeserializationRuntimeConverter> createConverterForSimpleType(
-            TypeInformation<?> simpleTypeInfo) {
-        if (simpleTypeInfo == Types.VOID) {
-            return Optional.of((mapper, jsonNode) -> null);
-        } else if (simpleTypeInfo == Types.BOOLEAN) {
-            return Optional.of(this::convertToBoolean);
-        } else if (simpleTypeInfo == Types.STRING) {
-            return Optional.of(this::convertToString);
-        } else if (simpleTypeInfo == Types.INT) {
-            return Optional.of(this::convertToInt);
-        } else if (simpleTypeInfo == Types.LONG) {
-            return Optional.of(this::convertToLong);
-        } else if (simpleTypeInfo == Types.DOUBLE) {
-            return Optional.of(this::convertToDouble);
-        } else if (simpleTypeInfo == Types.FLOAT) {
-            return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim()));
-        } else if (simpleTypeInfo == Types.SHORT) {
-            return Optional.of((mapper, jsonNode) -> Short.parseShort(jsonNode.asText().trim()));
-        } else if (simpleTypeInfo == Types.BYTE) {
-            return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim()));
-        } else if (simpleTypeInfo == Types.BIG_DEC) {
-            return Optional.of(this::convertToBigDecimal);
-        } else if (simpleTypeInfo == Types.BIG_INT) {
-            return Optional.of(this::convertToBigInteger);
-        } else if (simpleTypeInfo == Types.SQL_DATE) {
-            return Optional.of(this::convertToDate);
-        } else if (simpleTypeInfo == Types.SQL_TIME) {
-            return Optional.of(this::convertToTime);
-        } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) {
-            return Optional.of(this::convertToTimestamp);
-        } else if (simpleTypeInfo == Types.LOCAL_DATE) {
-            return Optional.of(this::convertToLocalDate);
-        } else if (simpleTypeInfo == Types.LOCAL_TIME) {
-            return Optional.of(this::convertToLocalTime);
-        } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) {
-            return Optional.of(this::convertToLocalDateTime);
-        } else {
-            return Optional.empty();
-        }
-    }
-
-    private String convertToString(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.isContainerNode()) {
-            return jsonNode.toString();
-        } else {
-            return jsonNode.asText();
-        }
-    }
-
-    private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.isBoolean()) {
-            // avoid redundant toString and parseBoolean, for better performance
-            return jsonNode.asBoolean();
-        } else {
-            return Boolean.parseBoolean(jsonNode.asText().trim());
-        }
-    }
-
-    private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.canConvertToInt()) {
-            // avoid redundant toString and parseInt, for better performance
-            return jsonNode.asInt();
-        } else {
-            return Integer.parseInt(jsonNode.asText().trim());
-        }
-    }
-
-    private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.canConvertToLong()) {
-            // avoid redundant toString and parseLong, for better performance
-            return jsonNode.asLong();
-        } else {
-            return Long.parseLong(jsonNode.asText().trim());
-        }
-    }
-
-    private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.isDouble()) {
-            // avoid redundant toString and parseDouble, for better performance
-            return jsonNode.asDouble();
-        } else {
-            return Double.parseDouble(jsonNode.asText().trim());
-        }
-    }
-
-    private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.isBigDecimal()) {
-            // avoid redundant toString and toDecimal, for better performance
-            return jsonNode.decimalValue();
-        } else {
-            return new BigDecimal(jsonNode.asText().trim());
-        }
-    }
-
-    private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) {
-        if (jsonNode.isBigInteger()) {
-            // avoid redundant toString and toBigInteger, for better performance
-            return jsonNode.bigIntegerValue();
-        } else {
-            return new BigInteger(jsonNode.asText().trim());
-        }
-    }
-
-    private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) {
-        return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate());
-    }
-
-    private Date convertToDate(ObjectMapper mapper, JsonNode jsonNode) {
-        return Date.valueOf(convertToLocalDate(mapper, jsonNode));
-    }
-
-    private LocalDateTime convertToLocalDateTime(ObjectMapper mapper, JsonNode jsonNode) {
-        // according to RFC 3339 every date-time must have a timezone;
-        // until we have full timezone support, we only support UTC;
-        // users can parse their time as string as a workaround
-        TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText());
-
-        ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset());
-
-        if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) {
-            throw new IllegalStateException(
-                    "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. "
-                            + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'");
-        }
-
-        LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime());
-        LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate());
-
-        return LocalDateTime.of(localDate, localTime);
-    }
-
-    private Timestamp convertToTimestamp(ObjectMapper mapper, JsonNode jsonNode) {
-        return Timestamp.valueOf(convertToLocalDateTime(mapper, jsonNode));
-    }
-
-    private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) {
-        // according to RFC 3339 every full-time must have a timezone;
-        // until we have full timezone support, we only support UTC;
-        // users can parse their time as string as a workaround
-
-        TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText());
-
-        ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset());
-        LocalTime localTime = parsedTime.query(TemporalQueries.localTime());
-
-        if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) {
-            throw new IllegalStateException(
-                    "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet.");
-        }
-
-        return localTime;
-    }
-
-    private Time convertToTime(ObjectMapper mapper, JsonNode jsonNode) {
-        return Time.valueOf(convertToLocalTime(mapper, jsonNode));
-    }
-
-    private DeserializationRuntimeConverter assembleRowConverter(
-            String[] fieldNames, List<DeserializationRuntimeConverter> fieldConverters) {
-        return (mapper, jsonNode) -> {
-            ObjectNode node = (ObjectNode) jsonNode;
-            int arity = fieldNames.length;
-            Row row = new Row(arity);
-            for (int i = 0; i < arity; i++) {
-                String fieldName = fieldNames[i];
-                JsonNode field = node.get(fieldName);
-                Object convertField =
-                        convertField(mapper, fieldConverters.get(i), fieldName, field);
-                row.setField(i, convertField);
-            }
-
-            return row;
-        };
-    }
-
-    private Object convertField(
-            ObjectMapper mapper,
-            DeserializationRuntimeConverter fieldConverter,
-            String fieldName,
-            JsonNode field) {
-        if (field == null) {
-            if (failOnMissingField) {
-                throw new IllegalStateException(
-                        "Could not find field with name '" + fieldName + "'.");
-            } else {
-                return null;
-            }
-        } else {
-            return fieldConverter.convert(mapper, field);
-        }
-    }
-
-    private DeserializationRuntimeConverter assembleArrayConverter(
-            TypeInformation<?> elementType, DeserializationRuntimeConverter elementConverter) {
-
-        final Class<?> elementClass = elementType.getTypeClass();
-
-        return (mapper, jsonNode) -> {
-            final ArrayNode node = (ArrayNode) jsonNode;
-            final Object[] array = (Object[]) Array.newInstance(elementClass, node.size());
-            for (int i = 0; i < node.size(); i++) {
-                final JsonNode innerNode = node.get(i);
-                array[i] = elementConverter.convert(mapper, innerNode);
-            }
-
-            return array;
-        };
-    }
-
-    /** Exception which refers to parse errors in converters. */
-    private static final class JsonParseException extends RuntimeException {
-        private static final long serialVersionUID = 1L;
-
-        public JsonParseException(String message, Throwable cause) {
-            super(message, cause);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java
deleted file mode 100644
index d776185..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java
+++ /dev/null
@@ -1,382 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-
-/**
- * Converts a JSON schema into Flink's type information. It uses {@link Row} for representing
- * objects and tuple arrays.
- *
- * <p>Note: This converter implements just a subset of the JSON schema specification. Union types
- * (as well as "allOf", "anyOf", "not") are not supported yet. Simple references that link to a
- * common definition in the document are supported. "oneOf" and arrays of types are only supported
- * for specifying nullability.
- *
- * <p>This converter has been developed for JSON Schema draft-07 but also includes keywords of older
- * drafts to be as compatible as possible.
- */
-public final class JsonRowSchemaConverter {
-
-    private JsonRowSchemaConverter() {
-        // private
-    }
-
-    // see https://spacetelescope.github.io/understanding-json-schema/UnderstandingJSONSchema.pdf
-    private static final String PROPERTIES = "properties";
-    private static final String ADDITIONAL_PROPERTIES = "additionalProperties";
-    private static final String TYPE = "type";
-    private static final String FORMAT = "format";
-    private static final String CONTENT_ENCODING = "contentEncoding";
-    private static final String ITEMS = "items";
-    private static final String ADDITIONAL_ITEMS = "additionalItems";
-    private static final String REF = "$ref";
-    private static final String ALL_OF = "allOf";
-    private static final String ANY_OF = "anyOf";
-    private static final String NOT = "not";
-    private static final String ONE_OF = "oneOf";
-
-    // from https://tools.ietf.org/html/draft-zyp-json-schema-03#page-14
-    private static final String DISALLOW = "disallow";
-    private static final String EXTENDS = "extends";
-
-    private static final String TYPE_NULL = "null";
-    private static final String TYPE_BOOLEAN = "boolean";
-    private static final String TYPE_OBJECT = "object";
-    private static final String TYPE_ARRAY = "array";
-    private static final String TYPE_NUMBER = "number";
-    private static final String TYPE_INTEGER = "integer";
-    private static final String TYPE_STRING = "string";
-
-    private static final String FORMAT_DATE = "date";
-    private static final String FORMAT_TIME = "time";
-    private static final String FORMAT_DATE_TIME = "date-time";
-
-    private static final String CONTENT_ENCODING_BASE64 = "base64";
-
-    /**
-     * Converts a JSON schema into Flink's type information. Throws an exception if the schema
-     * cannot converted because of loss of precision or too flexible schema.
-     *
-     * <p>The converter can resolve simple schema references to solve those cases where entities are
-     * defined at the beginning and then used throughout a document.
-     */
-    @SuppressWarnings("unchecked")
-    public static <T> TypeInformation<T> convert(String jsonSchema) {
-        Preconditions.checkNotNull(jsonSchema, "JSON schema");
-        final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper();
-        mapper.getFactory()
-                .enable(JsonParser.Feature.ALLOW_COMMENTS)
-                .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES)
-                .enable(JsonParser.Feature.ALLOW_SINGLE_QUOTES);
-        final JsonNode node;
-        try {
-            node = mapper.readTree(jsonSchema);
-        } catch (IOException e) {
-            throw new IllegalArgumentException("Invalid JSON schema.", e);
-        }
-        return (TypeInformation<T>) convertType("<root>", node, node);
-    }
-
-    private static TypeInformation<?> convertType(String location, JsonNode node, JsonNode root) {
-        // we use a set here to unify types (e.g. types that just add metadata such as 'multipleOf')
-        final Set<TypeInformation<?>> typeSet = new HashSet<>();
-
-        // search for ref
-        final Optional<JsonNode> ref;
-        if (node.has(REF) && node.get(REF).isTextual()) {
-            // try a simple ref resolver to solve those cases where entities are defined at
-            // the beginning and then used throughout a document
-            ref = Optional.of(resolveReference(node.get(REF).asText(), node, root));
-        } else {
-            ref = Optional.empty();
-        }
-
-        // use TYPE of this node
-        if (node.has(TYPE)) {
-            final JsonNode typeNode = node.get(TYPE);
-
-            List<String> types = new ArrayList<>();
-            // array of types
-            if (typeNode.isArray()) {
-                final Iterator<JsonNode> elements = typeNode.elements();
-                while (elements.hasNext()) {
-                    types.add(elements.next().asText());
-                }
-            }
-            // single type
-            else if (typeNode.isTextual()) {
-                types.add(typeNode.asText());
-            }
-
-            for (String type : types) {
-                // set field type
-                switch (type) {
-                    case TYPE_NULL:
-                        typeSet.add(Types.VOID);
-                        break;
-                    case TYPE_BOOLEAN:
-                        typeSet.add(Types.BOOLEAN);
-                        break;
-                    case TYPE_STRING:
-                        if (node.has(FORMAT)) {
-                            typeSet.add(convertStringFormat(location, node.get(FORMAT)));
-                        } else if (node.has(CONTENT_ENCODING)) {
-                            typeSet.add(
-                                    convertStringEncoding(location, node.get(CONTENT_ENCODING)));
-                        } else {
-                            typeSet.add(Types.STRING);
-                        }
-                        break;
-                    case TYPE_NUMBER:
-                        typeSet.add(Types.BIG_DEC);
-                        break;
-                    case TYPE_INTEGER:
-                        // use BigDecimal for easier interoperability
-                        // without affecting the correctness of the result
-                        typeSet.add(Types.BIG_DEC);
-                        break;
-                    case TYPE_OBJECT:
-                        typeSet.add(convertObject(location, node, root));
-                        break;
-                    case TYPE_ARRAY:
-                        typeSet.add(convertArray(location, node, root));
-                        break;
-                    default:
-                        throw new IllegalArgumentException(
-                                "Unsupported type '"
-                                        + node.get(TYPE).asText()
-                                        + "' in node: "
-                                        + location);
-                }
-            }
-        }
-        // use TYPE of reference as fallback if present
-        else {
-            ref.filter(r -> r.has(TYPE))
-                    .ifPresent(r -> typeSet.add(convertType(node.get(REF).asText(), r, root)));
-        }
-
-        // simple interpretation of ONE_OF for supporting "object or null"
-        if (node.has(ONE_OF) && node.get(ONE_OF).isArray()) {
-            final TypeInformation<?>[] types =
-                    convertTypes(location + '/' + ONE_OF, node.get(ONE_OF), root);
-            typeSet.addAll(Arrays.asList(types));
-        }
-        // use ONE_OF of reference as fallback
-        else if (ref.isPresent() && ref.get().has(ONE_OF) && ref.get().get(ONE_OF).isArray()) {
-            final TypeInformation<?>[] types =
-                    convertTypes(
-                            node.get(REF).asText() + '/' + ONE_OF, ref.get().get(ONE_OF), root);
-            typeSet.addAll(Arrays.asList(types));
-        }
-
-        // validate no union types or extending
-        if (node.has(ALL_OF)
-                || node.has(ANY_OF)
-                || node.has(NOT)
-                || node.has(EXTENDS)
-                || node.has(DISALLOW)) {
-            throw new IllegalArgumentException(
-                    "Union types are such as '"
-                            + ALL_OF
-                            + "', '"
-                            + ANY_OF
-                            + "' etc. "
-                            + "and extending are not supported yet.");
-        }
-
-        // only a type (with null) is supported yet
-        final List<TypeInformation<?>> types = new ArrayList<>(typeSet);
-        if (types.size() == 0) {
-            throw new IllegalArgumentException("No type could be found in node:" + location);
-        } else if (types.size() > 2 || (types.size() == 2 && !types.contains(Types.VOID))) {
-            throw new IllegalArgumentException(
-                    "Union types with more than just a null type are not supported yet.");
-        }
-
-        // return the first non-void type or void
-        if (types.size() == 2 && types.get(0) == Types.VOID) {
-            return types.get(1);
-        } else {
-            return types.get(0);
-        }
-    }
-
-    private static TypeInformation<Row> convertObject(
-            String location, JsonNode node, JsonNode root) {
-        // validate properties
-        if (!node.has(PROPERTIES)) {
-            return Types.ROW();
-        }
-        if (!node.isObject()) {
-            throw new IllegalArgumentException(
-                    "Invalid '" + PROPERTIES + "' property for object type in node: " + location);
-        }
-        final JsonNode props = node.get(PROPERTIES);
-        final String[] names = new String[props.size()];
-        final TypeInformation<?>[] types = new TypeInformation[props.size()];
-
-        final Iterator<Map.Entry<String, JsonNode>> fieldIter = props.fields();
-        int i = 0;
-        while (fieldIter.hasNext()) {
-            final Map.Entry<String, JsonNode> subNode = fieldIter.next();
-
-            // set field name
-            names[i] = subNode.getKey();
-
-            // set type
-            types[i] = convertType(location + '/' + subNode.getKey(), subNode.getValue(), root);
-
-            i++;
-        }
-
-        // validate that object does not contain additional properties
-        if (node.has(ADDITIONAL_PROPERTIES)
-                && node.get(ADDITIONAL_PROPERTIES).isBoolean()
-                && node.get(ADDITIONAL_PROPERTIES).asBoolean()) {
-            throw new IllegalArgumentException(
-                    "An object must not allow additional properties in node: " + location);
-        }
-
-        return Types.ROW_NAMED(names, types);
-    }
-
-    private static TypeInformation<?> convertArray(String location, JsonNode node, JsonNode root) {
-        // validate items
-        if (!node.has(ITEMS)) {
-            throw new IllegalArgumentException(
-                    "Arrays must specify an '" + ITEMS + "' property in node: " + location);
-        }
-        final JsonNode items = node.get(ITEMS);
-
-        // list (translated to object array)
-        if (items.isObject()) {
-            final TypeInformation<?> elementType = convertType(location + '/' + ITEMS, items, root);
-            // result type might either be ObjectArrayTypeInfo or BasicArrayTypeInfo for Strings
-            return Types.OBJECT_ARRAY(elementType);
-        }
-        // tuple (translated to row)
-        else if (items.isArray()) {
-            final TypeInformation<?>[] types = convertTypes(location + '/' + ITEMS, items, root);
-
-            // validate that array does not contain additional items
-            if (node.has(ADDITIONAL_ITEMS)
-                    && node.get(ADDITIONAL_ITEMS).isBoolean()
-                    && node.get(ADDITIONAL_ITEMS).asBoolean()) {
-                throw new IllegalArgumentException(
-                        "An array tuple must not allow additional items in node: " + location);
-            }
-
-            return Types.ROW(types);
-        }
-        throw new IllegalArgumentException(
-                "Invalid type for '" + ITEMS + "' property in node: " + location);
-    }
-
-    private static TypeInformation<?> convertStringFormat(String location, JsonNode node) {
-        if (!node.isTextual()) {
-            throw new IllegalArgumentException(
-                    "Invalid '" + FORMAT + "' property in node: " + location);
-        }
-
-        switch (node.asText()) {
-            case FORMAT_DATE:
-                return Types.SQL_DATE;
-            case FORMAT_TIME:
-                return Types.SQL_TIME;
-            case FORMAT_DATE_TIME:
-                return Types.SQL_TIMESTAMP;
-            default:
-                return Types.STRING; // unlikely that we will support other formats in the future
-        }
-    }
-
-    private static TypeInformation<?> convertStringEncoding(String location, JsonNode node) {
-        if (!node.isTextual()) {
-            throw new IllegalArgumentException(
-                    "Invalid '" + CONTENT_ENCODING + "' property in node: " + location);
-        }
-
-        // "If the instance value is a string, this property defines that the string SHOULD
-        // be interpreted as binary data and decoded using the encoding named by this property."
-
-        switch (node.asText()) {
-            case CONTENT_ENCODING_BASE64:
-                return Types.PRIMITIVE_ARRAY(Types.BYTE);
-            default:
-                // we fail hard here:
-                // this gives us the chance to support more encodings in the future without problems
-                // of backwards compatibility
-                throw new IllegalArgumentException(
-                        "Invalid encoding '" + node.asText() + "' in node: " + location);
-        }
-    }
-
-    private static JsonNode resolveReference(String ref, JsonNode origin, JsonNode root) {
-        if (!ref.startsWith("#")) {
-            throw new IllegalArgumentException(
-                    "Only JSON schemes with simple references "
-                            + "(one indirection in the same document) are supported yet. But was: "
-                            + ref);
-        }
-        final String path = ref.substring(1);
-        final JsonNode foundNode = root.at(path);
-        if (foundNode.isMissingNode()) {
-            throw new IllegalArgumentException("Could not find reference: " + ref);
-        }
-        // prevent obvious cyclic references
-        if (foundNode == origin) {
-            throw new IllegalArgumentException("Cyclic references are not supported:" + ref);
-        }
-        return foundNode;
-    }
-
-    private static TypeInformation<?>[] convertTypes(
-            String location, JsonNode arrayNode, JsonNode root) {
-        final TypeInformation<?>[] types = new TypeInformation[arrayNode.size()];
-        final Iterator<JsonNode> elements = arrayNode.elements();
-        int i = 0;
-        while (elements.hasNext()) {
-            final TypeInformation<?> elementType =
-                    convertType(location + '[' + i + ']', elements.next(), root);
-            types[i] = elementType;
-            i += 1;
-        }
-        return types;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java
deleted file mode 100644
index f185d21..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.WrappingRuntimeException;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeFactory;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
-import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT;
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Serialization schema that serializes an object of Flink types into a JSON bytes.
- *
- * <p>Serializes the input Flink object into a JSON string and converts it into <code>byte[]</code>.
- *
- * <p>Result <code>byte[]</code> messages can be deserialized using {@link
- * JsonRowDeserializationSchema}.
- *
- * @deprecated The format was developed for the Table API users and will not be maintained for
- *     DataStream API users anymore. Either use Table API or switch to Data Stream, defining your
- *     own {@link SerializationSchema}.
- */
-@PublicEvolving
-@Deprecated
-public class JsonRowSerializationSchema implements SerializationSchema<Row> {
-
-    private static final long serialVersionUID = -2885556750743978636L;
-
-    /** Type information describing the input type. */
-    private final RowTypeInfo typeInfo;
-
-    /** Object mapper that is used to create output JSON objects. */
-    private transient ObjectMapper mapper;
-
-    private final SerializationRuntimeConverter runtimeConverter;
-
-    /** Reusable object node. */
-    private transient ObjectNode node;
-
-    private JsonRowSerializationSchema(TypeInformation<Row> typeInfo) {
-        Preconditions.checkNotNull(typeInfo, "Type information");
-        Preconditions.checkArgument(
-                typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
-        this.typeInfo = (RowTypeInfo) typeInfo;
-        this.runtimeConverter = createConverter(typeInfo);
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        mapper = JacksonMapperFactory.createObjectMapper();
-    }
-
-    /** Builder for {@link JsonRowSerializationSchema}. */
-    @PublicEvolving
-    public static class Builder {
-
-        private RowTypeInfo typeInfo;
-
-        private Builder() {
-            // private constructor
-        }
-
-        /**
-         * Creates a JSON serialization schema for the given type information.
-         *
-         * @param typeInfo Type information describing the result type. The field names of {@link
-         *     Row} are used to parse the JSON properties.
-         * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead.
-         */
-        @Deprecated
-        public Builder(TypeInformation<Row> typeInfo) {
-            checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
-            this.typeInfo = (RowTypeInfo) typeInfo;
-        }
-
-        /**
-         * Creates a JSON serialization schema for the given JSON schema.
-         *
-         * @param jsonSchema JSON schema describing the result type
-         * @see <a href="http://json-schema.org/">http://json-schema.org/</a>
-         * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead.
-         */
-        @Deprecated
-        public Builder(String jsonSchema) {
-            this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)));
-        }
-
-        /**
-         * Sets type information for JSON serialization schema.
-         *
-         * @param typeInfo Type information describing the result type. The field names of {@link
-         *     Row} are used to parse the JSON properties.
-         */
-        public Builder withTypeInfo(TypeInformation<Row> typeInfo) {
-            checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
-            this.typeInfo = (RowTypeInfo) typeInfo;
-            return this;
-        }
-
-        /**
-         * Finalizes the configuration and checks validity.
-         *
-         * @return Configured {@link JsonRowSerializationSchema}
-         */
-        public JsonRowSerializationSchema build() {
-            checkArgument(typeInfo != null, "typeInfo should be set.");
-            return new JsonRowSerializationSchema(typeInfo);
-        }
-    }
-
-    /** Creates a builder for {@link JsonRowSerializationSchema.Builder}. */
-    public static Builder builder() {
-        return new Builder();
-    }
-
-    @Override
-    public byte[] serialize(Row row) {
-        if (node == null) {
-            node = mapper.createObjectNode();
-        }
-
-        try {
-            runtimeConverter.convert(mapper, node, row);
-            return mapper.writeValueAsBytes(node);
-        } catch (Throwable t) {
-            throw new RuntimeException(
-                    "Could not serialize row '"
-                            + row
-                            + "'. "
-                            + "Make sure that the schema matches the input.",
-                    t);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o;
-        return Objects.equals(typeInfo, that.typeInfo);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(typeInfo);
-    }
-
-    /*
-    Runtime converters
-    */
-
-    /** Runtime converter that maps between Java objects and corresponding {@link JsonNode}s. */
-    @FunctionalInterface
-    private interface SerializationRuntimeConverter extends Serializable {
-        JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object object);
-    }
-
-    private SerializationRuntimeConverter createConverter(TypeInformation<?> typeInfo) {
-        SerializationRuntimeConverter baseConverter =
-                createConverterForSimpleType(typeInfo)
-                        .orElseGet(
-                                () ->
-                                        createContainerConverter(typeInfo)
-                                                .orElseGet(this::createFallbackConverter));
-        return wrapIntoNullableConverter(baseConverter);
-    }
-
-    private SerializationRuntimeConverter wrapIntoNullableConverter(
-            SerializationRuntimeConverter converter) {
-        return (mapper, reuse, object) -> {
-            if (object == null) {
-                return mapper.getNodeFactory().nullNode();
-            }
-
-            return converter.convert(mapper, reuse, object);
-        };
-    }
-
-    private Optional<SerializationRuntimeConverter> createContainerConverter(
-            TypeInformation<?> typeInfo) {
-        if (typeInfo instanceof RowTypeInfo) {
-            return Optional.of(createRowConverter((RowTypeInfo) typeInfo));
-        } else if (typeInfo instanceof ObjectArrayTypeInfo) {
-            return Optional.of(
-                    createObjectArrayConverter(
-                            ((ObjectArrayTypeInfo) typeInfo).getComponentInfo()));
-        } else if (typeInfo instanceof BasicArrayTypeInfo) {
-            return Optional.of(
-                    createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo()));
-        } else if (isPrimitiveByteArray(typeInfo)) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().binaryNode((byte[]) object));
-        } else {
-            return Optional.empty();
-        }
-    }
-
-    private boolean isPrimitiveByteArray(TypeInformation<?> typeInfo) {
-        return typeInfo instanceof PrimitiveArrayTypeInfo
-                && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE;
-    }
-
-    private SerializationRuntimeConverter createObjectArrayConverter(
-            TypeInformation elementTypeInfo) {
-        SerializationRuntimeConverter elementConverter = createConverter(elementTypeInfo);
-        return assembleArrayConverter(elementConverter);
-    }
-
-    private SerializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) {
-        List<SerializationRuntimeConverter> fieldConverters =
-                Arrays.stream(typeInfo.getFieldTypes())
-                        .map(this::createConverter)
-                        .collect(Collectors.toList());
-
-        return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters);
-    }
-
-    private SerializationRuntimeConverter createFallbackConverter() {
-        return (mapper, reuse, object) -> {
-            // for types that were specified without JSON schema
-            // e.g. POJOs
-            try {
-                return mapper.valueToTree(object);
-            } catch (IllegalArgumentException e) {
-                throw new WrappingRuntimeException(
-                        format("Could not convert object: %s", object), e);
-            }
-        };
-    }
-
-    private Optional<SerializationRuntimeConverter> createConverterForSimpleType(
-            TypeInformation<?> simpleTypeInfo) {
-        if (simpleTypeInfo == Types.VOID) {
-            return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().nullNode());
-        } else if (simpleTypeInfo == Types.BOOLEAN) {
-            return Optional.of(
-                    (mapper, reuse, object) ->
-                            mapper.getNodeFactory().booleanNode((Boolean) object));
-        } else if (simpleTypeInfo == Types.STRING) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().textNode((String) object));
-        } else if (simpleTypeInfo == Types.INT) {
-            return Optional.of(
-                    (mapper, reuse, object) ->
-                            mapper.getNodeFactory().numberNode((Integer) object));
-        } else if (simpleTypeInfo == Types.LONG) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Long) object));
-        } else if (simpleTypeInfo == Types.DOUBLE) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Double) object));
-        } else if (simpleTypeInfo == Types.FLOAT) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Float) object));
-        } else if (simpleTypeInfo == Types.SHORT) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Short) object));
-        } else if (simpleTypeInfo == Types.BYTE) {
-            return Optional.of(
-                    (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Byte) object));
-        } else if (simpleTypeInfo == Types.BIG_DEC) {
-            return Optional.of(createBigDecimalConverter());
-        } else if (simpleTypeInfo == Types.BIG_INT) {
-            return Optional.of(createBigIntegerConverter());
-        } else if (simpleTypeInfo == Types.SQL_DATE) {
-            return Optional.of(this::convertDate);
-        } else if (simpleTypeInfo == Types.SQL_TIME) {
-            return Optional.of(this::convertTime);
-        } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) {
-            return Optional.of(this::convertTimestamp);
-        } else if (simpleTypeInfo == Types.LOCAL_DATE) {
-            return Optional.of(this::convertLocalDate);
-        } else if (simpleTypeInfo == Types.LOCAL_TIME) {
-            return Optional.of(this::convertLocalTime);
-        } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) {
-            return Optional.of(this::convertLocalDateTime);
-        } else {
-            return Optional.empty();
-        }
-    }
-
-    private JsonNode convertLocalDate(ObjectMapper mapper, JsonNode reuse, Object object) {
-        return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format((LocalDate) object));
-    }
-
-    private JsonNode convertDate(ObjectMapper mapper, JsonNode reuse, Object object) {
-        Date date = (Date) object;
-        return convertLocalDate(mapper, reuse, date.toLocalDate());
-    }
-
-    private JsonNode convertLocalDateTime(ObjectMapper mapper, JsonNode reuse, Object object) {
-        return mapper.getNodeFactory()
-                .textNode(RFC3339_TIMESTAMP_FORMAT.format((LocalDateTime) object));
-    }
-
-    private JsonNode convertTimestamp(ObjectMapper mapper, JsonNode reuse, Object object) {
-        Timestamp timestamp = (Timestamp) object;
-        return convertLocalDateTime(mapper, reuse, timestamp.toLocalDateTime());
-    }
-
-    private JsonNode convertLocalTime(ObjectMapper mapper, JsonNode reuse, Object object) {
-        JsonNodeFactory nodeFactory = mapper.getNodeFactory();
-        return nodeFactory.textNode(RFC3339_TIME_FORMAT.format((LocalTime) object));
-    }
-
-    private JsonNode convertTime(ObjectMapper mapper, JsonNode reuse, Object object) {
-        final Time time = (Time) object;
-        return convertLocalTime(mapper, reuse, time.toLocalTime());
-    }
-
-    private SerializationRuntimeConverter createBigDecimalConverter() {
-        return (mapper, reuse, object) -> {
-            // convert decimal if necessary
-            JsonNodeFactory nodeFactory = mapper.getNodeFactory();
-            if (object instanceof BigDecimal) {
-                return nodeFactory.numberNode((BigDecimal) object);
-            }
-            return nodeFactory.numberNode(BigDecimal.valueOf(((Number) object).doubleValue()));
-        };
-    }
-
-    private SerializationRuntimeConverter createBigIntegerConverter() {
-        return (mapper, reuse, object) -> {
-            // convert decimal if necessary
-            JsonNodeFactory nodeFactory = mapper.getNodeFactory();
-            if (object instanceof BigInteger) {
-                return nodeFactory.numberNode((BigInteger) object);
-            }
-            return nodeFactory.numberNode(BigInteger.valueOf(((Number) object).longValue()));
-        };
-    }
-
-    private SerializationRuntimeConverter assembleRowConverter(
-            String[] fieldNames, List<SerializationRuntimeConverter> fieldConverters) {
-        return (mapper, reuse, object) -> {
-            ObjectNode node;
-
-            // reuse could be a NullNode if last record is null.
-            if (reuse == null || reuse.isNull()) {
-                node = mapper.createObjectNode();
-            } else {
-                node = (ObjectNode) reuse;
-            }
-
-            Row row = (Row) object;
-
-            for (int i = 0; i < fieldNames.length; i++) {
-                String fieldName = fieldNames[i];
-                node.set(
-                        fieldName,
-                        fieldConverters
-                                .get(i)
-                                .convert(mapper, node.get(fieldNames[i]), row.getField(i)));
-            }
-
-            return node;
-        };
-    }
-
-    private SerializationRuntimeConverter assembleArrayConverter(
-            SerializationRuntimeConverter elementConverter) {
-        return (mapper, reuse, object) -> {
-            ArrayNode node;
-
-            // reuse could be a NullNode if last record is null.
-            if (reuse == null || reuse.isNull()) {
-                node = mapper.createArrayNode();
-            } else {
-                node = (ArrayNode) reuse;
-                node.removeAll();
-            }
-
-            Object[] array = (Object[]) object;
-
-            for (Object element : array) {
-                node.add(elementConverter.convert(mapper, null, element));
-            }
-
-            return node;
-        };
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java
deleted file mode 100644
index c029fa1..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.util.function.SerializableSupplier;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-
-/** SerializationSchema that serializes an object to a JSON String. */
-@PublicEvolving
-public class JsonSerializationSchema<T> implements SerializationSchema<T> {
-
-    private static final long serialVersionUID = 1L;
-
-    private final SerializableSupplier<ObjectMapper> mapperFactory;
-
-    protected transient ObjectMapper mapper;
-
-    public JsonSerializationSchema() {
-        this(() -> new ObjectMapper());
-    }
-
-    public JsonSerializationSchema(SerializableSupplier<ObjectMapper> mapperFactory) {
-        this.mapperFactory = mapperFactory;
-    }
-
-    @Override
-    public void open(InitializationContext context) {
-        mapper = mapperFactory.get();
-    }
-
-    @Override
-    public byte[] serialize(T element) {
-        try {
-            return mapper.writeValueAsBytes(element);
-        } catch (JsonProcessingException e) {
-            throw new RuntimeException(
-                    String.format("Could not serialize value '%s'.", element), e);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java
deleted file mode 100644
index c89b5f5..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java
+++ /dev/null
@@ -1,402 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.data.DecimalData;
-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 org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.logical.ArrayType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.IntType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeFamily;
-import org.apache.flink.table.types.logical.MapType;
-import org.apache.flink.table.types.logical.MultisetType;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.Array;
-import java.math.BigDecimal;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.ZoneOffset;
-import java.time.temporal.TemporalAccessor;
-import java.time.temporal.TemporalQueries;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
-import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT;
-
-/** Tool class used to convert from {@link JsonNode} to {@link RowData}. * */
-@Internal
-public class JsonToRowDataConverters implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    /** Flag indicating whether to fail if a field is missing. */
-    private final boolean failOnMissingField;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    /** Timestamp format specification which is used to parse timestamp. */
-    private final TimestampFormat timestampFormat;
-
-    public JsonToRowDataConverters(
-            boolean failOnMissingField,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        this.failOnMissingField = failOnMissingField;
-        this.ignoreParseErrors = ignoreParseErrors;
-        this.timestampFormat = timestampFormat;
-    }
-
-    /**
-     * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL internal
-     * data structures.
-     */
-    @FunctionalInterface
-    public interface JsonToRowDataConverter extends Serializable {
-        Object convert(JsonNode jsonNode);
-    }
-
-    /** Creates a runtime converter which is null safe. */
-    public JsonToRowDataConverter createConverter(LogicalType type) {
-        return wrapIntoNullableConverter(createNotNullConverter(type));
-    }
-
-    /** Creates a runtime converter which assuming input object is not null. */
-    private JsonToRowDataConverter createNotNullConverter(LogicalType type) {
-        switch (type.getTypeRoot()) {
-            case NULL:
-                return jsonNode -> null;
-            case BOOLEAN:
-                return this::convertToBoolean;
-            case TINYINT:
-                return jsonNode -> Byte.parseByte(jsonNode.asText().trim());
-            case SMALLINT:
-                return jsonNode -> Short.parseShort(jsonNode.asText().trim());
-            case INTEGER:
-            case INTERVAL_YEAR_MONTH:
-                return this::convertToInt;
-            case BIGINT:
-            case INTERVAL_DAY_TIME:
-                return this::convertToLong;
-            case DATE:
-                return this::convertToDate;
-            case TIME_WITHOUT_TIME_ZONE:
-                return this::convertToTime;
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-                return this::convertToTimestamp;
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-                return this::convertToTimestampWithLocalZone;
-            case FLOAT:
-                return this::convertToFloat;
-            case DOUBLE:
-                return this::convertToDouble;
-            case CHAR:
-            case VARCHAR:
-                return this::convertToString;
-            case BINARY:
-            case VARBINARY:
-                return this::convertToBytes;
-            case DECIMAL:
-                return createDecimalConverter((DecimalType) type);
-            case ARRAY:
-                return createArrayConverter((ArrayType) type);
-            case MAP:
-                MapType mapType = (MapType) type;
-                return createMapConverter(
-                        mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType());
-            case MULTISET:
-                MultisetType multisetType = (MultisetType) type;
-                return createMapConverter(
-                        multisetType.asSummaryString(),
-                        multisetType.getElementType(),
-                        new IntType());
-            case ROW:
-                return createRowConverter((RowType) type);
-            case RAW:
-            default:
-                throw new UnsupportedOperationException("Unsupported type: " + type);
-        }
-    }
-
-    private boolean convertToBoolean(JsonNode jsonNode) {
-        if (jsonNode.isBoolean()) {
-            // avoid redundant toString and parseBoolean, for better performance
-            return jsonNode.asBoolean();
-        } else {
-            return Boolean.parseBoolean(jsonNode.asText().trim());
-        }
-    }
-
-    private int convertToInt(JsonNode jsonNode) {
-        if (jsonNode.canConvertToInt()) {
-            // avoid redundant toString and parseInt, for better performance
-            return jsonNode.asInt();
-        } else {
-            return Integer.parseInt(jsonNode.asText().trim());
-        }
-    }
-
-    private long convertToLong(JsonNode jsonNode) {
-        if (jsonNode.canConvertToLong()) {
-            // avoid redundant toString and parseLong, for better performance
-            return jsonNode.asLong();
-        } else {
-            return Long.parseLong(jsonNode.asText().trim());
-        }
-    }
-
-    private double convertToDouble(JsonNode jsonNode) {
-        if (jsonNode.isDouble()) {
-            // avoid redundant toString and parseDouble, for better performance
-            return jsonNode.asDouble();
-        } else {
-            return Double.parseDouble(jsonNode.asText().trim());
-        }
-    }
-
-    private float convertToFloat(JsonNode jsonNode) {
-        if (jsonNode.isDouble()) {
-            // avoid redundant toString and parseDouble, for better performance
-            return (float) jsonNode.asDouble();
-        } else {
-            return Float.parseFloat(jsonNode.asText().trim());
-        }
-    }
-
-    private int convertToDate(JsonNode jsonNode) {
-        LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate());
-        return (int) date.toEpochDay();
-    }
-
-    private int convertToTime(JsonNode jsonNode) {
-        TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText());
-        LocalTime localTime = parsedTime.query(TemporalQueries.localTime());
-
-        // get number of milliseconds of the day
-        return localTime.toSecondOfDay() * 1000;
-    }
-
-    private TimestampData convertToTimestamp(JsonNode jsonNode) {
-        TemporalAccessor parsedTimestamp;
-        switch (timestampFormat) {
-            case SQL:
-                parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText());
-                break;
-            case ISO_8601:
-                parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText());
-                break;
-            default:
-                throw new TableException(
-                        String.format(
-                                "Unsupported timestamp format '%s'. Validator should have checked that.",
-                                timestampFormat));
-        }
-        LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime());
-        LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate());
-
-        return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime));
-    }
-
-    private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode) {
-        TemporalAccessor parsedTimestampWithLocalZone;
-        switch (timestampFormat) {
-            case SQL:
-                parsedTimestampWithLocalZone =
-                        SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText());
-                break;
-            case ISO_8601:
-                parsedTimestampWithLocalZone =
-                        ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText());
-                break;
-            default:
-                throw new TableException(
-                        String.format(
-                                "Unsupported timestamp format '%s'. Validator should have checked that.",
-                                timestampFormat));
-        }
-        LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime());
-        LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate());
-
-        return TimestampData.fromInstant(
-                LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC));
-    }
-
-    private StringData convertToString(JsonNode jsonNode) {
-        if (jsonNode.isContainerNode()) {
-            return StringData.fromString(jsonNode.toString());
-        } else {
-            return StringData.fromString(jsonNode.asText());
-        }
-    }
-
-    private byte[] convertToBytes(JsonNode jsonNode) {
-        try {
-            return jsonNode.binaryValue();
-        } catch (IOException e) {
-            throw new JsonParseException("Unable to deserialize byte array.", e);
-        }
-    }
-
-    private JsonToRowDataConverter createDecimalConverter(DecimalType decimalType) {
-        final int precision = decimalType.getPrecision();
-        final int scale = decimalType.getScale();
-        return jsonNode -> {
-            BigDecimal bigDecimal;
-            if (jsonNode.isBigDecimal()) {
-                bigDecimal = jsonNode.decimalValue();
-            } else {
-                bigDecimal = new BigDecimal(jsonNode.asText());
-            }
-            return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
-        };
-    }
-
-    private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
-        JsonToRowDataConverter elementConverter = createConverter(arrayType.getElementType());
-        final Class<?> elementClass =
-                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
-        return jsonNode -> {
-            final ArrayNode node = (ArrayNode) jsonNode;
-            final Object[] array = (Object[]) Array.newInstance(elementClass, node.size());
-            for (int i = 0; i < node.size(); i++) {
-                final JsonNode innerNode = node.get(i);
-                array[i] = elementConverter.convert(innerNode);
-            }
-            return new GenericArrayData(array);
-        };
-    }
-
-    private JsonToRowDataConverter createMapConverter(
-            String typeSummary, LogicalType keyType, LogicalType valueType) {
-        if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) {
-            throw new UnsupportedOperationException(
-                    "JSON format doesn't support non-string as key type of map. "
-                            + "The type is: "
-                            + typeSummary);
-        }
-        final JsonToRowDataConverter keyConverter = createConverter(keyType);
-        final JsonToRowDataConverter valueConverter = createConverter(valueType);
-
-        return jsonNode -> {
-            Iterator<Map.Entry<String, JsonNode>> fields = jsonNode.fields();
-            Map<Object, Object> result = new HashMap<>();
-            while (fields.hasNext()) {
-                Map.Entry<String, JsonNode> entry = fields.next();
-                Object key = keyConverter.convert(TextNode.valueOf(entry.getKey()));
-                Object value = valueConverter.convert(entry.getValue());
-                result.put(key, value);
-            }
-            return new GenericMapData(result);
-        };
-    }
-
-    public JsonToRowDataConverter createRowConverter(RowType rowType) {
-        final JsonToRowDataConverter[] fieldConverters =
-                rowType.getFields().stream()
-                        .map(RowType.RowField::getType)
-                        .map(this::createConverter)
-                        .toArray(JsonToRowDataConverter[]::new);
-        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
-
-        return jsonNode -> {
-            ObjectNode node = (ObjectNode) jsonNode;
-            int arity = fieldNames.length;
-            GenericRowData row = new GenericRowData(arity);
-            for (int i = 0; i < arity; i++) {
-                String fieldName = fieldNames[i];
-                JsonNode field = node.get(fieldName);
-                try {
-                    Object convertedField = convertField(fieldConverters[i], fieldName, field);
-                    row.setField(i, convertedField);
-                } catch (Throwable t) {
-                    throw new JsonParseException(
-                            String.format("Fail to deserialize at field: %s.", fieldName), t);
-                }
-            }
-            return row;
-        };
-    }
-
-    private Object convertField(
-            JsonToRowDataConverter fieldConverter, String fieldName, JsonNode field) {
-        if (field == null) {
-            if (failOnMissingField) {
-                throw new JsonParseException("Could not find field with name '" + fieldName + "'.");
-            } else {
-                return null;
-            }
-        } else {
-            return fieldConverter.convert(field);
-        }
-    }
-
-    private JsonToRowDataConverter wrapIntoNullableConverter(JsonToRowDataConverter converter) {
-        return jsonNode -> {
-            if (jsonNode == null || jsonNode.isNull() || jsonNode.isMissingNode()) {
-                return null;
-            }
-            try {
-                return converter.convert(jsonNode);
-            } catch (Throwable t) {
-                if (!ignoreParseErrors) {
-                    throw t;
-                }
-                return null;
-            }
-        };
-    }
-
-    /** Exception which refers to parse errors in converters. */
-    private static final class JsonParseException extends RuntimeException {
-        private static final long serialVersionUID = 1L;
-
-        public JsonParseException(String message) {
-            super(message);
-        }
-
-        public JsonParseException(String message, Throwable cause) {
-            super(message, cause);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java
deleted file mode 100644
index 2a1cd07..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.MapData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.logical.ArrayType;
-import org.apache.flink.table.types.logical.IntType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeFamily;
-import org.apache.flink.table.types.logical.MapType;
-import org.apache.flink.table.types.logical.MultisetType;
-import org.apache.flink.table.types.logical.RowType;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.time.LocalDate;
-import java.time.LocalTime;
-import java.time.ZoneOffset;
-import java.util.Arrays;
-
-import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
-import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT;
-import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT;
-
-/** Tool class used to convert from {@link RowData} to {@link JsonNode}. * */
-@Internal
-public class RowDataToJsonConverters implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    /** Timestamp format specification which is used to parse timestamp. */
-    private final TimestampFormat timestampFormat;
-
-    /** The handling mode when serializing null keys for map data. */
-    private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode;
-
-    /** The string literal when handling mode for map null key LITERAL. is */
-    private final String mapNullKeyLiteral;
-
-    public RowDataToJsonConverters(
-            TimestampFormat timestampFormat,
-            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
-            String mapNullKeyLiteral) {
-        this.timestampFormat = timestampFormat;
-        this.mapNullKeyMode = mapNullKeyMode;
-        this.mapNullKeyLiteral = mapNullKeyLiteral;
-    }
-
-    /**
-     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
-     * corresponding {@link JsonNode}s.
-     */
-    public interface RowDataToJsonConverter extends Serializable {
-        JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value);
-    }
-
-    /** Creates a runtime converter which is null safe. */
-    public RowDataToJsonConverter createConverter(LogicalType type) {
-        return wrapIntoNullableConverter(createNotNullConverter(type));
-    }
-
-    /** Creates a runtime converter which assuming input object is not null. */
-    private RowDataToJsonConverter createNotNullConverter(LogicalType type) {
-        switch (type.getTypeRoot()) {
-            case NULL:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode();
-            case BOOLEAN:
-                return (mapper, reuse, value) ->
-                        mapper.getNodeFactory().booleanNode((boolean) value);
-            case TINYINT:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value);
-            case SMALLINT:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value);
-            case INTEGER:
-            case INTERVAL_YEAR_MONTH:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value);
-            case BIGINT:
-            case INTERVAL_DAY_TIME:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value);
-            case FLOAT:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value);
-            case DOUBLE:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value);
-            case CHAR:
-            case VARCHAR:
-                // value is BinaryString
-                return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString());
-            case BINARY:
-            case VARBINARY:
-                return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value);
-            case DATE:
-                return createDateConverter();
-            case TIME_WITHOUT_TIME_ZONE:
-                return createTimeConverter();
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-                return createTimestampConverter();
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-                return createTimestampWithLocalZone();
-            case DECIMAL:
-                return createDecimalConverter();
-            case ARRAY:
-                return createArrayConverter((ArrayType) type);
-            case MAP:
-                MapType mapType = (MapType) type;
-                return createMapConverter(
-                        mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType());
-            case MULTISET:
-                MultisetType multisetType = (MultisetType) type;
-                return createMapConverter(
-                        multisetType.asSummaryString(),
-                        multisetType.getElementType(),
-                        new IntType());
-            case ROW:
-                return createRowConverter((RowType) type);
-            case RAW:
-            default:
-                throw new UnsupportedOperationException("Not support to parse type: " + type);
-        }
-    }
-
-    private RowDataToJsonConverter createDecimalConverter() {
-        return (mapper, reuse, value) -> {
-            BigDecimal bd = ((DecimalData) value).toBigDecimal();
-            return mapper.getNodeFactory().numberNode(bd);
-        };
-    }
-
-    private RowDataToJsonConverter createDateConverter() {
-        return (mapper, reuse, value) -> {
-            int days = (int) value;
-            LocalDate date = LocalDate.ofEpochDay(days);
-            return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date));
-        };
-    }
-
-    private RowDataToJsonConverter createTimeConverter() {
-        return (mapper, reuse, value) -> {
-            int millisecond = (int) value;
-            LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L);
-            return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time));
-        };
-    }
-
-    private RowDataToJsonConverter createTimestampConverter() {
-        switch (timestampFormat) {
-            case ISO_8601:
-                return (mapper, reuse, value) -> {
-                    TimestampData timestamp = (TimestampData) value;
-                    return mapper.getNodeFactory()
-                            .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime()));
-                };
-            case SQL:
-                return (mapper, reuse, value) -> {
-                    TimestampData timestamp = (TimestampData) value;
-                    return mapper.getNodeFactory()
-                            .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime()));
-                };
-            default:
-                throw new TableException(
-                        "Unsupported timestamp format. Validator should have checked that.");
-        }
-    }
-
-    private RowDataToJsonConverter createTimestampWithLocalZone() {
-        switch (timestampFormat) {
-            case ISO_8601:
-                return (mapper, reuse, value) -> {
-                    TimestampData timestampWithLocalZone = (TimestampData) value;
-                    return mapper.getNodeFactory()
-                            .textNode(
-                                    ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(
-                                            timestampWithLocalZone
-                                                    .toInstant()
-                                                    .atOffset(ZoneOffset.UTC)));
-                };
-            case SQL:
-                return (mapper, reuse, value) -> {
-                    TimestampData timestampWithLocalZone = (TimestampData) value;
-                    return mapper.getNodeFactory()
-                            .textNode(
-                                    SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(
-                                            timestampWithLocalZone
-                                                    .toInstant()
-                                                    .atOffset(ZoneOffset.UTC)));
-                };
-            default:
-                throw new TableException(
-                        "Unsupported timestamp format. Validator should have checked that.");
-        }
-    }
-
-    private RowDataToJsonConverter createArrayConverter(ArrayType type) {
-        final LogicalType elementType = type.getElementType();
-        final RowDataToJsonConverter elementConverter = createConverter(elementType);
-        final ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(elementType);
-        return (mapper, reuse, value) -> {
-            ArrayNode node;
-
-            // reuse could be a NullNode if last record is null.
-            if (reuse == null || reuse.isNull()) {
-                node = mapper.createArrayNode();
-            } else {
-                node = (ArrayNode) reuse;
-                node.removeAll();
-            }
-
-            ArrayData array = (ArrayData) value;
-            int numElements = array.size();
-            for (int i = 0; i < numElements; i++) {
-                Object element = elementGetter.getElementOrNull(array, i);
-                node.add(elementConverter.convert(mapper, null, element));
-            }
-
-            return node;
-        };
-    }
-
-    private RowDataToJsonConverter createMapConverter(
-            String typeSummary, LogicalType keyType, LogicalType valueType) {
-        if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) {
-            throw new UnsupportedOperationException(
-                    "JSON format doesn't support non-string as key type of map. "
-                            + "The type is: "
-                            + typeSummary);
-        }
-        final RowDataToJsonConverter valueConverter = createConverter(valueType);
-        final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType);
-        return (mapper, reuse, object) -> {
-            ObjectNode node;
-            // reuse could be a NullNode if last record is null.
-            if (reuse == null || reuse.isNull()) {
-                node = mapper.createObjectNode();
-            } else {
-                node = (ObjectNode) reuse;
-                node.removeAll();
-            }
-
-            MapData map = (MapData) object;
-            ArrayData keyArray = map.keyArray();
-            ArrayData valueArray = map.valueArray();
-            int numElements = map.size();
-            for (int i = 0; i < numElements; i++) {
-                String fieldName = null;
-                if (keyArray.isNullAt(i)) {
-                    // when map key is null
-                    switch (mapNullKeyMode) {
-                        case LITERAL:
-                            fieldName = mapNullKeyLiteral;
-                            break;
-                        case DROP:
-                            continue;
-                        case FAIL:
-                            throw new RuntimeException(
-                                    String.format(
-                                            "JSON format doesn't support to serialize map data with null keys. "
-                                                    + "You can drop null key entries or encode null in literals by specifying %s option.",
-                                            JsonFormatOptions.MAP_NULL_KEY_MODE.key()));
-                        default:
-                            throw new RuntimeException(
-                                    "Unsupported map null key mode. Validator should have checked that.");
-                    }
-                } else {
-                    fieldName = keyArray.getString(i).toString();
-                }
-
-                Object value = valueGetter.getElementOrNull(valueArray, i);
-                node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value));
-            }
-
-            return node;
-        };
-    }
-
-    private RowDataToJsonConverter createRowConverter(RowType type) {
-        final String[] fieldNames = type.getFieldNames().toArray(new String[0]);
-        final LogicalType[] fieldTypes =
-                type.getFields().stream()
-                        .map(RowType.RowField::getType)
-                        .toArray(LogicalType[]::new);
-        final RowDataToJsonConverter[] fieldConverters =
-                Arrays.stream(fieldTypes)
-                        .map(this::createConverter)
-                        .toArray(RowDataToJsonConverter[]::new);
-        final int fieldCount = type.getFieldCount();
-        final RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldTypes.length];
-        for (int i = 0; i < fieldCount; i++) {
-            fieldGetters[i] = RowData.createFieldGetter(fieldTypes[i], i);
-        }
-
-        return (mapper, reuse, value) -> {
-            ObjectNode node;
-            // reuse could be a NullNode if last record is null.
-            if (reuse == null || reuse.isNull()) {
-                node = mapper.createObjectNode();
-            } else {
-                node = (ObjectNode) reuse;
-            }
-            RowData row = (RowData) value;
-            for (int i = 0; i < fieldCount; i++) {
-                String fieldName = fieldNames[i];
-                try {
-                    Object field = fieldGetters[i].getFieldOrNull(row);
-                    node.set(
-                            fieldName,
-                            fieldConverters[i].convert(mapper, node.get(fieldName), field));
-                } catch (Throwable t) {
-                    throw new RuntimeException(
-                            String.format("Fail to serialize at field: %s.", fieldName), t);
-                }
-            }
-            return node;
-        };
-    }
-
-    private RowDataToJsonConverter wrapIntoNullableConverter(RowDataToJsonConverter converter) {
-        return (mapper, reuse, object) -> {
-            if (object == null) {
-                return mapper.getNodeFactory().nullNode();
-            }
-
-            return converter.convert(mapper, reuse, object);
-        };
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java
deleted file mode 100644
index 4c981d5..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * 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.common.TimestampFormat;
-import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
-import org.apache.flink.types.RowKind;
-
-import javax.annotation.Nullable;
-
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/** {@link DecodingFormat} for Canal using JSON encoding. */
-public class CanalJsonDecodingFormat
-        implements ProjectableDecodingFormat<DeserializationSchema<RowData>> {
-
-    // --------------------------------------------------------------------------------------------
-    // Mutable attributes
-    // --------------------------------------------------------------------------------------------
-
-    private List<String> metadataKeys;
-
-    // --------------------------------------------------------------------------------------------
-    // Canal-specific attributes
-    // --------------------------------------------------------------------------------------------
-
-    private final @Nullable String database;
-
-    private final @Nullable String table;
-
-    private final boolean ignoreParseErrors;
-
-    private final TimestampFormat timestampFormat;
-
-    public CanalJsonDecodingFormat(
-            String database,
-            String table,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        this.database = database;
-        this.table = table;
-        this.ignoreParseErrors = ignoreParseErrors;
-        this.timestampFormat = timestampFormat;
-        this.metadataKeys = Collections.emptyList();
-    }
-
-    @Override
-    public DeserializationSchema<RowData> createRuntimeDecoder(
-            DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) {
-        physicalDataType = Projection.of(projections).project(physicalDataType);
-        final List<ReadableMetadata> readableMetadata =
-                metadataKeys.stream()
-                        .map(
-                                k ->
-                                        Stream.of(ReadableMetadata.values())
-                                                .filter(rm -> rm.key.equals(k))
-                                                .findFirst()
-                                                .orElseThrow(IllegalStateException::new))
-                        .collect(Collectors.toList());
-        final List<DataTypes.Field> metadataFields =
-                readableMetadata.stream()
-                        .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                        .collect(Collectors.toList());
-        final DataType producedDataType =
-                DataTypeUtils.appendRowFields(physicalDataType, metadataFields);
-        final TypeInformation<RowData> producedTypeInfo =
-                context.createTypeInformation(producedDataType);
-        return CanalJsonDeserializationSchema.builder(
-                        physicalDataType, readableMetadata, producedTypeInfo)
-                .setDatabase(database)
-                .setTable(table)
-                .setIgnoreParseErrors(ignoreParseErrors)
-                .setTimestampFormat(timestampFormat)
-                .build();
-    }
-
-    @Override
-    public Map<String, DataType> listReadableMetadata() {
-        final Map<String, DataType> metadataMap = new LinkedHashMap<>();
-        Stream.of(ReadableMetadata.values())
-                .forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
-        return metadataMap;
-    }
-
-    @Override
-    public void applyReadableMetadata(List<String> metadataKeys) {
-        this.metadataKeys = metadataKeys;
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode() {
-        return ChangelogMode.newBuilder()
-                .addContainedKind(RowKind.INSERT)
-                .addContainedKind(RowKind.UPDATE_BEFORE)
-                .addContainedKind(RowKind.UPDATE_AFTER)
-                .addContainedKind(RowKind.DELETE)
-                .build();
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Metadata handling
-    // --------------------------------------------------------------------------------------------
-
-    /** List of metadata that can be read with this format. */
-    enum ReadableMetadata {
-        DATABASE(
-                "database",
-                DataTypes.STRING().nullable(),
-                DataTypes.FIELD("database", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getString(pos);
-                    }
-                }),
-
-        TABLE(
-                "table",
-                DataTypes.STRING().nullable(),
-                DataTypes.FIELD("table", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getString(pos);
-                    }
-                }),
-
-        SQL_TYPE(
-                "sql-type",
-                DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable()).nullable(),
-                DataTypes.FIELD(
-                        "sqlType",
-                        DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable())),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getMap(pos);
-                    }
-                }),
-
-        PK_NAMES(
-                "pk-names",
-                DataTypes.ARRAY(DataTypes.STRING()).nullable(),
-                DataTypes.FIELD("pkNames", DataTypes.ARRAY(DataTypes.STRING())),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getArray(pos);
-                    }
-                }),
-
-        INGESTION_TIMESTAMP(
-                "ingestion-timestamp",
-                DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(),
-                DataTypes.FIELD("ts", DataTypes.BIGINT()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        if (row.isNullAt(pos)) {
-                            return null;
-                        }
-                        return TimestampData.fromEpochMillis(row.getLong(pos));
-                    }
-                }),
-
-        EVENT_TIMESTAMP(
-                "event-timestamp",
-                DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(),
-                DataTypes.FIELD("es", DataTypes.BIGINT()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        if (row.isNullAt(pos)) {
-                            return null;
-                        }
-                        return TimestampData.fromEpochMillis(row.getLong(pos));
-                    }
-                });
-
-        final String key;
-
-        final DataType dataType;
-
-        final DataTypes.Field requiredJsonField;
-
-        final MetadataConverter converter;
-
-        ReadableMetadata(
-                String key,
-                DataType dataType,
-                DataTypes.Field requiredJsonField,
-                MetadataConverter converter) {
-            this.key = key;
-            this.dataType = dataType;
-            this.requiredJsonField = requiredJsonField;
-            this.converter = converter;
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
deleted file mode 100644
index bb97778..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
+++ /dev/null
@@ -1,412 +0,0 @@
-/*
- * 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.annotation.Internal;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
-import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata;
-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.table.types.utils.DataTypeUtils;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.Collector;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Objects;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-
-/**
- * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link
- * RowData}. The deserialization schema knows Canal'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 final class CanalJsonDeserializationSchema implements DeserializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    private static final String FIELD_OLD = "old";
-    private static final String OP_INSERT = "INSERT";
-    private static final String OP_UPDATE = "UPDATE";
-    private static final String OP_DELETE = "DELETE";
-    private static final String OP_CREATE = "CREATE";
-
-    /** The deserializer to deserialize Canal JSON data. */
-    private final JsonRowDataDeserializationSchema jsonDeserializer;
-
-    /** Flag that indicates that an additional projection is required for metadata. */
-    private final boolean hasMetadata;
-
-    /** Metadata to be extracted for every record. */
-    private final MetadataConverter[] metadataConverters;
-
-    /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */
-    private final TypeInformation<RowData> producedTypeInfo;
-
-    /** Only read changelogs from the specific database. */
-    private final @Nullable String database;
-
-    /** Only read changelogs from the specific table. */
-    private final @Nullable String table;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    /** Names of fields. */
-    private final List<String> fieldNames;
-
-    /** Number of fields. */
-    private final int fieldCount;
-
-    /** Pattern of the specific database. */
-    private final Pattern databasePattern;
-
-    /** Pattern of the specific table. */
-    private final Pattern tablePattern;
-
-    private CanalJsonDeserializationSchema(
-            DataType physicalDataType,
-            List<ReadableMetadata> requestedMetadata,
-            TypeInformation<RowData> producedTypeInfo,
-            @Nullable String database,
-            @Nullable String table,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata);
-        this.jsonDeserializer =
-                new JsonRowDataDeserializationSchema(
-                        jsonRowType,
-                        // the result type is never used, so it's fine to pass in the produced type
-                        // info
-                        producedTypeInfo,
-                        false, // ignoreParseErrors already contains the functionality of
-                        // failOnMissingField
-                        ignoreParseErrors,
-                        timestampFormat);
-        this.hasMetadata = requestedMetadata.size() > 0;
-        this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata);
-        this.producedTypeInfo = producedTypeInfo;
-        this.database = database;
-        this.table = table;
-        this.ignoreParseErrors = ignoreParseErrors;
-        final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType());
-        this.fieldNames = physicalRowType.getFieldNames();
-        this.fieldCount = physicalRowType.getFieldCount();
-        this.databasePattern = database == null ? null : Pattern.compile(database);
-        this.tablePattern = table == null ? null : Pattern.compile(table);
-    }
-
-    // ------------------------------------------------------------------------------------------
-    // Builder
-    // ------------------------------------------------------------------------------------------
-
-    /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */
-    public static Builder builder(
-            DataType physicalDataType,
-            List<ReadableMetadata> requestedMetadata,
-            TypeInformation<RowData> producedTypeInfo) {
-        return new Builder(physicalDataType, requestedMetadata, producedTypeInfo);
-    }
-
-    /** A builder for creating a {@link CanalJsonDeserializationSchema}. */
-    @Internal
-    public static final class Builder {
-        private final DataType physicalDataType;
-        private final List<ReadableMetadata> requestedMetadata;
-        private final TypeInformation<RowData> producedTypeInfo;
-        private String database = null;
-        private String table = null;
-        private boolean ignoreParseErrors = false;
-        private TimestampFormat timestampFormat = TimestampFormat.SQL;
-
-        private Builder(
-                DataType physicalDataType,
-                List<ReadableMetadata> requestedMetadata,
-                TypeInformation<RowData> producedTypeInfo) {
-            this.physicalDataType = physicalDataType;
-            this.requestedMetadata = requestedMetadata;
-            this.producedTypeInfo = producedTypeInfo;
-        }
-
-        public Builder setDatabase(String database) {
-            this.database = database;
-            return this;
-        }
-
-        public Builder setTable(String table) {
-            this.table = table;
-            return this;
-        }
-
-        public Builder setIgnoreParseErrors(boolean ignoreParseErrors) {
-            this.ignoreParseErrors = ignoreParseErrors;
-            return this;
-        }
-
-        public Builder setTimestampFormat(TimestampFormat timestampFormat) {
-            this.timestampFormat = timestampFormat;
-            return this;
-        }
-
-        public CanalJsonDeserializationSchema build() {
-            return new CanalJsonDeserializationSchema(
-                    physicalDataType,
-                    requestedMetadata,
-                    producedTypeInfo,
-                    database,
-                    table,
-                    ignoreParseErrors,
-                    timestampFormat);
-        }
-    }
-
-    // ------------------------------------------------------------------------------------------
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonDeserializer.open(context);
-    }
-
-    @Override
-    public RowData deserialize(byte[] message) throws IOException {
-        throw new RuntimeException(
-                "Please invoke DeserializationSchema#deserialize(byte[], Collector<RowData>) instead.");
-    }
-
-    @Override
-    public void deserialize(@Nullable byte[] message, Collector<RowData> out) throws IOException {
-        if (message == null || message.length == 0) {
-            return;
-        }
-        try {
-            final JsonNode root = jsonDeserializer.deserializeToJsonNode(message);
-            if (database != null) {
-                if (!databasePattern
-                        .matcher(root.get(ReadableMetadata.DATABASE.key).asText())
-                        .matches()) {
-                    return;
-                }
-            }
-            if (table != null) {
-                if (!tablePattern
-                        .matcher(root.get(ReadableMetadata.TABLE.key).asText())
-                        .matches()) {
-                    return;
-                }
-            }
-            final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root);
-            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++) {
-                    GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount);
-                    insert.setRowKind(RowKind.INSERT);
-                    emitRow(row, insert, out);
-                }
-            } 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);
-                    final JsonNode oldField = root.get(FIELD_OLD);
-                    for (int f = 0; f < fieldCount; f++) {
-                        if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) {
-                            // fields in "old" (before) means the fields are changed
-                            // fields not in "old" (before) means the fields are 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);
-                    emitRow(row, before, out);
-                    emitRow(row, after, out);
-                }
-            } 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++) {
-                    GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount);
-                    insert.setRowKind(RowKind.DELETE);
-                    emitRow(row, insert, out);
-                }
-            } else if (OP_CREATE.equals(type)) {
-                // "data" field is null and "type" is "CREATE" which means
-                // this is a DDL change event, and we should skip it.
-                return;
-            } else {
-                if (!ignoreParseErrors) {
-                    throw new IOException(
-                            format(
-                                    "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'",
-                                    type, new String(message)));
-                }
-            }
-        } catch (Throwable t) {
-            // a big try catch to protect the processing.
-            if (!ignoreParseErrors) {
-                throw new IOException(
-                        format("Corrupt Canal JSON message '%s'.", new String(message)), t);
-            }
-        }
-    }
-
-    private void emitRow(
-            GenericRowData rootRow, GenericRowData physicalRow, Collector<RowData> out) {
-        // shortcut in case no output projection is required
-        if (!hasMetadata) {
-            out.collect(physicalRow);
-            return;
-        }
-        final int physicalArity = physicalRow.getArity();
-        final int metadataArity = metadataConverters.length;
-        final GenericRowData producedRow =
-                new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity);
-        for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) {
-            producedRow.setField(physicalPos, physicalRow.getField(physicalPos));
-        }
-        for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) {
-            producedRow.setField(
-                    physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow));
-        }
-        out.collect(producedRow);
-    }
-
-    @Override
-    public boolean isEndOfStream(RowData nextElement) {
-        return false;
-    }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        return producedTypeInfo;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o;
-        return Objects.equals(jsonDeserializer, that.jsonDeserializer)
-                && hasMetadata == that.hasMetadata
-                && Objects.equals(producedTypeInfo, that.producedTypeInfo)
-                && Objects.equals(database, that.database)
-                && Objects.equals(table, that.table)
-                && ignoreParseErrors == that.ignoreParseErrors
-                && fieldCount == that.fieldCount;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                jsonDeserializer,
-                hasMetadata,
-                producedTypeInfo,
-                database,
-                table,
-                ignoreParseErrors,
-                fieldCount);
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    private static RowType createJsonRowType(
-            DataType physicalDataType, List<ReadableMetadata> readableMetadata) {
-        // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them
-        DataType root =
-                DataTypes.ROW(
-                        DataTypes.FIELD("data", DataTypes.ARRAY(physicalDataType)),
-                        DataTypes.FIELD("old", DataTypes.ARRAY(physicalDataType)),
-                        DataTypes.FIELD("type", DataTypes.STRING()),
-                        ReadableMetadata.DATABASE.requiredJsonField,
-                        ReadableMetadata.TABLE.requiredJsonField);
-        // append fields that are required for reading metadata in the root
-        final List<DataTypes.Field> rootMetadataFields =
-                readableMetadata.stream()
-                        .filter(m -> m != ReadableMetadata.DATABASE && m != ReadableMetadata.TABLE)
-                        .map(m -> m.requiredJsonField)
-                        .distinct()
-                        .collect(Collectors.toList());
-        return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType();
-    }
-
-    private static MetadataConverter[] createMetadataConverters(
-            RowType jsonRowType, List<ReadableMetadata> requestedMetadata) {
-        return requestedMetadata.stream()
-                .map(m -> convert(jsonRowType, m))
-                .toArray(MetadataConverter[]::new);
-    }
-
-    private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) {
-        final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName());
-        return new MetadataConverter() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public Object convert(GenericRowData root, int unused) {
-                return metadata.converter.convert(root, pos);
-            }
-        };
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    /**
-     * Converter that extracts a metadata field from the row that comes out of the JSON schema and
-     * converts it to the desired data type.
-     */
-    interface MetadataConverter extends Serializable {
-
-        // Method for top-level access.
-        default Object convert(GenericRowData row) {
-            return convert(row, -1);
-        }
-
-        Object convert(GenericRowData row, int pos);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java
deleted file mode 100644
index 9f3ccad..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.annotation.Internal;
-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.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonFormatOptionsUtil;
-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.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;
-
-import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.DATABASE_INCLUDE;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.IGNORE_PARSE_ERRORS;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_MODE;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TABLE_INCLUDE;
-import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TIMESTAMP_FORMAT;
-
-/**
- * Format factory for providing configured instances of Canal JSON to RowData {@link
- * DeserializationSchema}.
- */
-@Internal
-public class CanalJsonFormatFactory
-        implements DeserializationFormatFactory, SerializationFormatFactory {
-
-    public static final String IDENTIFIER = "canal-json";
-
-    @Override
-    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateDecodingFormatOptions(formatOptions);
-
-        final String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null);
-        final String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null);
-        final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
-        final TimestampFormat timestampFormat =
-                JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-
-        return new CanalJsonDecodingFormat(database, table, ignoreParseErrors, timestampFormat);
-    }
-
-    @Override
-    public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateEncodingFormatOptions(formatOptions);
-
-        TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-        JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
-                JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
-        String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL);
-
-        final boolean encodeDecimalAsPlainNumber =
-                formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-
-        return new EncodingFormat<SerializationSchema<RowData>>() {
-            @Override
-            public ChangelogMode getChangelogMode() {
-                return ChangelogMode.newBuilder()
-                        .addContainedKind(RowKind.INSERT)
-                        .addContainedKind(RowKind.UPDATE_BEFORE)
-                        .addContainedKind(RowKind.UPDATE_AFTER)
-                        .addContainedKind(RowKind.DELETE)
-                        .build();
-            }
-
-            @Override
-            public SerializationSchema<RowData> createRuntimeEncoder(
-                    DynamicTableSink.Context context, DataType consumedDataType) {
-                final RowType rowType = (RowType) consumedDataType.getLogicalType();
-                return new CanalJsonSerializationSchema(
-                        rowType,
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-            }
-        };
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(IGNORE_PARSE_ERRORS);
-        options.add(TIMESTAMP_FORMAT);
-        options.add(DATABASE_INCLUDE);
-        options.add(TABLE_INCLUDE);
-        options.add(JSON_MAP_NULL_KEY_MODE);
-        options.add(JSON_MAP_NULL_KEY_LITERAL);
-        options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-        return options;
-    }
-
-    /** Validator for canal decoding format. */
-    private static void validateDecodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions);
-    }
-
-    /** Validator for canal encoding format. */
-    private static void validateEncodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java
deleted file mode 100644
index 6fa11bc..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.formats.json.JsonFormatOptions;
-
-/** Option utils for canal-json format. */
-@PublicEvolving
-public class CanalJsonFormatOptions {
-
-    public static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS =
-            JsonFormatOptions.IGNORE_PARSE_ERRORS;
-
-    public static final ConfigOption<String> TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_MODE =
-            JsonFormatOptions.MAP_NULL_KEY_MODE;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_LITERAL =
-            JsonFormatOptions.MAP_NULL_KEY_LITERAL;
-
-    public static final ConfigOption<String> DATABASE_INCLUDE =
-            ConfigOptions.key("database.include")
-                    .stringType()
-                    .noDefaultValue()
-                    .withDescription(
-                            "An optional regular expression to only read the specific databases changelog rows by regular matching the \"database\" meta field in the Canal record."
-                                    + "The pattern string is compatible with Java's Pattern.");
-
-    public static final ConfigOption<String> TABLE_INCLUDE =
-            ConfigOptions.key("table.include")
-                    .stringType()
-                    .noDefaultValue()
-                    .withDescription(
-                            "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record."
-                                    + "The pattern string is compatible with Java's Pattern.");
-
-    private CanalJsonFormatOptions() {}
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java
deleted file mode 100644
index 362b9df..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
-import org.apache.flink.table.api.DataTypes;
-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.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 that serializes an object of Flink Table/SQL internal data structure {@link
- * RowData} into a Canal JSON bytes.
- *
- * @see <a href="https://github.com/alibaba/canal">Alibaba Canal</a>
- */
-public class CanalJsonSerializationSchema implements SerializationSchema<RowData> {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final StringData OP_INSERT = StringData.fromString("INSERT");
-    private static final StringData OP_DELETE = StringData.fromString("DELETE");
-
-    private transient GenericRowData reuse;
-
-    /** The serializer to serialize Canal JSON data. */
-    private final JsonRowDataSerializationSchema jsonSerializer;
-
-    public CanalJsonSerializationSchema(
-            RowType rowType,
-            TimestampFormat timestampFormat,
-            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
-            String mapNullKeyLiteral,
-            boolean encodeDecimalAsPlainNumber) {
-        jsonSerializer =
-                new JsonRowDataSerializationSchema(
-                        createJsonRowType(fromLogicalToDataType(rowType)),
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonSerializer.open(context);
-        reuse = new GenericRowData(2);
-    }
-
-    @Override
-    public byte[] serialize(RowData row) {
-        try {
-            StringData opType = rowKind2String(row.getRowKind());
-            ArrayData arrayData = new GenericArrayData(new RowData[] {row});
-            reuse.setField(0, arrayData);
-            reuse.setField(1, opType);
-            return jsonSerializer.serialize(reuse);
-        } catch (Throwable t) {
-            throw new RuntimeException("Could not serialize row '" + row + "'.", t);
-        }
-    }
-
-    private StringData rowKind2String(RowKind rowKind) {
-        switch (rowKind) {
-            case INSERT:
-            case UPDATE_AFTER:
-                return OP_INSERT;
-            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;
-        }
-        CanalJsonSerializationSchema that = (CanalJsonSerializationSchema) o;
-        return Objects.equals(jsonSerializer, that.jsonSerializer);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(jsonSerializer);
-    }
-
-    private static RowType createJsonRowType(DataType databaseSchema) {
-        // Canal JSON contains other information, e.g. "database", "ts"
-        // but we don't need them
-        // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER
-        return (RowType)
-                DataTypes.ROW(
-                                DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)),
-                                DataTypes.FIELD("type", DataTypes.STRING()))
-                        .getLogicalType();
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java
deleted file mode 100644
index 71d1ee9..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.maxwell.MaxwellJsonDeserializationSchema.MetadataConverter;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
-import org.apache.flink.types.RowKind;
-
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/** {@link DecodingFormat} for Maxwell using JSON encoding. */
-public class MaxwellJsonDecodingFormat
-        implements ProjectableDecodingFormat<DeserializationSchema<RowData>> {
-
-    // --------------------------------------------------------------------------------------------
-    // Mutable attributes
-    // --------------------------------------------------------------------------------------------
-
-    /** The requested metadata keys. */
-    private List<String> metadataKeys;
-
-    private final boolean ignoreParseErrors;
-
-    private final TimestampFormat timestampFormat;
-
-    public MaxwellJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) {
-        this.ignoreParseErrors = ignoreParseErrors;
-        this.timestampFormat = timestampFormat;
-        this.metadataKeys = Collections.emptyList();
-    }
-
-    @Override
-    public DeserializationSchema<RowData> createRuntimeDecoder(
-            DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) {
-        physicalDataType = Projection.of(projections).project(physicalDataType);
-
-        final List<ReadableMetadata> readableMetadata =
-                metadataKeys.stream()
-                        .map(
-                                k ->
-                                        Stream.of(ReadableMetadata.values())
-                                                .filter(rm -> rm.key.equals(k))
-                                                .findFirst()
-                                                .orElseThrow(
-                                                        () ->
-                                                                new IllegalStateException(
-                                                                        String.format(
-                                                                                "Could not find the requested metadata key: %s",
-                                                                                k))))
-                        .collect(Collectors.toList());
-        final List<DataTypes.Field> metadataFields =
-                readableMetadata.stream()
-                        .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                        .collect(Collectors.toList());
-        final DataType producedDataType =
-                DataTypeUtils.appendRowFields(physicalDataType, metadataFields);
-        final TypeInformation<RowData> producedTypeInfo =
-                context.createTypeInformation(producedDataType);
-        return new MaxwellJsonDeserializationSchema(
-                physicalDataType,
-                readableMetadata,
-                producedTypeInfo,
-                ignoreParseErrors,
-                timestampFormat);
-    }
-
-    @Override
-    public Map<String, DataType> listReadableMetadata() {
-        final Map<String, DataType> metadataMap = new LinkedHashMap<>();
-        Stream.of(ReadableMetadata.values())
-                .forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
-        return metadataMap;
-    }
-
-    @Override
-    public void applyReadableMetadata(List<String> metadataKeys) {
-        this.metadataKeys = metadataKeys;
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode() {
-        return ChangelogMode.newBuilder()
-                .addContainedKind(RowKind.INSERT)
-                .addContainedKind(RowKind.UPDATE_BEFORE)
-                .addContainedKind(RowKind.UPDATE_AFTER)
-                .addContainedKind(RowKind.DELETE)
-                .build();
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Metadata handling
-    // --------------------------------------------------------------------------------------------
-
-    /** List of metadata that can be read with this format. */
-    enum ReadableMetadata {
-        DATABASE(
-                "database",
-                DataTypes.STRING().nullable(),
-                DataTypes.FIELD("database", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getString(pos);
-                    }
-                }),
-
-        TABLE(
-                "table",
-                DataTypes.STRING().nullable(),
-                DataTypes.FIELD("table", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getString(pos);
-                    }
-                }),
-
-        PRIMARY_KEY_COLUMNS(
-                "primary-key-columns",
-                DataTypes.ARRAY(DataTypes.STRING()).nullable(),
-                DataTypes.FIELD("primary_key_columns", DataTypes.ARRAY(DataTypes.STRING())),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getArray(pos);
-                    }
-                }),
-
-        INGESTION_TIMESTAMP(
-                "ingestion-timestamp",
-                DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(),
-                DataTypes.FIELD("ts", DataTypes.BIGINT()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        if (row.isNullAt(pos)) {
-                            return null;
-                        }
-                        // the `ts` field of Maxwell JSON is assigned with `timestampSeconds`.
-                        return TimestampData.fromEpochMillis(row.getLong(pos) * 1000);
-                    }
-                });
-
-        final String key;
-
-        final DataType dataType;
-
-        final DataTypes.Field requiredJsonField;
-
-        final MetadataConverter converter;
-
-        ReadableMetadata(
-                String key,
-                DataType dataType,
-                DataTypes.Field requiredJsonField,
-                MetadataConverter converter) {
-            this.key = key;
-            this.dataType = dataType;
-            this.requiredJsonField = requiredJsonField;
-            this.converter = converter;
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java
deleted file mode 100644
index f58bfd4..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
-import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata;
-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.table.types.utils.DataTypeUtils;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.Collector;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-
-/**
- * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link
- * RowData}. The deserialization schema knows Maxwell'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="http://maxwells-daemon.io/">Maxwell</a>
- */
-public class MaxwellJsonDeserializationSchema implements DeserializationSchema<RowData> {
-    private static final long serialVersionUID = 2L;
-
-    private static final String FIELD_OLD = "old";
-    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 Maxwell JSON data. */
-    private final JsonRowDataDeserializationSchema jsonDeserializer;
-
-    /** Flag that indicates that an additional projection is required for metadata. */
-    private final boolean hasMetadata;
-
-    /** Metadata to be extracted for every record. */
-    private final MetadataConverter[] metadataConverters;
-
-    /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */
-    private final TypeInformation<RowData> producedTypeInfo;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    /** Names of physical fields. */
-    private final List<String> fieldNames;
-
-    /** Number of physical fields. */
-    private final int fieldCount;
-
-    public MaxwellJsonDeserializationSchema(
-            DataType physicalDataType,
-            List<ReadableMetadata> requestedMetadata,
-            TypeInformation<RowData> producedTypeInfo,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata);
-        this.jsonDeserializer =
-                new JsonRowDataDeserializationSchema(
-                        jsonRowType,
-                        // the result type is never used, so it's fine to pass in the produced type
-                        // info
-                        producedTypeInfo,
-                        // ignoreParseErrors already contains the functionality of
-                        // failOnMissingField
-                        false,
-                        ignoreParseErrors,
-                        timestampFormat);
-        this.hasMetadata = requestedMetadata.size() > 0;
-        this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata);
-        this.producedTypeInfo = producedTypeInfo;
-        this.ignoreParseErrors = ignoreParseErrors;
-        final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType());
-        this.fieldNames = physicalRowType.getFieldNames();
-        this.fieldCount = physicalRowType.getFieldCount();
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonDeserializer.open(context);
-    }
-
-    @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) {
-            return;
-        }
-        try {
-            final JsonNode root = jsonDeserializer.deserializeToJsonNode(message);
-            final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root);
-            String type = row.getString(2).toString(); // "type" field
-            if (OP_INSERT.equals(type)) {
-                // "data" field is a row, contains inserted rows
-                GenericRowData insert = (GenericRowData) row.getRow(0, fieldCount);
-                insert.setRowKind(RowKind.INSERT);
-                emitRow(row, insert, out);
-            } else if (OP_UPDATE.equals(type)) {
-                // "data" field is a row, contains new rows
-                // "old" field is a row, contains old values
-                // the underlying JSON deserialization schema always produce GenericRowData.
-                GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field
-                GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field
-                final JsonNode oldField = root.get(FIELD_OLD);
-                for (int f = 0; f < fieldCount; f++) {
-                    if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) {
-                        // not null fields in "old" (before) means the fields are changed
-                        // null/empty fields in "old" (before) means the fields are 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);
-                emitRow(row, before, out);
-                emitRow(row, after, out);
-            } else if (OP_DELETE.equals(type)) {
-                // "data" field is a row, contains deleted rows
-                GenericRowData delete = (GenericRowData) row.getRow(0, fieldCount);
-                delete.setRowKind(RowKind.DELETE);
-                emitRow(row, delete, out);
-            } else {
-                if (!ignoreParseErrors) {
-                    throw new IOException(
-                            format(
-                                    "Unknown \"type\" value \"%s\". The Maxwell JSON message is '%s'",
-                                    type, new String(message)));
-                }
-            }
-        } catch (Throwable t) {
-            // a big try catch to protect the processing.
-            if (!ignoreParseErrors) {
-                throw new IOException(
-                        format("Corrupt Maxwell JSON message '%s'.", new String(message)), t);
-            }
-        }
-    }
-
-    private void emitRow(
-            GenericRowData rootRow, GenericRowData physicalRow, Collector<RowData> out) {
-        // shortcut in case no output projection is required
-        if (!hasMetadata) {
-            out.collect(physicalRow);
-            return;
-        }
-        final int metadataArity = metadataConverters.length;
-        final GenericRowData producedRow =
-                new GenericRowData(physicalRow.getRowKind(), fieldCount + metadataArity);
-        for (int physicalPos = 0; physicalPos < fieldCount; physicalPos++) {
-            producedRow.setField(physicalPos, physicalRow.getField(physicalPos));
-        }
-        for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) {
-            producedRow.setField(
-                    fieldCount + metadataPos, metadataConverters[metadataPos].convert(rootRow));
-        }
-        out.collect(producedRow);
-    }
-
-    @Override
-    public boolean isEndOfStream(RowData nextElement) {
-        return false;
-    }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        return producedTypeInfo;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o;
-        return Objects.equals(jsonDeserializer, that.jsonDeserializer)
-                && hasMetadata == that.hasMetadata
-                && Objects.equals(producedTypeInfo, that.producedTypeInfo)
-                && ignoreParseErrors == that.ignoreParseErrors
-                && fieldCount == that.fieldCount;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors, fieldCount);
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    private static RowType createJsonRowType(
-            DataType physicalDataType, List<ReadableMetadata> readableMetadata) {
-        DataType root =
-                DataTypes.ROW(
-                        DataTypes.FIELD("data", physicalDataType),
-                        DataTypes.FIELD("old", physicalDataType),
-                        DataTypes.FIELD("type", DataTypes.STRING()));
-        // append fields that are required for reading metadata in the root
-        final List<DataTypes.Field> rootMetadataFields =
-                readableMetadata.stream()
-                        .map(m -> m.requiredJsonField)
-                        .distinct()
-                        .collect(Collectors.toList());
-        return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType();
-    }
-
-    private static MetadataConverter[] createMetadataConverters(
-            RowType jsonRowType, List<ReadableMetadata> requestedMetadata) {
-        return requestedMetadata.stream()
-                .map(m -> convert(jsonRowType, m))
-                .toArray(MetadataConverter[]::new);
-    }
-
-    private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) {
-        final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName());
-        return new MetadataConverter() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public Object convert(GenericRowData root, int unused) {
-                return metadata.converter.convert(root, pos);
-            }
-        };
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    /**
-     * Converter that extracts a metadata field from the row that comes out of the JSON schema and
-     * converts it to the desired data type.
-     */
-    interface MetadataConverter extends Serializable {
-
-        // Method for top-level access.
-        default Object convert(GenericRowData row) {
-            return convert(row, -1);
-        }
-
-        Object convert(GenericRowData row, int pos);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java
deleted file mode 100644
index 1bbbec8..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.annotation.Internal;
-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.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonFormatOptionsUtil;
-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.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;
-
-import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
-import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.IGNORE_PARSE_ERRORS;
-import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;
-import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.JSON_MAP_NULL_KEY_MODE;
-import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.TIMESTAMP_FORMAT;
-
-/**
- * Format factory for providing configured instances of Maxwell JSON to RowData {@link
- * DeserializationSchema}.
- */
-@Internal
-public class MaxwellJsonFormatFactory
-        implements DeserializationFormatFactory, SerializationFormatFactory {
-
-    public static final String IDENTIFIER = "maxwell-json";
-
-    @Override
-    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateDecodingFormatOptions(formatOptions);
-
-        final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
-        final TimestampFormat timestampFormat =
-                JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-
-        return new MaxwellJsonDecodingFormat(ignoreParseErrors, timestampFormat);
-    }
-
-    @Override
-    public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateEncodingFormatOptions(formatOptions);
-
-        TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-        JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
-                JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
-        String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL);
-
-        final boolean encodeDecimalAsPlainNumber =
-                formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-
-        return new EncodingFormat<SerializationSchema<RowData>>() {
-
-            @Override
-            public ChangelogMode getChangelogMode() {
-                return ChangelogMode.newBuilder()
-                        .addContainedKind(RowKind.INSERT)
-                        .addContainedKind(RowKind.UPDATE_BEFORE)
-                        .addContainedKind(RowKind.UPDATE_AFTER)
-                        .addContainedKind(RowKind.DELETE)
-                        .build();
-            }
-
-            @Override
-            public SerializationSchema<RowData> createRuntimeEncoder(
-                    DynamicTableSink.Context context, DataType consumedDataType) {
-                final RowType rowType = (RowType) consumedDataType.getLogicalType();
-                return new MaxwellJsonSerializationSchema(
-                        rowType,
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-            }
-        };
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(IGNORE_PARSE_ERRORS);
-        options.add(TIMESTAMP_FORMAT);
-        options.add(JSON_MAP_NULL_KEY_MODE);
-        options.add(JSON_MAP_NULL_KEY_LITERAL);
-        options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-        return options;
-    }
-
-    /** Validator for maxwell decoding format. */
-    private static void validateDecodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions);
-    }
-
-    /** Validator for maxwell encoding format. */
-    private static void validateEncodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java
deleted file mode 100644
index 011fa7b..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.formats.json.JsonFormatOptions;
-
-/** Option utils for maxwell-json format. */
-@PublicEvolving
-public class MaxwellJsonFormatOptions {
-
-    public static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS =
-            JsonFormatOptions.IGNORE_PARSE_ERRORS;
-
-    public static final ConfigOption<String> TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_MODE =
-            JsonFormatOptions.MAP_NULL_KEY_MODE;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_LITERAL =
-            JsonFormatOptions.MAP_NULL_KEY_LITERAL;
-
-    private MaxwellJsonFormatOptions() {}
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java
deleted file mode 100644
index 1fe567b..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
-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
- * maxwell-Json.
- */
-public class MaxwellJsonSerializationSchema implements SerializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    private static final StringData OP_INSERT = StringData.fromString("insert");
-    private static final StringData OP_DELETE = StringData.fromString("delete");
-
-    private final JsonRowDataSerializationSchema jsonSerializer;
-
-    /** Timestamp format specification which is used to parse timestamp. */
-    private final TimestampFormat timestampFormat;
-
-    private transient GenericRowData reuse;
-
-    public MaxwellJsonSerializationSchema(
-            RowType rowType,
-            TimestampFormat timestampFormat,
-            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
-            String mapNullKeyLiteral,
-            boolean encodeDecimalAsPlainNumber) {
-        this.jsonSerializer =
-                new JsonRowDataSerializationSchema(
-                        createJsonRowType(fromLogicalToDataType(rowType)),
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-        this.timestampFormat = timestampFormat;
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonSerializer.open(context);
-        this.reuse = new GenericRowData(2);
-    }
-
-    @Override
-    public byte[] serialize(RowData element) {
-        reuse.setField(0, element);
-        reuse.setField(1, rowKind2String(element.getRowKind()));
-        return jsonSerializer.serialize(reuse);
-    }
-
-    private StringData rowKind2String(RowKind rowKind) {
-        switch (rowKind) {
-            case INSERT:
-            case UPDATE_AFTER:
-                return OP_INSERT;
-            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;
-        }
-        MaxwellJsonSerializationSchema that = (MaxwellJsonSerializationSchema) o;
-        return Objects.equals(jsonSerializer, that.jsonSerializer)
-                && timestampFormat == that.timestampFormat;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(jsonSerializer, timestampFormat);
-    }
-
-    private RowType createJsonRowType(DataType databaseSchema) {
-        DataType payload =
-                DataTypes.ROW(
-                        DataTypes.FIELD("data", databaseSchema),
-                        DataTypes.FIELD("type", DataTypes.STRING()));
-        return (RowType) payload.getLogicalType();
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java
deleted file mode 100644
index 6730fdc..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.ogg.OggJsonDeserializationSchema.MetadataConverter;
-import org.apache.flink.table.api.DataTypes;
-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.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
-import org.apache.flink.table.utils.DateTimeUtils;
-import org.apache.flink.types.RowKind;
-
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/** {@link DecodingFormat} for Ogg using JSON encoding. */
-public class OggJsonDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> {
-
-    // --------------------------------------------------------------------------------------------
-    // Mutable attributes
-    // --------------------------------------------------------------------------------------------
-
-    private List<String> metadataKeys;
-
-    // --------------------------------------------------------------------------------------------
-    // Ogg-specific attributes
-    // --------------------------------------------------------------------------------------------
-
-    private final boolean ignoreParseErrors;
-    private final TimestampFormat timestampFormat;
-
-    public OggJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) {
-        this.ignoreParseErrors = ignoreParseErrors;
-        this.timestampFormat = timestampFormat;
-        this.metadataKeys = Collections.emptyList();
-    }
-
-    @Override
-    public DeserializationSchema<RowData> createRuntimeDecoder(
-            DynamicTableSource.Context context, DataType physicalDataType) {
-
-        final List<ReadableMetadata> readableMetadata =
-                metadataKeys.stream()
-                        .map(
-                                k ->
-                                        Stream.of(ReadableMetadata.values())
-                                                .filter(rm -> rm.key.equals(k))
-                                                .findFirst()
-                                                .orElseThrow(IllegalStateException::new))
-                        .collect(Collectors.toList());
-
-        final List<DataTypes.Field> metadataFields =
-                readableMetadata.stream()
-                        .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                        .collect(Collectors.toList());
-
-        final DataType producedDataType =
-                DataTypeUtils.appendRowFields(physicalDataType, metadataFields);
-
-        final TypeInformation<RowData> producedTypeInfo =
-                context.createTypeInformation(producedDataType);
-
-        return new OggJsonDeserializationSchema(
-                physicalDataType,
-                readableMetadata,
-                producedTypeInfo,
-                ignoreParseErrors,
-                timestampFormat);
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Metadata handling
-    // --------------------------------------------------------------------------------------------
-
-    @Override
-    public Map<String, DataType> listReadableMetadata() {
-        final Map<String, DataType> metadataMap = new LinkedHashMap<>();
-        Stream.of(ReadableMetadata.values())
-                .forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
-        return metadataMap;
-    }
-
-    @Override
-    public void applyReadableMetadata(List<String> metadataKeys) {
-        this.metadataKeys = metadataKeys;
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode() {
-        return ChangelogMode.newBuilder()
-                .addContainedKind(RowKind.INSERT)
-                .addContainedKind(RowKind.UPDATE_BEFORE)
-                .addContainedKind(RowKind.UPDATE_AFTER)
-                .addContainedKind(RowKind.DELETE)
-                .build();
-    }
-
-    /** List of metadata that can be read with this format. */
-    enum ReadableMetadata {
-        TABLE(
-                "table",
-                DataTypes.STRING().nullable(),
-                DataTypes.FIELD("table", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getString(pos);
-                    }
-                }),
-
-        PRIMARY_KEYS(
-                "primary-keys",
-                DataTypes.ARRAY(DataTypes.STRING()).nullable(),
-                DataTypes.FIELD("primary_keys", DataTypes.ARRAY(DataTypes.STRING())),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        return row.getArray(pos);
-                    }
-                }),
-
-        INGESTION_TIMESTAMP(
-                "ingestion-timestamp",
-                DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(),
-                DataTypes.FIELD("current_ts", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        if (row.isNullAt(pos)) {
-                            return null;
-                        }
-                        // the timestamp follows the ISO-8601 format
-                        return DateTimeUtils.parseTimestampData(
-                                row.getString(pos).toString(), "yyyy-MM-dd'T'HH:mm:ss.SSSSSS");
-                    }
-                }),
-
-        EVENT_TIMESTAMP(
-                "event-timestamp",
-                DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(),
-                DataTypes.FIELD("op_ts", DataTypes.STRING()),
-                new MetadataConverter() {
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public Object convert(GenericRowData row, int pos) {
-                        if (row.isNullAt(pos)) {
-                            return null;
-                        }
-                        // the timestamp format is like "yyyy-MM-dd HH:mm:ss.SSSSSS"
-                        return DateTimeUtils.parseTimestampData(row.getString(pos).toString());
-                    }
-                });
-
-        final String key;
-
-        final DataType dataType;
-
-        final DataTypes.Field requiredJsonField;
-
-        final MetadataConverter converter;
-
-        ReadableMetadata(
-                String key,
-                DataType dataType,
-                DataTypes.Field requiredJsonField,
-                MetadataConverter converter) {
-            this.key = key;
-            this.dataType = dataType;
-            this.requiredJsonField = requiredJsonField;
-            this.converter = converter;
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java
deleted file mode 100644
index 85bd52d..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * 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.ogg;
-
-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.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
-import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata;
-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.table.types.utils.DataTypeUtils;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.Collector;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-
-/**
- * Deserialization schema from Ogg JSON to Flink Table/SQL internal data structure {@link RowData}.
- * The deserialization schema knows Ogg'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://www.oracle.com/cn/middleware/technologies/goldengate/overview.html">Ogg</a>
- */
-@Internal
-public final class OggJsonDeserializationSchema implements DeserializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    private static final String OP_CREATE = "I"; // insert
-    private static final String OP_UPDATE = "U"; // update
-    private static final String OP_DELETE = "D"; // delete
-    private static final String OP_TRUNCATE = "T"; // truncate
-
-    private static final String REPLICA_IDENTITY_EXCEPTION =
-            "The \"before\" field of %s message is null, "
-                    + "if you are using Ogg Postgres Connector, "
-                    + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.";
-
-    /** The deserializer to deserialize Ogg JSON data. */
-    private final JsonRowDataDeserializationSchema jsonDeserializer;
-
-    /** Flag that indicates that an additional projection is required for metadata. */
-    private final boolean hasMetadata;
-
-    /** Metadata to be extracted for every record. */
-    private final MetadataConverter[] metadataConverters;
-
-    /** {@link TypeInformation} of the produced {@link RowData} (physical + metadata). */
-    private final TypeInformation<RowData> producedTypeInfo;
-
-    /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
-    private final boolean ignoreParseErrors;
-
-    public OggJsonDeserializationSchema(
-            DataType physicalDataType,
-            List<ReadableMetadata> requestedMetadata,
-            TypeInformation<RowData> producedTypeInfo,
-            boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
-        final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata);
-        this.jsonDeserializer =
-                new JsonRowDataDeserializationSchema(
-                        jsonRowType,
-                        // the result type is never used, so it's fine to pass in the produced type
-                        // info
-                        producedTypeInfo,
-                        false, // ignoreParseErrors already contains the functionality of
-                        // failOnMissingField
-                        ignoreParseErrors,
-                        timestampFormat);
-        this.hasMetadata = requestedMetadata.size() > 0;
-        this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata);
-        this.producedTypeInfo = producedTypeInfo;
-        this.ignoreParseErrors = ignoreParseErrors;
-    }
-
-    private static RowType createJsonRowType(
-            DataType physicalDataType, List<ReadableMetadata> readableMetadata) {
-        DataType root =
-                DataTypes.ROW(
-                        DataTypes.FIELD("before", physicalDataType),
-                        DataTypes.FIELD("after", physicalDataType),
-                        DataTypes.FIELD("op_type", DataTypes.STRING()));
-        // append fields that are required for reading metadata in the root
-        final List<DataTypes.Field> rootMetadataFields =
-                readableMetadata.stream()
-                        .map(m -> m.requiredJsonField)
-                        .distinct()
-                        .collect(Collectors.toList());
-        return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType();
-    }
-
-    private static MetadataConverter[] createMetadataConverters(
-            RowType jsonRowType, List<ReadableMetadata> requestedMetadata) {
-        return requestedMetadata.stream()
-                .map(m -> convertInRoot(jsonRowType, m))
-                .toArray(MetadataConverter[]::new);
-    }
-
-    private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) {
-        final int pos = findFieldPos(metadata, jsonRowType);
-        return new MetadataConverter() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public Object convert(GenericRowData root, int unused) {
-                return metadata.converter.convert(root, pos);
-            }
-        };
-    }
-
-    private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) {
-        return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName());
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonDeserializer.open(context);
-    }
-
-    @Override
-    public RowData deserialize(byte[] message) {
-        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) jsonDeserializer.deserialize(message);
-
-            GenericRowData before = (GenericRowData) row.getField(0);
-            GenericRowData after = (GenericRowData) row.getField(1);
-            String op = row.getField(2).toString();
-            if (OP_CREATE.equals(op)) {
-                after.setRowKind(RowKind.INSERT);
-                emitRow(row, after, out);
-            } else if (OP_UPDATE.equals(op)) {
-                if (before == null) {
-                    throw new IllegalStateException(
-                            String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE"));
-                }
-                before.setRowKind(RowKind.UPDATE_BEFORE);
-                after.setRowKind(RowKind.UPDATE_AFTER);
-                emitRow(row, before, out);
-                emitRow(row, after, out);
-            } else if (OP_DELETE.equals(op)) {
-                if (before == null) {
-                    throw new IllegalStateException(
-                            String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE"));
-                }
-                before.setRowKind(RowKind.DELETE);
-                emitRow(row, before, out);
-            } else {
-                if (!ignoreParseErrors) {
-                    throw new IOException(
-                            format(
-                                    "Unknown \"op_type\" value \"%s\". The Ogg JSON message is '%s'",
-                                    op, new String(message)));
-                }
-            }
-        } catch (Throwable t) {
-            // a big try catch to protect the processing.
-            if (!ignoreParseErrors) {
-                throw new IOException(
-                        format("Corrupt Ogg JSON message '%s'.", new String(message)), t);
-            }
-        }
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    private void emitRow(
-            GenericRowData rootRow, GenericRowData physicalRow, Collector<RowData> out) {
-        // shortcut in case no output projection is required
-        if (!hasMetadata) {
-            out.collect(physicalRow);
-            return;
-        }
-
-        final int physicalArity = physicalRow.getArity();
-        final int metadataArity = metadataConverters.length;
-
-        final GenericRowData producedRow =
-                new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity);
-
-        for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) {
-            producedRow.setField(physicalPos, physicalRow.getField(physicalPos));
-        }
-
-        for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) {
-            producedRow.setField(
-                    physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow));
-        }
-
-        out.collect(producedRow);
-    }
-
-    @Override
-    public boolean isEndOfStream(RowData nextElement) {
-        return false;
-    }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        return producedTypeInfo;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        OggJsonDeserializationSchema that = (OggJsonDeserializationSchema) o;
-        return Objects.equals(jsonDeserializer, that.jsonDeserializer)
-                && hasMetadata == that.hasMetadata
-                && Objects.equals(producedTypeInfo, that.producedTypeInfo)
-                && ignoreParseErrors == that.ignoreParseErrors;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors);
-    }
-
-    // --------------------------------------------------------------------------------------------
-
-    /**
-     * Converter that extracts a metadata field from the row (root or payload) that comes out of the
-     * JSON schema and converts it to the desired data type.
-     */
-    interface MetadataConverter extends Serializable {
-
-        // Method for top-level access.
-        default Object convert(GenericRowData row) {
-            return convert(row, -1);
-        }
-
-        Object convert(GenericRowData row, int pos);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java
deleted file mode 100644
index f853983..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.annotation.Internal;
-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.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonFormatOptionsUtil;
-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.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;
-
-import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
-import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.IGNORE_PARSE_ERRORS;
-import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;
-import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_MODE;
-import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.TIMESTAMP_FORMAT;
-
-/**
- * Format factory for providing configured instances of Ogg JSON to RowData {@link
- * DeserializationSchema}.
- */
-@Internal
-public class OggJsonFormatFactory
-        implements DeserializationFormatFactory, SerializationFormatFactory {
-
-    public static final String IDENTIFIER = "ogg-json";
-
-    /** Validator for ogg decoding format. */
-    private static void validateDecodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions);
-    }
-
-    /** Validator for ogg encoding format. */
-    private static void validateEncodingFormatOptions(ReadableConfig tableOptions) {
-        JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions);
-    }
-
-    @Override
-    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateDecodingFormatOptions(formatOptions);
-
-        final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
-
-        final TimestampFormat timestampFormat =
-                JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-
-        return new OggJsonDecodingFormat(ignoreParseErrors, timestampFormat);
-    }
-
-    @Override
-    public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(
-            DynamicTableFactory.Context context, ReadableConfig formatOptions) {
-
-        FactoryUtil.validateFactoryOptions(this, formatOptions);
-        validateEncodingFormatOptions(formatOptions);
-
-        TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
-        JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
-                JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
-        String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL);
-
-        final boolean encodeDecimalAsPlainNumber =
-                formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-
-        return new EncodingFormat<SerializationSchema<RowData>>() {
-
-            @Override
-            public ChangelogMode getChangelogMode() {
-                return ChangelogMode.newBuilder()
-                        .addContainedKind(RowKind.INSERT)
-                        .addContainedKind(RowKind.UPDATE_BEFORE)
-                        .addContainedKind(RowKind.UPDATE_AFTER)
-                        .addContainedKind(RowKind.DELETE)
-                        .build();
-            }
-
-            @Override
-            public SerializationSchema<RowData> createRuntimeEncoder(
-                    DynamicTableSink.Context context, DataType consumedDataType) {
-                final RowType rowType = (RowType) consumedDataType.getLogicalType();
-                return new OggJsonSerializationSchema(
-                        rowType,
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-            }
-        };
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(IGNORE_PARSE_ERRORS);
-        options.add(TIMESTAMP_FORMAT);
-        options.add(JSON_MAP_NULL_KEY_MODE);
-        options.add(JSON_MAP_NULL_KEY_LITERAL);
-        options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
-        return options;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java
deleted file mode 100644
index feeca79..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.formats.json.JsonFormatOptions;
-
-/** Option utils for ogg-json format. */
-@PublicEvolving
-public class OggJsonFormatOptions {
-
-    public static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS =
-            JsonFormatOptions.IGNORE_PARSE_ERRORS;
-
-    public static final ConfigOption<String> TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_MODE =
-            JsonFormatOptions.MAP_NULL_KEY_MODE;
-
-    public static final ConfigOption<String> JSON_MAP_NULL_KEY_LITERAL =
-            JsonFormatOptions.MAP_NULL_KEY_LITERAL;
-
-    private OggJsonFormatOptions() {}
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java
deleted file mode 100644
index 635ff3d..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
-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 java.util.Objects;
-
-import static java.lang.String.format;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
-
-/**
- * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Ogg JSON.
- *
- * @see <a
- *     href="https://docs.oracle.com/goldengate/bd1221/gg-bd/GADBD/GUID-F0FA2781-0802-4530-B1F0-5E102B982EC0.htm#GADBD505">Ogg
- *     JSON Message</a>
- */
-public class OggJsonSerializationSchema implements SerializationSchema<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    private static final StringData OP_INSERT = StringData.fromString("I"); // insert
-    private static final StringData OP_DELETE = StringData.fromString("D"); // delete
-
-    /** The serializer to serialize Ogg JSON data. * */
-    private final JsonRowDataSerializationSchema jsonSerializer;
-
-    private transient GenericRowData genericRowData;
-
-    public OggJsonSerializationSchema(
-            RowType rowType,
-            TimestampFormat timestampFormat,
-            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
-            String mapNullKeyLiteral,
-            boolean encodeDecimalAsPlainNumber) {
-        jsonSerializer =
-                new JsonRowDataSerializationSchema(
-                        createJsonRowType(fromLogicalToDataType(rowType)),
-                        timestampFormat,
-                        mapNullKeyMode,
-                        mapNullKeyLiteral,
-                        encodeDecimalAsPlainNumber);
-    }
-
-    private static RowType createJsonRowType(DataType databaseSchema) {
-        // Ogg JSON contains some other information, e.g. "source", "ts_ms"
-        // but we don't need them.
-        return (RowType)
-                DataTypes.ROW(
-                                DataTypes.FIELD("before", databaseSchema),
-                                DataTypes.FIELD("after", databaseSchema),
-                                DataTypes.FIELD("op_type", DataTypes.STRING()))
-                        .getLogicalType();
-    }
-
-    @Override
-    public void open(InitializationContext context) throws Exception {
-        jsonSerializer.open(context);
-        genericRowData = new GenericRowData(3);
-    }
-
-    @Override
-    public byte[] serialize(RowData rowData) {
-        try {
-            switch (rowData.getRowKind()) {
-                case INSERT:
-                case UPDATE_AFTER:
-                    genericRowData.setField(0, null);
-                    genericRowData.setField(1, rowData);
-                    genericRowData.setField(2, OP_INSERT);
-                    return jsonSerializer.serialize(genericRowData);
-                case UPDATE_BEFORE:
-                case DELETE:
-                    genericRowData.setField(0, rowData);
-                    genericRowData.setField(1, null);
-                    genericRowData.setField(2, OP_DELETE);
-                    return jsonSerializer.serialize(genericRowData);
-                default:
-                    throw new UnsupportedOperationException(
-                            format(
-                                    "Unsupported operation '%s' for row kind.",
-                                    rowData.getRowKind()));
-            }
-        } catch (Throwable t) {
-            throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        OggJsonSerializationSchema that = (OggJsonSerializationSchema) o;
-        return Objects.equals(jsonSerializer, that.jsonSerializer);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(jsonSerializer);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 35e647d..3b83658 100644
--- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,8 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-org.apache.flink.formats.json.JsonFormatFactory
 org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory
-org.apache.flink.formats.json.canal.CanalJsonFormatFactory
-org.apache.flink.formats.json.maxwell.MaxwellJsonFormatFactory
-org.apache.flink.formats.json.ogg.OggJsonFormatFactory
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java
deleted file mode 100644
index b2efe8c..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase;
-import org.apache.flink.table.utils.LegacyRowResource;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.FileUtils;
-
-import org.junit.Rule;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** ITCase to test json format for {@link JsonFormatFactory}. */
-public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase {
-
-    @Rule public final LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE;
-
-    @Override
-    public String[] formatProperties() {
-        List<String> ret = new ArrayList<>();
-        ret.add("'format'='json'");
-        ret.add("'json.ignore-parse-errors'='true'");
-        return ret.toArray(new String[0]);
-    }
-
-    @Test
-    public void testParseError() throws Exception {
-        String path = new URI(resultPath()).getPath();
-        new File(path).mkdirs();
-        File file = new File(path, "temp_file");
-        file.createNewFile();
-        FileUtils.writeFileUtf8(
-                file,
-                "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}\n"
-                        + "{I am a wrong json.}\n"
-                        + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}");
-
-        check(
-                "select * from nonPartitionedTable",
-                Arrays.asList(Row.of("x5,5,1,1"), Row.of("x5,5,1,1")));
-    }
-
-    @Test
-    public void bigDataTest() throws IOException {
-        int numRecords = 1000;
-        File dir = generateTestData(numRecords);
-
-        env().setParallelism(1);
-
-        String sql =
-                String.format(
-                        "CREATE TABLE bigdata_source ( "
-                                + "	id INT, "
-                                + "	content STRING"
-                                + ") PARTITIONED by (id) WITH ("
-                                + "	'connector' = 'filesystem',"
-                                + "	'path' = '%s',"
-                                + "	'format' = 'json'"
-                                + ")",
-                        dir);
-        tEnv().executeSql(sql);
-        TableResult result = tEnv().executeSql("select * from bigdata_source");
-        List<String> elements = new ArrayList<>();
-        result.collect().forEachRemaining(r -> elements.add((String) r.getField(1)));
-        assertThat(elements).hasSize(numRecords);
-        elements.sort(String::compareTo);
-
-        List<String> expected = new ArrayList<>();
-        for (int i = 0; i < numRecords; i++) {
-            expected.add(String.valueOf(i));
-        }
-        expected.sort(String::compareTo);
-
-        assertThat(elements).isEqualTo(expected);
-    }
-
-    private static File generateTestData(int numRecords) throws IOException {
-        File tempDir = TEMPORARY_FOLDER.newFolder();
-
-        File root = new File(tempDir, "id=0");
-        root.mkdir();
-
-        File dataFile = new File(root, "testdata");
-        try (PrintWriter writer = new PrintWriter(dataFile)) {
-            for (int i = 0; i < numRecords; ++i) {
-                writer.println(String.format("{\"content\":\"%s\"}", i));
-            }
-        }
-
-        return tempDir;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java
deleted file mode 100644
index 9e2cde3..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.table.planner.runtime.stream.sql.FileCompactionITCaseBase;
-
-/** Compaction it case for json. */
-public class JsonFileCompactionITCase extends FileCompactionITCaseBase {
-
-    @Override
-    protected String format() {
-        return "json";
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java
deleted file mode 100644
index 7c1c553..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.table.api.ValidationException;
-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.TestDynamicTableFactory;
-import org.apache.flink.table.factories.utils.FactoryMocks;
-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.assertj.core.api.AbstractThrowableAssert;
-import org.junit.jupiter.api.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for the {@link JsonFormatFactory}. */
-class JsonFormatFactoryTest {
-
-    @Test
-    void testSeDeSchema() {
-        final Map<String, String> tableOptions = getAllOptions();
-
-        testSchemaSerializationSchema(tableOptions);
-        testSchemaDeserializationSchema(tableOptions);
-    }
-
-    @Test
-    void testFailOnMissingField() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(options -> options.put("json.fail-on-missing-field", "true"));
-
-        assertThatCreateRuntimeDecoder(tableOptions)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "fail-on-missing-field and ignore-parse-errors shouldn't both be true."));
-    }
-
-    @Test
-    void testInvalidOptionForIgnoreParseErrors() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(options -> options.put("json.ignore-parse-errors", "abc"));
-
-        assertThatCreateRuntimeDecoder(tableOptions)
-                .satisfies(
-                        anyCauseMatches(
-                                IllegalArgumentException.class,
-                                "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"));
-    }
-
-    @Test
-    void testInvalidOptionForTimestampFormat() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(options -> options.put("json.timestamp-format.standard", "test"));
-
-        assertThatCreateRuntimeDecoder(tableOptions)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."));
-    }
-
-    @Test
-    void testLowerCaseOptionForTimestampFormat() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(
-                        options -> options.put("json.timestamp-format.standard", "iso-8601"));
-
-        assertThatCreateRuntimeDecoder(tableOptions)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."));
-    }
-
-    @Test
-    void testInvalidOptionForMapNullKeyMode() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(options -> options.put("json.map-null-key.mode", "invalid"));
-
-        assertThatCreateRuntimeEncoder(tableOptions)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."));
-    }
-
-    @Test
-    void testLowerCaseOptionForMapNullKeyMode() {
-        final Map<String, String> tableOptions =
-                getModifyOptions(options -> options.put("json.map-null-key.mode", "fail"));
-
-        testSchemaDeserializationSchema(tableOptions);
-    }
-
-    // ------------------------------------------------------------------------
-    //  Utilities
-    // ------------------------------------------------------------------------
-
-    private AbstractThrowableAssert<?, ? extends Throwable> assertThatCreateRuntimeDecoder(
-            Map<String, String> options) {
-        return assertThatThrownBy(
-                () ->
-                        createTableSource(options)
-                                .valueFormat
-                                .createRuntimeDecoder(
-                                        ScanRuntimeProviderContext.INSTANCE,
-                                        SCHEMA.toPhysicalRowDataType()));
-    }
-
-    private AbstractThrowableAssert<?, ? extends Throwable> assertThatCreateRuntimeEncoder(
-            Map<String, String> options) {
-        return assertThatThrownBy(
-                () ->
-                        createTableSink(options)
-                                .valueFormat
-                                .createRuntimeEncoder(
-                                        new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE));
-    }
-
-    private void testSchemaDeserializationSchema(Map<String, String> options) {
-        final JsonRowDataDeserializationSchema expectedDeser =
-                new JsonRowDataDeserializationSchema(
-                        PHYSICAL_TYPE,
-                        InternalTypeInfo.of(PHYSICAL_TYPE),
-                        false,
-                        true,
-                        TimestampFormat.ISO_8601);
-
-        DeserializationSchema<RowData> actualDeser =
-                createTableSource(options)
-                        .valueFormat
-                        .createRuntimeDecoder(
-                                ScanRuntimeProviderContext.INSTANCE,
-                                SCHEMA.toPhysicalRowDataType());
-
-        assertThat(actualDeser).isEqualTo(expectedDeser);
-    }
-
-    private void testSchemaSerializationSchema(Map<String, String> options) {
-        final JsonRowDataSerializationSchema expectedSer =
-                new JsonRowDataSerializationSchema(
-                        PHYSICAL_TYPE,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-
-        SerializationSchema<RowData> actualSer =
-                createTableSink(options)
-                        .valueFormat
-                        .createRuntimeEncoder(
-                                new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE);
-
-        assertThat(actualSer).isEqualTo(expectedSer);
-    }
-
-    private TestDynamicTableFactory.DynamicTableSinkMock createTableSink(
-            Map<String, String> options) {
-        final DynamicTableSink actualSink = FactoryMocks.createTableSink(SCHEMA, options);
-        assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class);
-
-        return (TestDynamicTableFactory.DynamicTableSinkMock) actualSink;
-    }
-
-    private TestDynamicTableFactory.DynamicTableSourceMock createTableSource(
-            Map<String, String> options) {
-        final DynamicTableSource actualSource = FactoryMocks.createTableSource(SCHEMA, options);
-        assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class);
-
-        return (TestDynamicTableFactory.DynamicTableSourceMock) actualSource;
-    }
-
-    /**
-     * Returns the full options modified by the given consumer {@code optionModifier}.
-     *
-     * @param optionModifier Consumer to modify the options
-     */
-    private Map<String, String> getModifyOptions(Consumer<Map<String, String>> optionModifier) {
-        Map<String, String> options = getAllOptions();
-        optionModifier.accept(options);
-        return options;
-    }
-
-    private Map<String, String> getAllOptions() {
-        final Map<String, String> options = new HashMap<>();
-        options.put("connector", TestDynamicTableFactory.IDENTIFIER);
-        options.put("target", "MyTarget");
-        options.put("buffer-size", "1000");
-
-        options.put("format", JsonFormatFactory.IDENTIFIER);
-        options.put("json.fail-on-missing-field", "false");
-        options.put("json.ignore-parse-errors", "true");
-        options.put("json.timestamp-format.standard", "ISO-8601");
-        options.put("json.map-null-key.mode", "LITERAL");
-        options.put("json.map-null-key.literal", "null");
-        options.put("json.encode.decimal-as-plain-number", "true");
-        return options;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java
deleted file mode 100644
index 253707a..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.table.planner.runtime.stream.FsStreamingSinkITCaseBase;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/** Test checkpoint for file system table factory with json format. */
-public class JsonFsStreamSinkITCase extends FsStreamingSinkITCaseBase {
-
-    @Override
-    public String[] additionalProperties() {
-        List<String> ret = new ArrayList<>();
-        ret.add("'format'='json'");
-        // for test purpose
-        ret.add("'sink.rolling-policy.file-size'='1b'");
-        return ret.toArray(new String[0]);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java
deleted file mode 100644
index e6b2a3e..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.connector.testutils.formats.DummyInitializationContext;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.IOException;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the {@link JsonNodeDeserializationSchema}. */
-@SuppressWarnings("deprecation")
-class JsonNodeDeserializationSchemaTest {
-
-    @Test
-    void testDeserialize() throws IOException {
-        ObjectMapper mapper = JacksonMapperFactory.createObjectMapper();
-        ObjectNode initialValue = mapper.createObjectNode();
-        initialValue.put("key", 4).put("value", "world");
-        byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
-
-        JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema();
-        schema.open(new DummyInitializationContext());
-        ObjectNode deserializedValue = schema.deserialize(serializedValue);
-
-        assertThat(deserializedValue.get("key").asInt()).isEqualTo(4);
-        assertThat(deserializedValue.get("value").asText()).isEqualTo("world");
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java
deleted file mode 100644
index 883c3f0..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java
+++ /dev/null
@@ -1,872 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.connector.testutils.formats.DummyInitializationContext;
-import org.apache.flink.core.testutils.FlinkAssertions;
-import org.apache.flink.formats.common.TimestampFormat;
-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 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.types.Row;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.junit.jupiter.api.Test;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.ZoneOffset;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.api.DataTypes.ARRAY;
-import static org.apache.flink.table.api.DataTypes.BIGINT;
-import static org.apache.flink.table.api.DataTypes.BOOLEAN;
-import static org.apache.flink.table.api.DataTypes.BYTES;
-import static org.apache.flink.table.api.DataTypes.DATE;
-import static org.apache.flink.table.api.DataTypes.DECIMAL;
-import static org.apache.flink.table.api.DataTypes.DOUBLE;
-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.MAP;
-import static org.apache.flink.table.api.DataTypes.MULTISET;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.SMALLINT;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.apache.flink.table.api.DataTypes.TIME;
-import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
-import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
-import static org.apache.flink.table.api.DataTypes.TINYINT;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/**
- * Tests for {@link JsonRowDataDeserializationSchema} and {@link JsonRowDataSerializationSchema}.
- */
-class JsonRowDataSerDeSchemaTest {
-
-    private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper();
-
-    @Test
-    void testSerDe() throws Exception {
-        byte tinyint = 'c';
-        short smallint = 128;
-        int intValue = 45536;
-        float floatValue = 33.333F;
-        long bigint = 1238123899121L;
-        String name = "asdlkjasjkdla998y1122";
-        byte[] bytes = new byte[1024];
-        ThreadLocalRandom.current().nextBytes(bytes);
-        BigDecimal decimal = new BigDecimal("123.456789");
-        Double[] doubles = new Double[] {1.1, 2.2, 3.3};
-        LocalDate date = LocalDate.parse("1990-10-14");
-        LocalTime time = LocalTime.parse("12:12:43");
-        Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123");
-        Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789");
-        Instant timestampWithLocalZone =
-                LocalDateTime.of(1990, 10, 14, 12, 12, 43, 123456789)
-                        .atOffset(ZoneOffset.of("Z"))
-                        .toInstant();
-
-        Map<String, Long> map = new HashMap<>();
-        map.put("element", 123L);
-
-        Map<String, Integer> multiSet = new HashMap<>();
-        multiSet.put("element", 2);
-
-        Map<String, Map<String, Integer>> nestedMap = new HashMap<>();
-        Map<String, Integer> innerMap = new HashMap<>();
-        innerMap.put("key", 234);
-        nestedMap.put("inner_map", innerMap);
-
-        ArrayNode doubleNode = OBJECT_MAPPER.createArrayNode().add(1.1D).add(2.2D).add(3.3D);
-
-        // Root
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("bool", true);
-        root.put("tinyint", tinyint);
-        root.put("smallint", smallint);
-        root.put("int", intValue);
-        root.put("bigint", bigint);
-        root.put("float", floatValue);
-        root.put("name", name);
-        root.put("bytes", bytes);
-        root.put("decimal", decimal);
-        root.set("doubles", doubleNode);
-        root.put("date", "1990-10-14");
-        root.put("time", "12:12:43");
-        root.put("timestamp3", "1990-10-14T12:12:43.123");
-        root.put("timestamp9", "1990-10-14T12:12:43.123456789");
-        root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z");
-        root.putObject("map").put("element", 123);
-        root.putObject("multiSet").put("element", 2);
-        root.putObject("map2map").putObject("inner_map").put("key", 234);
-
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        DataType dataType =
-                ROW(
-                        FIELD("bool", BOOLEAN()),
-                        FIELD("tinyint", TINYINT()),
-                        FIELD("smallint", SMALLINT()),
-                        FIELD("int", INT()),
-                        FIELD("bigint", BIGINT()),
-                        FIELD("float", FLOAT()),
-                        FIELD("name", STRING()),
-                        FIELD("bytes", BYTES()),
-                        FIELD("decimal", DECIMAL(9, 6)),
-                        FIELD("doubles", ARRAY(DOUBLE())),
-                        FIELD("date", DATE()),
-                        FIELD("time", TIME(0)),
-                        FIELD("timestamp3", TIMESTAMP(3)),
-                        FIELD("timestamp9", TIMESTAMP(9)),
-                        FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)),
-                        FIELD("map", MAP(STRING(), BIGINT())),
-                        FIELD("multiSet", MULTISET(STRING())),
-                        FIELD("map2map", MAP(STRING(), MAP(STRING(), INT()))));
-        RowType schema = (RowType) dataType.getLogicalType();
-        TypeInformation<RowData> resultTypeInfo = InternalTypeInfo.of(schema);
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        Row expected = new Row(18);
-        expected.setField(0, true);
-        expected.setField(1, tinyint);
-        expected.setField(2, smallint);
-        expected.setField(3, intValue);
-        expected.setField(4, bigint);
-        expected.setField(5, floatValue);
-        expected.setField(6, name);
-        expected.setField(7, bytes);
-        expected.setField(8, decimal);
-        expected.setField(9, doubles);
-        expected.setField(10, date);
-        expected.setField(11, time);
-        expected.setField(12, timestamp3.toLocalDateTime());
-        expected.setField(13, timestamp9.toLocalDateTime());
-        expected.setField(14, timestampWithLocalZone);
-        expected.setField(15, map);
-        expected.setField(16, multiSet);
-        expected.setField(17, nestedMap);
-
-        RowData rowData = deserializationSchema.deserialize(serializedJson);
-        Row actual = convertToExternal(rowData, dataType);
-        assertThat(actual).isEqualTo(expected);
-
-        // test serialization
-        JsonRowDataSerializationSchema serializationSchema =
-                new JsonRowDataSerializationSchema(
-                        schema,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        open(serializationSchema);
-
-        byte[] actualBytes = serializationSchema.serialize(rowData);
-        assertThat(serializedJson).containsExactly(actualBytes);
-    }
-
-    /**
-     * Tests the deserialization slow path, e.g. convert into string and use {@link
-     * Double#parseDouble(String)}.
-     */
-    @Test
-    void testSlowDeserialization() throws Exception {
-        Random random = new Random();
-        boolean bool = random.nextBoolean();
-        int integer = random.nextInt();
-        long bigint = random.nextLong();
-        double doubleValue = random.nextDouble();
-        float floatValue = random.nextFloat();
-
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("bool", String.valueOf(bool));
-        root.put("int", String.valueOf(integer));
-        root.put("bigint", String.valueOf(bigint));
-        root.put("double1", String.valueOf(doubleValue));
-        root.put("double2", new BigDecimal(doubleValue));
-        root.put("float1", String.valueOf(floatValue));
-        root.put("float2", new BigDecimal(floatValue));
-
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        DataType dataType =
-                ROW(
-                        FIELD("bool", BOOLEAN()),
-                        FIELD("int", INT()),
-                        FIELD("bigint", BIGINT()),
-                        FIELD("double1", DOUBLE()),
-                        FIELD("double2", DOUBLE()),
-                        FIELD("float1", FLOAT()),
-                        FIELD("float2", FLOAT()));
-        RowType rowType = (RowType) dataType.getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        rowType,
-                        InternalTypeInfo.of(rowType),
-                        false,
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        Row expected = new Row(7);
-        expected.setField(0, bool);
-        expected.setField(1, integer);
-        expected.setField(2, bigint);
-        expected.setField(3, doubleValue);
-        expected.setField(4, doubleValue);
-        expected.setField(5, floatValue);
-        expected.setField(6, floatValue);
-
-        RowData rowData = deserializationSchema.deserialize(serializedJson);
-        Row actual = convertToExternal(rowData, dataType);
-        assertThat(actual).isEqualTo(expected);
-    }
-
-    @Test
-    void testSerDeMultiRows() throws Exception {
-        RowType rowType =
-                (RowType)
-                        ROW(
-                                        FIELD("f1", INT()),
-                                        FIELD("f2", BOOLEAN()),
-                                        FIELD("f3", STRING()),
-                                        FIELD("f4", MAP(STRING(), STRING())),
-                                        FIELD("f5", ARRAY(STRING())),
-                                        FIELD("f6", ROW(FIELD("f1", STRING()), FIELD("f2", INT()))))
-                                .getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        rowType,
-                        InternalTypeInfo.of(rowType),
-                        false,
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        JsonRowDataSerializationSchema serializationSchema =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        open(serializationSchema);
-
-        // the first row
-        {
-            ObjectNode root = OBJECT_MAPPER.createObjectNode();
-            root.put("f1", 1);
-            root.put("f2", true);
-            root.put("f3", "str");
-            ObjectNode map = root.putObject("f4");
-            map.put("hello1", "flink");
-            ArrayNode array = root.putArray("f5");
-            array.add("element1");
-            array.add("element2");
-            ObjectNode row = root.putObject("f6");
-            row.put("f1", "this is row1");
-            row.put("f2", 12);
-            byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-            RowData rowData = deserializationSchema.deserialize(serializedJson);
-            byte[] actual = serializationSchema.serialize(rowData);
-            assertThat(serializedJson).containsExactly(actual);
-        }
-
-        // the second row
-        {
-            ObjectNode root = OBJECT_MAPPER.createObjectNode();
-            root.put("f1", 10);
-            root.put("f2", false);
-            root.put("f3", "newStr");
-            ObjectNode map = root.putObject("f4");
-            map.put("hello2", "json");
-            ArrayNode array = root.putArray("f5");
-            array.add("element3");
-            array.add("element4");
-            ObjectNode row = root.putObject("f6");
-            row.put("f1", "this is row2");
-            row.putNull("f2");
-            byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-            RowData rowData = deserializationSchema.deserialize(serializedJson);
-            byte[] actual = serializationSchema.serialize(rowData);
-            assertThat(serializedJson).containsExactly(actual);
-        }
-    }
-
-    @Test
-    void testSerDeMultiRowsWithNullValues() throws Exception {
-        String[] jsons =
-                new String[] {
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, "
-                            + "\"ids\":[1, 2, 3]}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{}}",
-                };
-
-        String[] expected =
-                new String[] {
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{\"k1\":10.01,\"k2\":null}}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"},"
-                            + "\"ids\":[1,2,3],\"metrics\":null}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{}}",
-                };
-
-        RowType rowType =
-                (RowType)
-                        ROW(
-                                        FIELD("svt", STRING()),
-                                        FIELD("ops", ROW(FIELD("id", STRING()))),
-                                        FIELD("ids", ARRAY(INT())),
-                                        FIELD("metrics", MAP(STRING(), DOUBLE())))
-                                .getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        rowType,
-                        InternalTypeInfo.of(rowType),
-                        false,
-                        true,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        JsonRowDataSerializationSchema serializationSchema =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        open(serializationSchema);
-
-        for (int i = 0; i < jsons.length; i++) {
-            String json = jsons[i];
-            RowData row = deserializationSchema.deserialize(json.getBytes());
-            String result = new String(serializationSchema.serialize(row));
-            assertThat(result).isEqualTo(expected[i]);
-        }
-    }
-
-    @Test
-    void testDeserializationNullRow() throws Exception {
-        DataType dataType = ROW(FIELD("name", STRING()));
-        RowType schema = (RowType) dataType.getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        assertThat(deserializationSchema.deserialize(null)).isNull();
-    }
-
-    @Test
-    void testDeserializationMissingNode() throws Exception {
-        DataType dataType = ROW(FIELD("name", STRING()));
-        RowType schema = (RowType) dataType.getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        RowData rowData = deserializationSchema.deserialize("".getBytes());
-        assertThat(rowData).isNull();
-    }
-
-    @Test
-    void testDeserializationMissingField() throws Exception {
-        // Root
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("id", 123123123);
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        DataType dataType = ROW(FIELD("name", STRING()));
-        RowType schema = (RowType) dataType.getLogicalType();
-
-        // pass on missing field
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema,
-                        InternalTypeInfo.of(schema),
-                        false,
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        Row expected = new Row(1);
-        Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType);
-        assertThat(actual).isEqualTo(expected);
-
-        // fail on missing field
-        deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'.";
-
-        JsonRowDataDeserializationSchema finalDeserializationSchema = deserializationSchema;
-        assertThatThrownBy(() -> finalDeserializationSchema.deserialize(serializedJson))
-                .hasMessage(errorMessage);
-
-        // ignore on parse error
-        deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType);
-        assertThat(actual).isEqualTo(expected);
-
-        errorMessage =
-                "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled.";
-        assertThatThrownBy(
-                        () ->
-                                new JsonRowDataDeserializationSchema(
-                                        schema,
-                                        InternalTypeInfo.of(schema),
-                                        true,
-                                        true,
-                                        TimestampFormat.ISO_8601))
-                .hasMessage(errorMessage);
-    }
-
-    @Test
-    void testSerDeSQLTimestampFormat() throws Exception {
-        RowType rowType =
-                (RowType)
-                        ROW(
-                                        FIELD("timestamp3", TIMESTAMP(3)),
-                                        FIELD("timestamp9", TIMESTAMP(9)),
-                                        FIELD(
-                                                "timestamp_with_local_timezone3",
-                                                TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)),
-                                        FIELD(
-                                                "timestamp_with_local_timezone9",
-                                                TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)))
-                                .getLogicalType();
-
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL);
-        open(deserializationSchema);
-        JsonRowDataSerializationSchema serializationSchema =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        open(serializationSchema);
-
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("timestamp3", "1990-10-14 12:12:43.123");
-        root.put("timestamp9", "1990-10-14 12:12:43.123456789");
-        root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z");
-        root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z");
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-        RowData rowData = deserializationSchema.deserialize(serializedJson);
-        byte[] actual = serializationSchema.serialize(rowData);
-        assertThat(serializedJson).containsExactly(actual);
-    }
-
-    @Test
-    void testSerializationMapNullKey() {
-        RowType rowType =
-                (RowType)
-                        ROW(FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT()))))
-                                .getLogicalType();
-
-        // test data
-        // use LinkedHashMap to make sure entries order
-        Map<StringData, Integer> map = new LinkedHashMap<>();
-        map.put(StringData.fromString("no-null key"), 1);
-        map.put(StringData.fromString(null), 2);
-        GenericMapData mapData = new GenericMapData(map);
-
-        Map<StringData, GenericMapData> nestedMap = new LinkedHashMap<>();
-        nestedMap.put(StringData.fromString("no-null key"), mapData);
-        nestedMap.put(StringData.fromString(null), mapData);
-
-        GenericMapData nestedMapData = new GenericMapData(nestedMap);
-        GenericRowData rowData = new GenericRowData(1);
-        rowData.setField(0, nestedMapData);
-
-        JsonRowDataSerializationSchema serializationSchema1 =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.FAIL,
-                        "null",
-                        true);
-        open(serializationSchema1);
-        // expect message for serializationSchema1
-        String errorMessage1 =
-                "JSON format doesn't support to serialize map data with null keys."
-                        + " You can drop null key entries or encode null in literals by specifying map-null-key.mode option.";
-
-        JsonRowDataSerializationSchema serializationSchema2 =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.DROP,
-                        "null",
-                        true);
-        open(serializationSchema2);
-        // expect result for serializationSchema2
-        String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}";
-
-        JsonRowDataSerializationSchema serializationSchema3 =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "nullKey",
-                        true);
-        open(serializationSchema3);
-        // expect result for serializationSchema3
-        String expectResult3 =
-                "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}";
-
-        assertThatThrownBy(() -> serializationSchema1.serialize(rowData))
-                .satisfies(FlinkAssertions.anyCauseMatches(errorMessage1));
-
-        // mapNullKey Mode is drop
-        byte[] actual2 = serializationSchema2.serialize(rowData);
-        assertThat(new String(actual2)).isEqualTo(expectResult2);
-
-        // mapNullKey Mode is literal
-        byte[] actual3 = serializationSchema3.serialize(rowData);
-        assertThat(new String(actual3)).isEqualTo(expectResult3);
-    }
-
-    @Test
-    void testSerializationDecimalEncode() throws Exception {
-        RowType schema =
-                (RowType)
-                        ROW(
-                                        FIELD("decimal1", DECIMAL(9, 6)),
-                                        FIELD("decimal2", DECIMAL(20, 0)),
-                                        FIELD("decimal3", DECIMAL(11, 9)))
-                                .getLogicalType();
-
-        TypeInformation<RowData> resultTypeInfo = InternalTypeInfo.of(schema);
-
-        JsonRowDataDeserializationSchema deserializer =
-                new JsonRowDataDeserializationSchema(
-                        schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601);
-        deserializer.open(new DummyInitializationContext());
-
-        JsonRowDataSerializationSchema plainDecimalSerializer =
-                new JsonRowDataSerializationSchema(
-                        schema,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        plainDecimalSerializer.open(new DummyInitializationContext());
-        JsonRowDataSerializationSchema scientificDecimalSerializer =
-                new JsonRowDataSerializationSchema(
-                        schema,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        false);
-        scientificDecimalSerializer.open(new DummyInitializationContext());
-
-        String plainDecimalJson =
-                "{\"decimal1\":123.456789,\"decimal2\":454621864049246170,\"decimal3\":0.000000027}";
-        RowData rowData = deserializer.deserialize(plainDecimalJson.getBytes());
-
-        String plainDecimalResult = new String(plainDecimalSerializer.serialize(rowData));
-        assertThat(plainDecimalResult).isEqualTo(plainDecimalJson);
-
-        String scientificDecimalJson =
-                "{\"decimal1\":123.456789,\"decimal2\":4.5462186404924617E+17,\"decimal3\":2.7E-8}";
-
-        String scientificDecimalResult = new String(scientificDecimalSerializer.serialize(rowData));
-        assertThat(scientificDecimalResult).isEqualTo(scientificDecimalJson);
-    }
-
-    @Test
-    void testJsonParse() throws Exception {
-        for (TestSpec spec : testData) {
-            testIgnoreParseErrors(spec);
-            if (spec.errorMessage != null) {
-                testParseErrors(spec);
-            }
-        }
-    }
-
-    @Test
-    void testSerializationWithTypesMismatch() {
-        RowType rowType = (RowType) ROW(FIELD("f0", INT()), FIELD("f1", STRING())).getLogicalType();
-        GenericRowData genericRowData = new GenericRowData(2);
-        genericRowData.setField(0, 1);
-        genericRowData.setField(1, 1);
-        JsonRowDataSerializationSchema serializationSchema =
-                new JsonRowDataSerializationSchema(
-                        rowType,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.FAIL,
-                        "null",
-                        true);
-        open(serializationSchema);
-        String errorMessage = "Fail to serialize at field: f1.";
-
-        assertThatThrownBy(() -> serializationSchema.serialize(genericRowData))
-                .satisfies(anyCauseMatches(RuntimeException.class, errorMessage));
-    }
-
-    @Test
-    void testDeserializationWithTypesMismatch() {
-        RowType rowType = (RowType) ROW(FIELD("f0", STRING()), FIELD("f1", INT())).getLogicalType();
-        String json = "{\"f0\":\"abc\", \"f1\": \"abc\"}";
-        JsonRowDataDeserializationSchema deserializationSchema =
-                new JsonRowDataDeserializationSchema(
-                        rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL);
-        open(deserializationSchema);
-        String errorMessage = "Fail to deserialize at field: f1.";
-
-        assertThatThrownBy(() -> deserializationSchema.deserialize(json.getBytes()))
-                .satisfies(anyCauseMatches(errorMessage));
-    }
-
-    private void testIgnoreParseErrors(TestSpec spec) throws Exception {
-        // the parsing field should be null and no exception is thrown
-        JsonRowDataDeserializationSchema ignoreErrorsSchema =
-                new JsonRowDataDeserializationSchema(
-                        spec.rowType,
-                        InternalTypeInfo.of(spec.rowType),
-                        false,
-                        true,
-                        spec.timestampFormat);
-        ignoreErrorsSchema.open(new DummyInitializationContext());
-
-        Row expected;
-        if (spec.expected != null) {
-            expected = spec.expected;
-        } else {
-            expected = new Row(1);
-        }
-        RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes());
-        Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType));
-        assertThat(actual)
-                .isEqualTo(expected)
-                .withFailMessage("Test Ignore Parse Error: " + spec.json);
-    }
-
-    private void testParseErrors(TestSpec spec) {
-        // expect exception if parse error is not ignored
-        JsonRowDataDeserializationSchema failingSchema =
-                new JsonRowDataDeserializationSchema(
-                        spec.rowType,
-                        InternalTypeInfo.of(spec.rowType),
-                        false,
-                        false,
-                        spec.timestampFormat);
-        open(failingSchema);
-
-        assertThatThrownBy(() -> failingSchema.deserialize(spec.json.getBytes()))
-                .hasMessageContaining(spec.errorMessage);
-    }
-
-    private static List<TestSpec> testData =
-            Arrays.asList(
-                    TestSpec.json("{\"id\": \"trueA\"}")
-                            .rowType(ROW(FIELD("id", BOOLEAN())))
-                            .expect(Row.of(false)),
-                    TestSpec.json("{\"id\": true}")
-                            .rowType(ROW(FIELD("id", BOOLEAN())))
-                            .expect(Row.of(true)),
-                    TestSpec.json("{\"id\":\"abc\"}")
-                            .rowType(ROW(FIELD("id", INT())))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."),
-                    TestSpec.json("{\"id\":112.013}")
-                            .rowType(ROW(FIELD("id", BIGINT())))
-                            .expect(Row.of(112L)),
-                    TestSpec.json("{\"id\":\"long\"}")
-                            .rowType(ROW(FIELD("id", BIGINT())))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'."),
-                    TestSpec.json("{\"id\":\"112.013.123\"}")
-                            .rowType(ROW(FIELD("id", FLOAT())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."),
-                    TestSpec.json("{\"id\":\"112.013.123\"}")
-                            .rowType(ROW(FIELD("id", DOUBLE())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."),
-                    TestSpec.json("{\"id\":\"18:00:243\"}")
-                            .rowType(ROW(FIELD("id", TIME())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."),
-                    TestSpec.json("{\"id\":\"18:00:243\"}")
-                            .rowType(ROW(FIELD("id", TIME())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."),
-                    TestSpec.json("{\"id\":\"20191112\"}")
-                            .rowType(ROW(FIELD("id", DATE())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"20191112\"}'."),
-                    TestSpec.json("{\"id\":\"20191112\"}")
-                            .rowType(ROW(FIELD("id", DATE())))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"20191112\"}'."),
-                    TestSpec.json("{\"id\":true}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("true")),
-                    TestSpec.json("{\"id\":123.234}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("123.234")),
-                    TestSpec.json("{\"id\":1234567}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("1234567")),
-                    TestSpec.json("{\"id\":\"string field\"}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("string field")),
-                    TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")),
-                    TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}")
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")),
-                    TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP(0))))
-                            .timestampFormat(TimestampFormat.ISO_8601)
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'."),
-                    TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP(0))))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."),
-                    TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP(0))))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."),
-                    TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP(0))))
-                            .timestampFormat(TimestampFormat.ISO_8601)
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."),
-                    TestSpec.json("{\"id\":\"abc\"}")
-                            .rowType(ROW(FIELD("id", DECIMAL(10, 3))))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."),
-                    TestSpec.json("{\"row\":{\"id\":\"abc\"}}")
-                            .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN())))))
-                            .expect(Row.of(Row.of(false))),
-                    TestSpec.json("{\"array\":[123, \"abc\"]}")
-                            .rowType(ROW(FIELD("array", ARRAY(INT()))))
-                            .expect(Row.of((Object) new Integer[] {123, null}))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'."),
-                    TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}")
-                            .rowType(ROW(FIELD("map", MAP(STRING(), INT()))))
-                            .expect(Row.of(createHashMap("key1", 123, "key2", null)))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."),
-                    TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0))))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."),
-                    TestSpec.json("{\"id\":\"2019-11-12T18:00:12+0800\"}")
-                            .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0))))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."),
-                    TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}")
-                            .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18))))
-                            .expect(Row.of(1, new BigDecimal("799.929496989092949698"))),
-                    TestSpec.json("{\"id\":\"\tstring field\"}") // test to parse control chars
-                            .rowType(ROW(FIELD("id", STRING())))
-                            .expect(Row.of("\tstring field")));
-
-    private static Map<String, Integer> createHashMap(
-            String k1, Integer v1, String k2, Integer v2) {
-        Map<String, Integer> map = new HashMap<>();
-        map.put(k1, v1);
-        map.put(k2, v2);
-        return map;
-    }
-
-    @SuppressWarnings("unchecked")
-    private static Row convertToExternal(RowData rowData, DataType dataType) {
-        return (Row) DataFormatConverters.getConverterForDataType(dataType).toExternal(rowData);
-    }
-
-    private static class TestSpec {
-        private final String json;
-        private RowType rowType;
-        private TimestampFormat timestampFormat = TimestampFormat.SQL;
-        private Row expected;
-        private String errorMessage;
-
-        private TestSpec(String json) {
-            this.json = json;
-        }
-
-        public static TestSpec json(String json) {
-            return new TestSpec(json);
-        }
-
-        TestSpec expect(Row row) {
-            this.expected = row;
-            return this;
-        }
-
-        TestSpec rowType(DataType rowType) {
-            this.rowType = (RowType) rowType.getLogicalType();
-            return this;
-        }
-
-        TestSpec expectErrorMessage(String errorMessage) {
-            this.errorMessage = errorMessage;
-            return this;
-        }
-
-        TestSpec timestampFormat(TimestampFormat timestampFormat) {
-            this.timestampFormat = timestampFormat;
-            return this;
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java
deleted file mode 100644
index 81e370c..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java
+++ /dev/null
@@ -1,432 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import javax.annotation.Nullable;
-
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.junit.Assert.assertThat;
-import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause;
-import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
-
-/** Tests for the {@link JsonRowDeserializationSchema}. */
-public class JsonRowDeserializationSchemaTest {
-
-    private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper();
-
-    @Rule public ExpectedException thrown = ExpectedException.none();
-
-    /** Tests simple deserialization using type information. */
-    @Test
-    public void testTypeInfoDeserialization() throws Exception {
-        long id = 1238123899121L;
-        String name = "asdlkjasjkdla998y1122";
-        byte[] bytes = new byte[1024];
-        ThreadLocalRandom.current().nextBytes(bytes);
-        Timestamp timestamp = Timestamp.valueOf("1990-10-14 12:12:43");
-        Date date = Date.valueOf("1990-10-14");
-        Time time = Time.valueOf("12:12:43");
-
-        Map<String, Long> map = new HashMap<>();
-        map.put("flink", 123L);
-
-        Map<String, Map<String, Integer>> nestedMap = new HashMap<>();
-        Map<String, Integer> innerMap = new HashMap<>();
-        innerMap.put("key", 234);
-        nestedMap.put("inner_map", innerMap);
-
-        // Root
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("id", id);
-        root.put("name", name);
-        root.put("bytes", bytes);
-        root.put("date1", "1990-10-14");
-        root.put("date2", "1990-10-14");
-        root.put("time1", "12:12:43Z");
-        root.put("time2", "12:12:43Z");
-        root.put("timestamp1", "1990-10-14T12:12:43Z");
-        root.put("timestamp2", "1990-10-14T12:12:43Z");
-        root.putObject("map").put("flink", 123);
-        root.putObject("map2map").putObject("inner_map").put("key", 234);
-
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(
-                                Types.ROW_NAMED(
-                                        new String[] {
-                                            "id",
-                                            "name",
-                                            "bytes",
-                                            "date1",
-                                            "date2",
-                                            "time1",
-                                            "time2",
-                                            "timestamp1",
-                                            "timestamp2",
-                                            "map",
-                                            "map2map"
-                                        },
-                                        Types.LONG,
-                                        Types.STRING,
-                                        Types.PRIMITIVE_ARRAY(Types.BYTE),
-                                        Types.SQL_DATE,
-                                        Types.LOCAL_DATE,
-                                        Types.SQL_TIME,
-                                        Types.LOCAL_TIME,
-                                        Types.SQL_TIMESTAMP,
-                                        Types.LOCAL_DATE_TIME,
-                                        Types.MAP(Types.STRING, Types.LONG),
-                                        Types.MAP(
-                                                Types.STRING, Types.MAP(Types.STRING, Types.INT))))
-                        .build();
-
-        Row row = new Row(11);
-        row.setField(0, id);
-        row.setField(1, name);
-        row.setField(2, bytes);
-        row.setField(3, date);
-        row.setField(4, date.toLocalDate());
-        row.setField(5, time);
-        row.setField(6, time.toLocalTime());
-        row.setField(7, timestamp);
-        row.setField(8, timestamp.toLocalDateTime());
-        row.setField(9, map);
-        row.setField(10, nestedMap);
-
-        assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row));
-    }
-
-    @Test
-    public void testSchemaDeserialization() throws Exception {
-        final BigDecimal id = BigDecimal.valueOf(1238123899121L);
-        final String name = "asdlkjasjkdla998y1122";
-        final byte[] bytes = new byte[1024];
-        ThreadLocalRandom.current().nextBytes(bytes);
-        final BigDecimal[] numbers =
-                new BigDecimal[] {
-                    BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)
-                };
-        final String[] strings = new String[] {"one", "two", "three"};
-
-        // Root
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("id", id.longValue());
-        root.putNull("idOrNull");
-        root.put("name", name);
-        root.put("date", "1990-10-14");
-        root.put("time", "12:12:43Z");
-        root.put("timestamp", "1990-10-14T12:12:43Z");
-        root.put("bytes", bytes);
-        root.putArray("numbers").add(1).add(2).add(3);
-        root.putArray("strings").add("one").add("two").add("three");
-        root.putObject("nested").put("booleanField", true).put("decimalField", 12);
-
-        final byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(
-                                "{"
-                                        + "    type: 'object',"
-                                        + "    properties: {"
-                                        + "         id: { type: 'integer' },"
-                                        + "         idOrNull: { type: ['integer', 'null'] },"
-                                        + "         name: { type: 'string' },"
-                                        + "         date: { type: 'string', format: 'date' },"
-                                        + "         time: { type: 'string', format: 'time' },"
-                                        + "         timestamp: { type: 'string', format: 'date-time' },"
-                                        + "         bytes: { type: 'string', contentEncoding: 'base64' },"
-                                        + "         numbers: { type: 'array', items: { type: 'integer' } },"
-                                        + "         strings: { type: 'array', items: { type: 'string' } },"
-                                        + "         nested: { "
-                                        + "             type: 'object',"
-                                        + "             properties: { "
-                                        + "                 booleanField: { type: 'boolean' },"
-                                        + "                 decimalField: { type: 'number' }"
-                                        + "             }"
-                                        + "         }"
-                                        + "    }"
-                                        + "}")
-                        .build();
-
-        final Row expected = new Row(10);
-        expected.setField(0, id);
-        expected.setField(1, null);
-        expected.setField(2, name);
-        expected.setField(3, Date.valueOf("1990-10-14"));
-        expected.setField(4, Time.valueOf("12:12:43"));
-        expected.setField(5, Timestamp.valueOf("1990-10-14 12:12:43"));
-        expected.setField(6, bytes);
-        expected.setField(7, numbers);
-        expected.setField(8, strings);
-        final Row nestedRow = new Row(2);
-        nestedRow.setField(0, true);
-        nestedRow.setField(1, BigDecimal.valueOf(12));
-        expected.setField(9, nestedRow);
-
-        assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected));
-    }
-
-    /** Tests deserialization with non-existing field name. */
-    @Test
-    public void testMissingNode() throws Exception {
-        // Root
-        ObjectNode root = OBJECT_MAPPER.createObjectNode();
-        root.put("id", 123123123);
-        byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root);
-
-        TypeInformation<Row> rowTypeInformation =
-                Types.ROW_NAMED(new String[] {"name"}, Types.STRING);
-
-        JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowTypeInformation).build();
-
-        Row row = new Row(1);
-        assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row));
-
-        deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowTypeInformation)
-                        .failOnMissingField()
-                        .build();
-
-        assertThat(
-                serializedJson,
-                whenDeserializedWith(deserializationSchema)
-                        .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
-
-        // ignore-parse-errors ignores missing field exception too
-        deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowTypeInformation)
-                        .ignoreParseErrors()
-                        .build();
-        assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row));
-
-        thrown.expect(IllegalArgumentException.class);
-        thrown.expectMessage(
-                "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true");
-        new JsonRowDeserializationSchema.Builder(rowTypeInformation)
-                .failOnMissingField()
-                .ignoreParseErrors()
-                .build();
-    }
-
-    /** Tests that number of field names and types has to match. */
-    @Test
-    public void testNumberOfFieldNamesAndTypesMismatch() {
-        try {
-            new JsonRowDeserializationSchema.Builder(
-                            Types.ROW_NAMED(new String[] {"one", "two", "three"}, Types.LONG))
-                    .build();
-            Assert.fail("Did not throw expected Exception");
-        } catch (IllegalArgumentException ignored) {
-            // Expected
-        }
-    }
-
-    @Test
-    public void testJsonParse() {
-        for (TestSpec spec : testData) {
-            testIgnoreParseErrors(spec);
-            if (spec.errorMessage != null) {
-                testParseErrors(spec);
-            }
-        }
-    }
-
-    private void testIgnoreParseErrors(TestSpec spec) {
-        // the parsing field should be null and no exception is thrown
-        JsonRowDeserializationSchema ignoreErrorsSchema =
-                new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation)
-                        .ignoreParseErrors()
-                        .build();
-        Row expected;
-        if (spec.expected != null) {
-            expected = spec.expected;
-        } else {
-            expected = new Row(1);
-        }
-        assertThat(
-                "Test Ignore Parse Error: " + spec.json,
-                spec.json.getBytes(),
-                whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected));
-    }
-
-    private void testParseErrors(TestSpec spec) {
-        // expect exception if parse error is not ignored
-        JsonRowDeserializationSchema failingSchema =
-                new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation).build();
-        assertThat(
-                "Test Parse Error: " + spec.json,
-                spec.json.getBytes(),
-                whenDeserializedWith(failingSchema)
-                        .failsWithException(hasMessage(containsString(spec.errorMessage))));
-    }
-
-    private static List<TestSpec> testData =
-            Arrays.asList(
-                    TestSpec.json("{\"id\": \"trueA\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN))
-                            .expect(Row.of(false)),
-                    TestSpec.json("{\"id\": true}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN))
-                            .expect(Row.of(true)),
-                    TestSpec.json("{\"id\":\"abc\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.INT))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"),
-                    TestSpec.json("{\"id\":112.013}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG))
-                            .expect(Row.of(112L)),
-                    TestSpec.json("{\"id\":true}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("true")),
-                    TestSpec.json("{\"id\":123.234}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("123.234")),
-                    TestSpec.json("{\"id\":1234567}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("1234567")),
-                    TestSpec.json("{\"id\":\"string field\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("string field")),
-                    TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")),
-                    TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING))
-                            .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")),
-                    TestSpec.json("{\"id\":\"long\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"),
-                    TestSpec.json("{\"id\":\"112.013.123\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.FLOAT))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"),
-                    TestSpec.json("{\"id\":\"112.013.123\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.DOUBLE))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"),
-                    TestSpec.json("{\"id\":\"18:00:243\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIME))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"),
-                    TestSpec.json("{\"id\":\"20191112\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_DATE))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"20191112\"}'"),
-                    TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIMESTAMP))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"),
-                    TestSpec.json("{\"id\":\"abc\"}")
-                            .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BIG_DEC))
-                            .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"),
-                    TestSpec.json("{\"row\":{\"id\":\"abc\"}}")
-                            .typeInfo(
-                                    Types.ROW_NAMED(
-                                            new String[] {"row"},
-                                            Types.ROW_NAMED(new String[] {"id"}, Types.INT)))
-                            .expect(Row.of(new Row(1)))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"),
-                    TestSpec.json("{\"array\":[123, \"abc\"]}")
-                            .typeInfo(
-                                    Types.ROW_NAMED(
-                                            new String[] {"array"}, Types.OBJECT_ARRAY(Types.INT)))
-                            .expect(Row.of((Object) new Integer[] {123, null}))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"),
-                    TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}")
-                            .typeInfo(
-                                    Types.ROW_NAMED(
-                                            new String[] {"map"},
-                                            Types.MAP(Types.STRING, Types.INT)))
-                            .expect(Row.of(createHashMap("key1", 123, "key2", null)))
-                            .expectErrorMessage(
-                                    "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'"),
-                    TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}")
-                            .typeInfo(
-                                    Types.ROW_NAMED(
-                                            new String[] {"id", "factor"},
-                                            Types.INT,
-                                            Types.BIG_DEC))
-                            .expect(Row.of(1, new BigDecimal("799.929496989092949698"))));
-
-    private static Map<String, Integer> createHashMap(
-            String k1, Integer v1, String k2, Integer v2) {
-        Map<String, Integer> map = new HashMap<>();
-        map.put(k1, v1);
-        map.put(k2, v2);
-        return map;
-    }
-
-    private static class TestSpec {
-        private final String json;
-        private @Nullable TypeInformation<Row> rowTypeInformation;
-        private @Nullable Row expected;
-        private @Nullable String errorMessage;
-
-        private TestSpec(String json) {
-            this.json = json;
-        }
-
-        public static TestSpec json(String json) {
-            return new TestSpec(json);
-        }
-
-        TestSpec expect(Row row) {
-            this.expected = row;
-            return this;
-        }
-
-        TestSpec typeInfo(TypeInformation<Row> rowTypeInformation) {
-            this.rowTypeInformation = rowTypeInformation;
-            return this;
-        }
-
-        TestSpec expectErrorMessage(String errorMessage) {
-            this.errorMessage = errorMessage;
-            return this;
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java
deleted file mode 100644
index b15444d..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.util.FileUtils;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.File;
-import java.net.URL;
-import java.util.Objects;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link JsonRowSchemaConverter}. */
-class JsonRowSchemaConverterTest {
-
-    @Test
-    void testComplexSchema() throws Exception {
-        final URL url = getClass().getClassLoader().getResource("complex-schema.json");
-        Objects.requireNonNull(url);
-        final String schema = FileUtils.readFileUtf8(new File(url.getFile()));
-        final TypeInformation<?> result = JsonRowSchemaConverter.convert(schema);
-
-        final TypeInformation<?> expected =
-                Types.ROW_NAMED(
-                        new String[] {
-                            "fn",
-                            "familyName",
-                            "additionalName",
-                            "tuples",
-                            "honorificPrefix",
-                            "url",
-                            "email",
-                            "tel",
-                            "sound",
-                            "org"
-                        },
-                        Types.STRING,
-                        Types.STRING,
-                        Types.BOOLEAN,
-                        Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING),
-                        Types.OBJECT_ARRAY(Types.STRING),
-                        Types.STRING,
-                        Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING),
-                        Types.ROW_NAMED(
-                                new String[] {"type", "value"}, Types.BIG_DEC, Types.STRING),
-                        Types.VOID,
-                        Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW()));
-
-        assertThat(result).isEqualTo(expected);
-    }
-
-    @Test
-    void testReferenceSchema() throws Exception {
-        final URL url = getClass().getClassLoader().getResource("reference-schema.json");
-        Objects.requireNonNull(url);
-        final String schema = FileUtils.readFileUtf8(new File(url.getFile()));
-        final TypeInformation<?> result = JsonRowSchemaConverter.convert(schema);
-
-        final TypeInformation<?> expected =
-                Types.ROW_NAMED(
-                        new String[] {"billing_address", "shipping_address", "optional_address"},
-                        Types.ROW_NAMED(
-                                new String[] {"street_address", "city", "state"},
-                                Types.STRING,
-                                Types.STRING,
-                                Types.STRING),
-                        Types.ROW_NAMED(
-                                new String[] {"street_address", "city", "state"},
-                                Types.STRING,
-                                Types.STRING,
-                                Types.STRING),
-                        Types.ROW_NAMED(
-                                new String[] {"street_address", "city", "state"},
-                                Types.STRING,
-                                Types.STRING,
-                                Types.STRING));
-
-        assertThat(result).isEqualTo(expected);
-    }
-
-    @Test
-    void testAtomicType() {
-        final TypeInformation<?> result = JsonRowSchemaConverter.convert("{ type: 'number' }");
-
-        assertThat(result).isEqualTo(Types.BIG_DEC);
-    }
-
-    @Test
-    void testMissingType() {
-        assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ }"))
-                .isInstanceOf(IllegalArgumentException.class);
-    }
-
-    @Test
-    void testWrongType() {
-        assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ type: 'whatever' }"))
-                .isInstanceOf(IllegalArgumentException.class);
-    }
-
-    @Test
-    void testArrayWithAdditionalItems() {
-        assertThatThrownBy(
-                        () ->
-                                JsonRowSchemaConverter.convert(
-                                        "{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"))
-                .isInstanceOf(IllegalArgumentException.class);
-    }
-
-    @Test
-    void testMissingProperties() {
-        final TypeInformation<?> result = JsonRowSchemaConverter.convert("{ type: 'object' }");
-
-        assertThat(result).isEqualTo(Types.ROW());
-    }
-
-    @Test
-    void testNullUnionTypes() {
-        final TypeInformation<?> result =
-                JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }");
-
-        assertThat(result).isEqualTo(Types.STRING);
-    }
-
-    @Test
-    void testTimestamp() {
-        final TypeInformation<?> result =
-                JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }");
-
-        assertThat(result).isEqualTo(Types.SQL_TIMESTAMP);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java
deleted file mode 100644
index ac1fe18..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.types.Row;
-
-import org.junit.Test;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-import static org.apache.flink.formats.utils.SerializationSchemaMatcher.whenSerializedWith;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-/** Tests for the {@link JsonRowSerializationSchema}. */
-public class JsonRowSerializationSchemaTest {
-
-    @Test
-    public void testRowSerialization() {
-        final TypeInformation<Row> rowSchema =
-                Types.ROW_NAMED(
-                        new String[] {"f1", "f2", "f3", "f4", "f5"},
-                        Types.INT,
-                        Types.BOOLEAN,
-                        Types.STRING,
-                        Types.SQL_TIMESTAMP,
-                        Types.LOCAL_DATE_TIME);
-
-        final Row row = new Row(5);
-        row.setField(0, 1);
-        row.setField(1, true);
-        row.setField(2, "str");
-        row.setField(3, Timestamp.valueOf("1990-10-14 12:12:43"));
-        row.setField(4, Timestamp.valueOf("1990-10-14 12:12:43").toLocalDateTime());
-
-        final JsonRowSerializationSchema serializationSchema =
-                new JsonRowSerializationSchema.Builder(rowSchema).build();
-        final JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowSchema).build();
-
-        assertThat(
-                row,
-                whenSerializedWith(serializationSchema)
-                        .andDeserializedWith(deserializationSchema)
-                        .equalsTo(row));
-    }
-
-    @Test
-    public void testSerializationOfTwoRows() throws IOException {
-        final TypeInformation<Row> rowSchema =
-                Types.ROW_NAMED(
-                        new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING);
-
-        final Row row1 = new Row(3);
-        row1.setField(0, 1);
-        row1.setField(1, true);
-        row1.setField(2, "str");
-
-        final JsonRowSerializationSchema serializationSchema =
-                new JsonRowSerializationSchema.Builder(rowSchema).build();
-        open(serializationSchema);
-        final JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowSchema).build();
-        open(deserializationSchema);
-
-        byte[] bytes = serializationSchema.serialize(row1);
-        assertEquals(row1, deserializationSchema.deserialize(bytes));
-
-        final Row row2 = new Row(3);
-        row2.setField(0, 10);
-        row2.setField(1, false);
-        row2.setField(2, "newStr");
-
-        bytes = serializationSchema.serialize(row2);
-        assertEquals(row2, deserializationSchema.deserialize(bytes));
-    }
-
-    @Test
-    public void testMultiRowsWithNullValues() throws IOException {
-        String[] jsons =
-                new String[] {
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, "
-                            + "\"ids\":[1, 2, 3]}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}",
-                };
-
-        String[] expected =
-                new String[] {
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"},"
-                            + "\"ids\":[1,2,3]}",
-                    "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}",
-                };
-
-        TypeInformation<Row> schema =
-                Types.ROW_NAMED(
-                        new String[] {"svt", "ops", "ids"},
-                        Types.STRING,
-                        Types.ROW_NAMED(new String[] {"id"}, Types.STRING),
-                        Types.PRIMITIVE_ARRAY(Types.INT));
-        JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(schema).build();
-        open(deserializationSchema);
-        JsonRowSerializationSchema serializationSchema =
-                JsonRowSerializationSchema.builder().withTypeInfo(schema).build();
-        open(serializationSchema);
-
-        for (int i = 0; i < jsons.length; i++) {
-            String json = jsons[i];
-            Row row = deserializationSchema.deserialize(json.getBytes());
-            String result = new String(serializationSchema.serialize(row));
-            assertEquals(expected[i], result);
-        }
-    }
-
-    @Test
-    public void testNestedSchema() {
-        final TypeInformation<Row> rowSchema =
-                Types.ROW_NAMED(
-                        new String[] {"f1", "f2", "f3"},
-                        Types.INT,
-                        Types.BOOLEAN,
-                        Types.ROW(Types.INT, Types.DOUBLE));
-
-        final Row row = new Row(3);
-        row.setField(0, 42);
-        row.setField(1, false);
-        final Row nested = new Row(2);
-        nested.setField(0, 22);
-        nested.setField(1, 2.3);
-        row.setField(2, nested);
-
-        final JsonRowSerializationSchema serializationSchema =
-                new JsonRowSerializationSchema.Builder(rowSchema).build();
-        final JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowSchema).build();
-
-        assertThat(
-                row,
-                whenSerializedWith(serializationSchema)
-                        .andDeserializedWith(deserializationSchema)
-                        .equalsTo(row));
-    }
-
-    @Test
-    public void testSerializeRowWithInvalidNumberOfFields() {
-        final TypeInformation<Row> rowSchema =
-                Types.ROW_NAMED(
-                        new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING);
-
-        final Row row = new Row(1);
-        row.setField(0, 1);
-
-        final JsonRowSerializationSchema serializationSchema =
-                new JsonRowSerializationSchema.Builder(rowSchema).build();
-        open(serializationSchema);
-        assertThat(
-                row,
-                whenSerializedWith(serializationSchema)
-                        .failsWithException(instanceOf(RuntimeException.class)));
-    }
-
-    @Test
-    public void testSchema() {
-        final TypeInformation<Row> rowSchema =
-                JsonRowSchemaConverter.convert(
-                        "{"
-                                + "    type: 'object',"
-                                + "    properties: {"
-                                + "         id: { type: 'integer' },"
-                                + "         idNumber: { type: 'number' },"
-                                + "         idOrNull: { type: ['integer', 'null'] },"
-                                + "         name: { type: 'string' },"
-                                + "         date: { type: 'string', format: 'date' },"
-                                + "         time: { type: 'string', format: 'time' },"
-                                + "         timestamp: { type: 'string', format: 'date-time' },"
-                                + "         bytes: { type: 'string', contentEncoding: 'base64' },"
-                                + "         numbers: { type: 'array', items: { type: 'integer' } },"
-                                + "         strings: { type: 'array', items: { type: 'string' } },"
-                                + "         nested: { "
-                                + "             type: 'object',"
-                                + "             properties: { "
-                                + "                 booleanField: { type: 'boolean' },"
-                                + "                 decimalField: { type: 'number' }"
-                                + "             }"
-                                + "         }"
-                                + "    }"
-                                + "}");
-
-        final Row row = new Row(11);
-        row.setField(0, BigDecimal.valueOf(-333));
-        row.setField(1, BigDecimal.valueOf(12.2222));
-        row.setField(2, null);
-        row.setField(3, "");
-        row.setField(4, Date.valueOf("1990-10-14"));
-        row.setField(5, Time.valueOf("12:12:43"));
-        row.setField(6, Timestamp.valueOf("1990-10-14 12:12:43"));
-
-        final byte[] bytes = new byte[1024];
-        ThreadLocalRandom.current().nextBytes(bytes);
-        row.setField(7, bytes);
-        final BigDecimal[] numbers =
-                new BigDecimal[] {
-                    BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)
-                };
-        row.setField(8, numbers);
-        final String[] strings = new String[] {"one", "two", "three"};
-        row.setField(9, strings);
-        final Row nestedRow = new Row(2);
-        nestedRow.setField(0, true);
-        nestedRow.setField(1, BigDecimal.valueOf(12));
-        row.setField(10, nestedRow);
-
-        final JsonRowSerializationSchema serializationSchema =
-                new JsonRowSerializationSchema.Builder(rowSchema).build();
-        final JsonRowDeserializationSchema deserializationSchema =
-                new JsonRowDeserializationSchema.Builder(rowSchema).build();
-
-        assertThat(
-                row,
-                whenSerializedWith(serializationSchema)
-                        .andDeserializedWith(deserializationSchema)
-                        .equalsTo(row));
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java
deleted file mode 100644
index 5ed992c..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.connector.testutils.formats.DummyInitializationContext;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Objects;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-class JsonSerDeSchemaTest {
-    private static final JsonSerializationSchema<Event> SERIALIZATION_SCHEMA;
-    private static final JsonDeserializationSchema<Event> DESERIALIZATION_SCHEMA;
-    private static final String JSON = "{\"x\":34,\"y\":\"hello\"}";
-
-    static {
-        SERIALIZATION_SCHEMA = new JsonSerializationSchema<>();
-        SERIALIZATION_SCHEMA.open(new DummyInitializationContext());
-        DESERIALIZATION_SCHEMA = new JsonDeserializationSchema<>(Event.class);
-        DESERIALIZATION_SCHEMA.open(new DummyInitializationContext());
-    }
-
-    @Test
-    void testSrialization() throws IOException {
-        final byte[] serialized = SERIALIZATION_SCHEMA.serialize(new Event(34, "hello"));
-        assertThat(serialized).isEqualTo(JSON.getBytes(StandardCharsets.UTF_8));
-    }
-
-    @Test
-    void testDeserialization() throws IOException {
-        final Event deserialized =
-                DESERIALIZATION_SCHEMA.deserialize(JSON.getBytes(StandardCharsets.UTF_8));
-        assertThat(deserialized).isEqualTo(new Event(34, "hello"));
-    }
-
-    @Test
-    void testRoundTrip() throws IOException {
-        final Event original = new Event(34, "hello");
-
-        final byte[] serialized = SERIALIZATION_SCHEMA.serialize(original);
-
-        final Event deserialized = DESERIALIZATION_SCHEMA.deserialize(serialized);
-
-        assertThat(deserialized).isEqualTo(original);
-    }
-
-    private static class Event {
-
-        private int x;
-        private String y = null;
-
-        public Event() {}
-
-        public Event(int x, String y) {
-            this.x = x;
-            this.y = y;
-        }
-
-        public int getX() {
-            return x;
-        }
-
-        public void setX(int x) {
-            this.x = x;
-        }
-
-        public String getY() {
-            return y;
-        }
-
-        public void setY(String y) {
-            this.y = y;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) {
-                return true;
-            }
-            if (o == null || getClass() != o.getClass()) {
-                return false;
-            }
-            Event event = (Event) o;
-            return x == event.x && Objects.equals(y, event.y);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(x, y);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java
deleted file mode 100644
index 00bd5a0..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.table.api.ValidationException;
-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.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.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link CanalJsonFormatFactory}. */
-class CanalJsonFormatFactoryTest {
-
-    private static final InternalTypeInfo<RowData> ROW_TYPE_INFO =
-            InternalTypeInfo.of(PHYSICAL_TYPE);
-
-    @Test
-    void testDefaultOptions() {
-        Map<String, String> options = getAllOptions();
-
-        // test Deser
-        CanalJsonDeserializationSchema expectedDeser =
-                CanalJsonDeserializationSchema.builder(
-                                PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO)
-                        .setIgnoreParseErrors(false)
-                        .setTimestampFormat(TimestampFormat.SQL)
-                        .build();
-        DeserializationSchema<RowData> actualDeser = createDeserializationSchema(options);
-        assertThat(actualDeser).isEqualTo(expectedDeser);
-
-        // test Ser
-        CanalJsonSerializationSchema expectedSer =
-                new CanalJsonSerializationSchema(
-                        PHYSICAL_TYPE,
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.FAIL,
-                        "null",
-                        false);
-        SerializationSchema<RowData> actualSer = createSerializationSchema(options);
-        assertThat(actualSer).isEqualTo(expectedSer);
-    }
-
-    @Test
-    void testUserDefinedOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("canal-json.ignore-parse-errors", "true");
-        options.put("canal-json.timestamp-format.standard", "ISO-8601");
-        options.put("canal-json.database.include", "mydb");
-        options.put("canal-json.table.include", "mytable");
-        options.put("canal-json.map-null-key.mode", "LITERAL");
-        options.put("canal-json.map-null-key.literal", "nullKey");
-        options.put("canal-json.encode.decimal-as-plain-number", "true");
-
-        // test Deser
-        CanalJsonDeserializationSchema expectedDeser =
-                CanalJsonDeserializationSchema.builder(
-                                PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO)
-                        .setIgnoreParseErrors(true)
-                        .setTimestampFormat(TimestampFormat.ISO_8601)
-                        .setDatabase("mydb")
-                        .setTable("mytable")
-                        .build();
-        DeserializationSchema<RowData> actualDeser = createDeserializationSchema(options);
-        assertThat(actualDeser).isEqualTo(expectedDeser);
-
-        // test Ser
-        CanalJsonSerializationSchema expectedSer =
-                new CanalJsonSerializationSchema(
-                        PHYSICAL_TYPE,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "nullKey",
-                        true);
-        SerializationSchema<RowData> actualSer = createSerializationSchema(options);
-        assertThat(actualSer).isEqualTo(expectedSer);
-    }
-
-    @Test
-    void testInvalidIgnoreParseError() {
-        final Map<String, String> options =
-                getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc"));
-
-        assertThatThrownBy(() -> createDeserializationSchema(options))
-                .satisfies(
-                        anyCauseMatches(
-                                IllegalArgumentException.class,
-                                "Unrecognized option for boolean: abc. Expected either true"
-                                        + " or false(case insensitive)"));
-    }
-
-    @Test
-    void testInvalidOptionForTimestampFormat() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(
-                        opts -> opts.put("canal-json.timestamp-format.standard", "test"));
-
-        assertThatThrownBy(() -> createDeserializationSchema(tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'test' for timestamp-format.standard. "
-                                        + "Supported values are [SQL, ISO-8601]."));
-    }
-
-    @Test
-    void testInvalidOptionForMapNullKeyMode() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid"));
-
-        assertThatThrownBy(() -> createSerializationSchema(tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'invalid' for option map-null-key.mode. "
-                                        + "Supported values are [LITERAL, FAIL, DROP]."));
-    }
-
-    // ------------------------------------------------------------------------
-    //  Utilities
-    // ------------------------------------------------------------------------
-
-    /**
-     * Returns the full options modified by the given consumer {@code optionModifier}.
-     *
-     * @param optionModifier Consumer to modify the options
-     */
-    private Map<String, String> getModifiedOptions(Consumer<Map<String, String>> optionModifier) {
-        Map<String, String> options = getAllOptions();
-        optionModifier.accept(options);
-        return options;
-    }
-
-    private Map<String, String> getAllOptions() {
-        final Map<String, String> options = new HashMap<>();
-        options.put("connector", TestDynamicTableFactory.IDENTIFIER);
-        options.put("target", "MyTarget");
-        options.put("buffer-size", "1000");
-        options.put("format", "canal-json");
-        return options;
-    }
-
-    private static DeserializationSchema<RowData> createDeserializationSchema(
-            Map<String, String> options) {
-        DynamicTableSource source = createTableSource(SCHEMA, options);
-
-        assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock;
-        TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock =
-                (TestDynamicTableFactory.DynamicTableSourceMock) source;
-
-        return scanSourceMock.valueFormat.createRuntimeDecoder(
-                ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE);
-    }
-
-    private static SerializationSchema<RowData> createSerializationSchema(
-            Map<String, String> options) {
-        DynamicTableSink sink = createTableSink(SCHEMA, options);
-
-        assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock;
-        TestDynamicTableFactory.DynamicTableSinkMock sinkMock =
-                (TestDynamicTableFactory.DynamicTableSinkMock) sink;
-
-        return sinkMock.valueFormat.createRuntimeEncoder(
-                new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE);
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java
deleted file mode 100644
index e45bfcc..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java
+++ /dev/null
@@ -1,321 +0,0 @@
-/*
- * 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.connector.testutils.formats.DummyInitializationContext;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.RowData;
-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.DataTypeUtils;
-import org.apache.flink.util.Collector;
-
-import org.junit.jupiter.api.Test;
-
-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.Collections;
-import java.util.List;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-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.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */
-class CanalJsonSerDeSchemaTest {
-
-    private static final DataType PHYSICAL_DATA_TYPE =
-            ROW(
-                    FIELD("id", INT().notNull()),
-                    FIELD("name", STRING()),
-                    FIELD("description", STRING()),
-                    FIELD("weight", FLOAT()));
-
-    @Test
-    void testFilteringTables() throws Exception {
-        List<String> lines = readLines("canal-data-filter-table.txt");
-        CanalJsonDeserializationSchema deserializationSchema =
-                CanalJsonDeserializationSchema.builder(
-                                PHYSICAL_DATA_TYPE,
-                                Collections.emptyList(),
-                                InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()))
-                        .setDatabase("^my.*")
-                        .setTable("^prod.*")
-                        .build();
-        runTest(lines, deserializationSchema);
-    }
-
-    @Test
-    void testDeserializeNullRow() throws Exception {
-        final List<ReadableMetadata> requestedMetadata = Arrays.asList(ReadableMetadata.values());
-        final CanalJsonDeserializationSchema deserializationSchema =
-                createCanalJsonDeserializationSchema(null, null, requestedMetadata);
-        open(deserializationSchema);
-        final SimpleCollector collector = new SimpleCollector();
-
-        deserializationSchema.deserialize(null, collector);
-        deserializationSchema.deserialize(new byte[0], collector);
-        assertThat(collector.list).hasSize(0);
-    }
-
-    @Test
-    void testDeserializationWithMetadata() throws Exception {
-        testDeserializationWithMetadata(
-                "canal-data.txt",
-                null,
-                null,
-                row -> {
-                    assertThat(row.getInt(0)).isEqualTo(101);
-                    assertThat(row.getString(1).toString()).isEqualTo("scooter");
-                    assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter");
-                    assertThat(row.getFloat(3)).isEqualTo(3.14f);
-                    assertThat(row.getString(4).toString()).isEqualTo("inventory");
-                    assertThat(row.getString(5).toString()).isEqualTo("products2");
-                    assertThat(row.getMap(6).size()).isEqualTo(4);
-                    assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id");
-                    assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1589373515477L);
-                    assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1589373515000L);
-                });
-        testDeserializationWithMetadata(
-                "canal-data-filter-table.txt",
-                "mydb",
-                "product",
-                row -> {
-                    assertThat(row.getInt(0)).isEqualTo(101);
-                    assertThat(row.getString(1).toString()).isEqualTo("scooter");
-                    assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter");
-                    assertThat(row.getFloat(3)).isEqualTo(3.14f);
-                    assertThat(row.getString(4).toString()).isEqualTo("mydb");
-                    assertThat(row.getString(5).toString()).isEqualTo("product");
-                    assertThat(row.getMap(6).size()).isEqualTo(4);
-                    assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id");
-                    assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1598944146308L);
-                    assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1598944132000L);
-                });
-    }
-
-    @Test
-    void testSerializationDeserialization() throws Exception {
-        List<String> lines = readLines("canal-data.txt");
-        CanalJsonDeserializationSchema deserializationSchema =
-                CanalJsonDeserializationSchema.builder(
-                                PHYSICAL_DATA_TYPE,
-                                Collections.emptyList(),
-                                InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()))
-                        .setIgnoreParseErrors(false)
-                        .setTimestampFormat(TimestampFormat.ISO_8601)
-                        .build();
-        runTest(lines, deserializationSchema);
-    }
-
-    public void runTest(List<String> lines, CanalJsonDeserializationSchema deserializationSchema)
-            throws Exception {
-        open(deserializationSchema);
-        SimpleCollector collector = new SimpleCollector();
-        for (String line : lines) {
-            deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector);
-        }
-
-        // Canal captures change data (`canal-data.txt`) on the `product` table:
-        //
-        // CREATE TABLE product (
-        //  id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
-        //  name VARCHAR(255),
-        //  description VARCHAR(512),
-        //  weight FLOAT
-        // );
-        // ALTER TABLE product AUTO_INCREMENT = 101;
-        //
-        // INSERT INTO product
-        // VALUES (default,"scooter","Small 2-wheel scooter",3.14),
-        //        (default,"car battery","12V car battery",8.1),
-        //        (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40
-        // to #3",0.8),
-        //        (default,"hammer","12oz carpenter's hammer",0.75),
-        //        (default,"hammer","14oz carpenter's hammer",0.875),
-        //        (default,"hammer","16oz carpenter's hammer",1.0),
-        //        (default,"rocks","box of assorted rocks",5.3),
-        //        (default,"jacket","water resistent black wind breaker",0.1),
-        //        (default,"spare tire","24 inch spare tire",22.2);
-        // UPDATE product SET description='18oz carpenter hammer' WHERE id=106;
-        // UPDATE product SET weight='5.1' WHERE id=107;
-        // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2);
-        // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
-        // UPDATE product SET description='new water resistent white wind breaker', weight='0.5'
-        // WHERE id=110;
-        // UPDATE product SET weight='5.17' WHERE id=111;
-        // DELETE FROM product WHERE id=111;
-        // UPDATE product SET weight='5.17' WHERE id=102 or id = 101;
-        // DELETE FROM product WHERE id=102 or id = 103;
-        List<String> expected =
-                Arrays.asList(
-                        "+I(101,scooter,Small 2-wheel scooter,3.14)",
-                        "+I(102,car battery,12V car battery,8.1)",
-                        "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)",
-                        "+I(104,hammer,12oz carpenter's hammer,0.75)",
-                        "+I(105,hammer,14oz carpenter's hammer,0.875)",
-                        "+I(106,hammer,null,1.0)",
-                        "+I(107,rocks,box of assorted rocks,5.3)",
-                        "+I(108,jacket,water resistent black wind breaker,0.1)",
-                        "+I(109,spare tire,24 inch spare tire,22.2)",
-                        "-U(106,hammer,null,1.0)",
-                        "+U(106,hammer,18oz carpenter hammer,1.0)",
-                        "-U(107,rocks,box of assorted rocks,5.3)",
-                        "+U(107,rocks,box of assorted rocks,5.1)",
-                        "+I(110,jacket,water resistent white wind breaker,0.2)",
-                        "+I(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "-U(110,jacket,water resistent white wind breaker,0.2)",
-                        "+U(110,jacket,new water resistent white wind breaker,0.5)",
-                        "-U(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "+U(111,scooter,Big 2-wheel scooter ,5.17)",
-                        "-D(111,scooter,Big 2-wheel scooter ,5.17)",
-                        "-U(101,scooter,Small 2-wheel scooter,3.14)",
-                        "+U(101,scooter,Small 2-wheel scooter,5.17)",
-                        "-U(102,car battery,12V car battery,8.1)",
-                        "+U(102,car battery,12V car battery,5.17)",
-                        "-D(102,car battery,12V car battery,5.17)",
-                        "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)");
-        List<String> actual =
-                collector.list.stream().map(Object::toString).collect(Collectors.toList());
-        assertThat(actual).isEqualTo(expected);
-
-        // test Serialization
-        CanalJsonSerializationSchema serializationSchema =
-                new CanalJsonSerializationSchema(
-                        (RowType) PHYSICAL_DATA_TYPE.getLogicalType(),
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        serializationSchema.open(new DummyInitializationContext());
-
-        List<String> result = new ArrayList<>();
-        for (RowData rowData : collector.list) {
-            result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8));
-        }
-
-        List<String> expectedResult =
-                Arrays.asList(
-                        "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\"}",
-                        "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}",
-                        "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}");
-
-        assertThat(result).isEqualTo(expectedResult);
-    }
-
-    private void testDeserializationWithMetadata(
-            String resourceFile, String database, String table, Consumer<RowData> testConsumer)
-            throws Exception {
-        // we only read the first line for keeping the test simple
-        final String firstLine = readLines(resourceFile).get(0);
-        final List<ReadableMetadata> requestedMetadata = Arrays.asList(ReadableMetadata.values());
-        final CanalJsonDeserializationSchema deserializationSchema =
-                createCanalJsonDeserializationSchema(database, table, requestedMetadata);
-        open(deserializationSchema);
-        final SimpleCollector collector = new SimpleCollector();
-
-        deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector);
-        assertThat(collector.list).hasSize(9);
-        assertThat(collector.list.get(0)).satisfies(testConsumer);
-    }
-
-    private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema(
-            String database, String table, List<ReadableMetadata> requestedMetadata) {
-        final DataType producedDataType =
-                DataTypeUtils.appendRowFields(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata.stream()
-                                .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                                .collect(Collectors.toList()));
-        return CanalJsonDeserializationSchema.builder(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata,
-                        InternalTypeInfo.of(producedDataType.getLogicalType()))
-                .setDatabase(database)
-                .setTable(table)
-                .setIgnoreParseErrors(false)
-                .setTimestampFormat(TimestampFormat.ISO_8601)
-                .build();
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Utilities
-    // --------------------------------------------------------------------------------------------
-
-    private static List<String> readLines(String resource) throws IOException {
-        final URL url = CanalJsonSerDeSchemaTest.class.getClassLoader().getResource(resource);
-        assert url != null;
-        Path path = new File(url.getFile()).toPath();
-        return Files.readAllLines(path);
-    }
-
-    private static class SimpleCollector implements Collector<RowData> {
-
-        private final List<RowData> list = new ArrayList<>();
-
-        @Override
-        public void collect(RowData record) {
-            list.add(record);
-        }
-
-        @Override
-        public void close() {
-            // do nothing
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java
deleted file mode 100644
index bc47d1e..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.table.api.ValidationException;
-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.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.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link MaxwellJsonFormatFactory}. */
-class MaxwellJsonFormatFactoryTest {
-
-    private static final InternalTypeInfo<RowData> ROW_TYPE_INFO =
-            InternalTypeInfo.of(PHYSICAL_TYPE);
-
-    @Test
-    void testSeDeSchema() {
-        final MaxwellJsonDeserializationSchema expectedDeser =
-                new MaxwellJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        Collections.emptyList(),
-                        ROW_TYPE_INFO,
-                        true,
-                        TimestampFormat.ISO_8601);
-
-        final MaxwellJsonSerializationSchema expectedSer =
-                new MaxwellJsonSerializationSchema(
-                        PHYSICAL_TYPE,
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-
-        final Map<String, String> options = getAllOptions();
-
-        final DynamicTableSource actualSource = createTableSource(SCHEMA, options);
-        assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock;
-        TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock =
-                (TestDynamicTableFactory.DynamicTableSourceMock) actualSource;
-
-        DeserializationSchema<RowData> actualDeser =
-                scanSourceMock.valueFormat.createRuntimeDecoder(
-                        ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType());
-
-        assertThat(actualDeser).isEqualTo(expectedDeser);
-
-        final DynamicTableSink actualSink = createTableSink(SCHEMA, options);
-        assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock;
-        TestDynamicTableFactory.DynamicTableSinkMock sinkMock =
-                (TestDynamicTableFactory.DynamicTableSinkMock) actualSink;
-
-        SerializationSchema<RowData> actualSer =
-                sinkMock.valueFormat.createRuntimeEncoder(
-                        new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType());
-
-        assertThat(actualSer).isEqualTo(expectedSer);
-    }
-
-    @Test
-    void testInvalidIgnoreParseError() {
-        final Map<String, String> options =
-                getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc"));
-
-        assertThatThrownBy(() -> createTableSource(SCHEMA, options))
-                .satisfies(
-                        anyCauseMatches(
-                                IllegalArgumentException.class,
-                                "Unrecognized option for boolean: abc. "
-                                        + "Expected either true or false(case insensitive)"));
-    }
-
-    @Test
-    void testInvalidOptionForTimestampFormat() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(
-                        opts -> opts.put("maxwell-json.timestamp-format.standard", "test"));
-
-        assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'test' for timestamp-format.standard. "
-                                        + "Supported values are [SQL, ISO-8601]."));
-    }
-
-    @Test
-    void testInvalidOptionForMapNullKeyMode() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid"));
-
-        assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'invalid' for option map-null-key.mode. "
-                                        + "Supported values are [LITERAL, FAIL, DROP]."));
-    }
-
-    // ------------------------------------------------------------------------
-    //  Utilities
-    // ------------------------------------------------------------------------
-
-    /**
-     * Returns the full options modified by the given consumer {@code optionModifier}.
-     *
-     * @param optionModifier Consumer to modify the options
-     */
-    private Map<String, String> getModifiedOptions(Consumer<Map<String, String>> optionModifier) {
-        Map<String, String> options = getAllOptions();
-        optionModifier.accept(options);
-        return options;
-    }
-
-    private Map<String, String> getAllOptions() {
-        final Map<String, String> options = new HashMap<>();
-        options.put("connector", TestDynamicTableFactory.IDENTIFIER);
-        options.put("target", "MyTarget");
-        options.put("buffer-size", "1000");
-
-        options.put("format", "maxwell-json");
-        options.put("maxwell-json.ignore-parse-errors", "true");
-        options.put("maxwell-json.timestamp-format.standard", "ISO-8601");
-        options.put("maxwell-json.map-null-key.mode", "LITERAL");
-        options.put("maxwell-json.map-null-key.literal", "null");
-        options.put("maxwell-json.encode.decimal-as-plain-number", "true");
-        return options;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java
deleted file mode 100644
index 12d64fd..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * 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.maxwell;
-
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.RowData;
-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.DataTypeUtils;
-import org.apache.flink.util.Collector;
-
-import org.junit.jupiter.api.Test;
-
-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.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-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.assertj.core.api.Assertions.assertThat;
-
-/**
- * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}.
- */
-class MaxwellJsonSerDerTest {
-
-    private static final DataType PHYSICAL_DATA_TYPE =
-            ROW(
-                    FIELD("id", INT().notNull()),
-                    FIELD("name", STRING()),
-                    FIELD("description", STRING()),
-                    FIELD("weight", FLOAT()));
-
-    @Test
-    void testDeserializationWithMetadata() throws Exception {
-        // we only read the first line for keeping the test simple
-        final String firstLine = readLines("maxwell-data.txt").get(0);
-        final List<ReadableMetadata> requestedMetadata = Arrays.asList(ReadableMetadata.values());
-        final DataType producedDataType =
-                DataTypeUtils.appendRowFields(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata.stream()
-                                .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                                .collect(Collectors.toList()));
-        final MaxwellJsonDeserializationSchema deserializationSchema =
-                new MaxwellJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata,
-                        InternalTypeInfo.of(producedDataType.getLogicalType()),
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        final SimpleCollector collector = new SimpleCollector();
-        deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector);
-        assertThat(collector.list).hasSize(1);
-        assertThat(collector.list.get(0))
-                .satisfies(
-                        row -> {
-                            assertThat(row.getInt(0)).isEqualTo(101);
-                            assertThat(row.getString(1).toString()).isEqualTo("scooter");
-                            assertThat(row.getString(2).toString())
-                                    .isEqualTo("Small 2-wheel scooter");
-                            assertThat(row.getFloat(3)).isEqualTo(3.14f);
-                            assertThat(row.getString(4).toString()).isEqualTo("test");
-                            assertThat(row.getString(5).toString()).isEqualTo("product");
-                            assertThat(row.getArray(6).getString(0).toString()).isEqualTo("id");
-                            assertThat(row.getTimestamp(7, 3).getMillisecond())
-                                    .isEqualTo(1596684883000L);
-                        });
-    }
-
-    @Test
-    void testSerializationDeserialization() throws Exception {
-        List<String> lines = readLines("maxwell-data.txt");
-        MaxwellJsonDeserializationSchema deserializationSchema =
-                new MaxwellJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        Collections.emptyList(),
-                        InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()),
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        SimpleCollector collector = new SimpleCollector();
-        for (String line : lines) {
-            deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector);
-        }
-
-        // Maxwell captures change data (`maxwell-data.txt`) on the `product` table:
-        //
-        // CREATE TABLE product (
-        //  id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
-        //  name VARCHAR(255),
-        //  description VARCHAR(512),
-        //  weight FLOAT
-        // );
-        // ALTER TABLE product AUTO_INCREMENT = 101;
-        //
-        // INSERT INTO product
-        // VALUES (default,"scooter","Small 2-wheel scooter",3.14),
-        //        (default,"car battery","12V car battery",8.1),
-        //        (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40
-        // to #3",0.8),
-        //        (default,"hammer","12oz carpenter's hammer",0.75),
-        //        (default,"hammer","14oz carpenter's hammer",0.875),
-        //        (default,"hammer","16oz carpenter's hammer",1.0),
-        //        (default,"rocks","box of assorted rocks",5.3),
-        //        (default,"jacket","water resistent black wind breaker",0.1),
-        //        (default,"spare tire","24 inch spare tire",22.2);
-        // UPDATE product SET description='18oz carpenter hammer' WHERE id=106;
-        // UPDATE product SET weight='5.1' WHERE id=107;
-        // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2);
-        // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
-        // UPDATE product SET description='new water resistent white wind breaker', weight='0.5'
-        // WHERE id=110;
-        // UPDATE product SET weight='5.17' WHERE id=111;
-        // DELETE FROM product WHERE id=111;
-        // UPDATE product SET weight='5.17' WHERE id=102 or id = 101;
-        // DELETE FROM product WHERE id=102 or id = 103;
-        List<String> expected =
-                Arrays.asList(
-                        "+I(101,scooter,Small 2-wheel scooter,3.14)",
-                        "+I(102,car battery,12V car battery,8.1)",
-                        "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)",
-                        "+I(104,hammer,12oz carpenter's hammer,0.75)",
-                        "+I(105,hammer,14oz carpenter's hammer,0.875)",
-                        "+I(106,hammer,16oz carpenter's hammer,1.0)",
-                        "+I(107,rocks,box of assorted rocks,5.3)",
-                        "+I(108,jacket,water resistent black wind breaker,0.1)",
-                        "+I(109,spare tire,24 inch spare tire,22.2)",
-                        "-U(106,hammer,16oz carpenter's hammer,1.0)",
-                        "+U(106,hammer,18oz carpenter hammer,1.0)",
-                        "-U(107,rocks,box of assorted rocks,5.3)",
-                        "+U(107,rocks,box of assorted rocks,5.1)",
-                        "+I(110,jacket,water resistent white wind breaker,0.2)",
-                        "+I(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "-U(110,jacket,water resistent white wind breaker,0.2)",
-                        "+U(110,jacket,new water resistent white wind breaker,0.5)",
-                        "-U(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "+U(111,scooter,Big 2-wheel scooter ,5.17)",
-                        "-D(111,scooter,Big 2-wheel scooter ,5.17)",
-                        "-U(101,scooter,Small 2-wheel scooter,3.14)",
-                        "+U(101,scooter,Small 2-wheel scooter,5.17)",
-                        "-U(102,car battery,12V car battery,8.1)",
-                        "+U(102,car battery,12V car battery,5.17)",
-                        "-D(102,car battery,12V car battery,5.17)",
-                        "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)");
-        List<String> actual =
-                collector.list.stream().map(Object::toString).collect(Collectors.toList());
-        assertThat(actual).isEqualTo(expected);
-
-        MaxwellJsonSerializationSchema serializationSchema =
-                new MaxwellJsonSerializationSchema(
-                        (RowType) PHYSICAL_DATA_TYPE.getLogicalType(),
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-        open(serializationSchema);
-        List<String> result = new ArrayList<>();
-        for (RowData rowData : collector.list) {
-            result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8));
-        }
-        List<String> expectedResult =
-                Arrays.asList(
-                        "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}",
-                        "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}",
-                        "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}");
-        assertThat(result).isEqualTo(expectedResult);
-    }
-
-    // --------------------------------------------------------------------------------------------
-    // Utilities
-    // --------------------------------------------------------------------------------------------
-
-    private static List<String> readLines(String resource) throws IOException {
-        final URL url = MaxwellJsonSerDerTest.class.getClassLoader().getResource(resource);
-        assert url != null;
-        Path path = new File(url.getFile()).toPath();
-        return Files.readAllLines(path);
-    }
-
-    private static class SimpleCollector implements Collector<RowData> {
-
-        private List<RowData> list = new ArrayList<>();
-
-        @Override
-        public void collect(RowData record) {
-            list.add(record);
-        }
-
-        @Override
-        public void close() {
-            // do nothing
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java
deleted file mode 100644
index 84b293c..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.CloseableIterator;
-import org.apache.flink.util.CollectionUtil;
-
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static java.lang.String.format;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test Filesystem connector with OGG Json. */
-class OggJsonFileSystemITCase extends StreamingTestBase {
-
-    private static final List<String> EXPECTED =
-            Arrays.asList(
-                    "+I[101, SCOOTER, Small 2-wheel scooter, 3.14]",
-                    "+I[102, CAR BATTERY, 12V car battery, 8.1]",
-                    "+I[103, 12-PACK DRILL BITS, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]",
-                    "+I[104, HAMMER, 12oz carpenter's hammer, 0.75]",
-                    "+I[105, HAMMER, 14oz carpenter's hammer, 0.875]",
-                    "+I[106, HAMMER, 16oz carpenter's hammer, 1.0]",
-                    "+I[107, ROCKS, box of assorted rocks, 5.3]",
-                    "+I[108, JACKET, water resistent black wind breaker, 0.1]",
-                    "+I[109, SPARE TIRE, 24 inch spare tire, 22.2]",
-                    "-D[106, HAMMER, 16oz carpenter's hammer, 1.0]", // -U
-                    "+I[106, HAMMER, 18oz carpenter hammer, 1.0]", // +U
-                    "-D[107, ROCKS, box of assorted rocks, 5.3]", // -U
-                    "+I[107, ROCKS, box of assorted rocks, 5.1]", // +U
-                    "+I[110, JACKET, water resistent white wind breaker, 0.2]",
-                    "+I[111, SCOOTER, Big 2-wheel scooter , 5.18]",
-                    "-D[110, JACKET, water resistent white wind breaker, 0.2]", // -U
-                    "+I[110, JACKET, new water resistent white wind breaker, 0.5]", // +U
-                    "-D[111, SCOOTER, Big 2-wheel scooter , 5.18]", // -U
-                    "+I[111, SCOOTER, Big 2-wheel scooter , 5.17]", // +U
-                    "-D[111, SCOOTER, Big 2-wheel scooter , 5.17]");
-
-    private File source;
-    private File sink;
-
-    private static byte[] readBytes(String resource) throws IOException {
-        final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource);
-        assert url != null;
-        Path path = new File(url.getFile()).toPath();
-        return Files.readAllBytes(path);
-    }
-
-    private void prepareTables(boolean isPartition, Path tempSourceDir, Path tempSinkDir)
-            throws IOException {
-        byte[] bytes = readBytes("ogg-data.txt");
-        source = tempSourceDir.toFile();
-        File file;
-        if (isPartition) {
-            File partition = new File(source, "p=1");
-            partition.mkdirs();
-            file = new File(partition, "my_file");
-        } else {
-            file = new File(source, "my_file");
-        }
-        file.createNewFile();
-        Files.write(file.toPath(), bytes);
-
-        sink = tempSinkDir.toFile();
-
-        env().setParallelism(1);
-    }
-
-    private void createTable(boolean isSink, String path, boolean isPartition) {
-        tEnv().executeSql(
-                        format("create table %s (", isSink ? "sink" : "source")
-                                + "id int, name string,"
-                                + (isSink ? "upper_name string," : "")
-                                + " description string, weight float"
-                                + (isPartition ? ", p int) partitioned by (p) " : ")")
-                                + " with ("
-                                + "'connector'='filesystem',"
-                                + "'format'='ogg-json',"
-                                + format("'path'='%s'", path)
-                                + ")");
-    }
-
-    @Test
-    void testNonPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception {
-        prepareTables(true, tempSourceDir, tempSinkDir);
-        createTable(false, source.toURI().toString(), false);
-        createTable(true, sink.toURI().toString(), false);
-
-        tEnv().executeSql(
-                        "insert into sink select id,name,UPPER(name),description,weight from source")
-                .await();
-        CloseableIterator<Row> iter =
-                tEnv().executeSql("select id,upper_name,description,weight from sink").collect();
-
-        List<String> results =
-                CollectionUtil.iteratorToList(iter).stream()
-                        .map(Row::toString)
-                        .collect(Collectors.toList());
-        iter.close();
-
-        assertThat(results).isEqualTo(EXPECTED);
-    }
-
-    @Test
-    void testPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception {
-        prepareTables(true, tempSourceDir, tempSinkDir);
-        createTable(false, source.toURI().toString(), true);
-        createTable(true, sink.toURI().toString(), true);
-
-        tEnv().executeSql(
-                        "insert into sink select id,name,UPPER(name),description,weight,p from source")
-                .await();
-        CloseableIterator<Row> iter =
-                tEnv().executeSql("select id,upper_name,description,weight,p from sink").collect();
-        List<Row> list = CollectionUtil.iteratorToList(iter);
-        iter.close();
-
-        List<String> results =
-                list.stream()
-                        .map(row -> Row.project(row, new int[] {0, 1, 2, 3}))
-                        .map(Row::toString)
-                        .collect(Collectors.toList());
-
-        assertThat(results).isEqualTo(EXPECTED);
-
-        // check partition value
-        for (Row row : list) {
-            assertThat(row.getField(4)).isEqualTo(1);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java
deleted file mode 100644
index c04e991..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.TestDynamicTableFactory;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.types.logical.RowType;
-
-import org.junit.jupiter.api.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE;
-import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link OggJsonFormatFactory}. */
-class OggJsonFormatFactoryTest {
-
-    @Test
-    void testSeDeSchema() {
-        final Map<String, String> options = getAllOptions();
-
-        final OggJsonSerializationSchema expectedSer =
-                new OggJsonSerializationSchema(
-                        (RowType) PHYSICAL_DATA_TYPE.getLogicalType(),
-                        TimestampFormat.ISO_8601,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-
-        final DynamicTableSink actualSink = createTableSink(SCHEMA, options);
-        assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock;
-        TestDynamicTableFactory.DynamicTableSinkMock sinkMock =
-                (TestDynamicTableFactory.DynamicTableSinkMock) actualSink;
-
-        SerializationSchema<RowData> actualSer =
-                sinkMock.valueFormat.createRuntimeEncoder(
-                        new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE);
-
-        assertThat(actualSer).isEqualTo(expectedSer);
-    }
-
-    @Test
-    void testInvalidIgnoreParseError() {
-        final Map<String, String> options =
-                getModifiedOptions(opts -> opts.put("ogg-json.ignore-parse-errors", "abc"));
-
-        assertThatThrownBy(() -> createTableSource(SCHEMA, options))
-                .satisfies(
-                        anyCauseMatches(
-                                IllegalArgumentException.class,
-                                "Unrecognized option for boolean: abc. "
-                                        + "Expected either true or false(case insensitive)"));
-    }
-
-    @Test
-    void testInvalidOptionForTimestampFormat() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(opts -> opts.put("ogg-json.timestamp-format.standard", "test"));
-
-        assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'test' for timestamp-format.standard. "
-                                        + "Supported values are [SQL, ISO-8601]."));
-    }
-
-    @Test
-    void testInvalidOptionForMapNullKeyMode() {
-        final Map<String, String> tableOptions =
-                getModifiedOptions(opts -> opts.put("ogg-json.map-null-key.mode", "invalid"));
-
-        assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions))
-                .isInstanceOf(ValidationException.class)
-                .satisfies(
-                        anyCauseMatches(
-                                ValidationException.class,
-                                "Unsupported value 'invalid' for option map-null-key.mode. "
-                                        + "Supported values are [LITERAL, FAIL, DROP]."));
-    }
-
-    // ------------------------------------------------------------------------
-    //  Utilities
-    // ------------------------------------------------------------------------
-
-    /**
-     * Returns the full options modified by the given consumer {@code optionModifier}.
-     *
-     * @param optionModifier Consumer to modify the options
-     */
-    private Map<String, String> getModifiedOptions(Consumer<Map<String, String>> optionModifier) {
-        Map<String, String> options = getAllOptions();
-        optionModifier.accept(options);
-        return options;
-    }
-
-    private Map<String, String> getAllOptions() {
-        final Map<String, String> options = new HashMap<>();
-        options.put("connector", TestDynamicTableFactory.IDENTIFIER);
-        options.put("target", "MyTarget");
-        options.put("buffer-size", "1000");
-
-        options.put("format", "ogg-json");
-        options.put("ogg-json.ignore-parse-errors", "true");
-        options.put("ogg-json.timestamp-format.standard", "ISO-8601");
-        options.put("ogg-json.map-null-key.mode", "LITERAL");
-        options.put("ogg-json.map-null-key.literal", "null");
-        options.put("ogg-json.encode.decimal-as-plain-number", "true");
-        return options;
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java
deleted file mode 100644
index 2fa78c8..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/*
- * 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.ogg;
-
-import org.apache.flink.formats.common.TimestampFormat;
-import org.apache.flink.formats.json.JsonFormatOptions;
-import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.RowData;
-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.DataTypeUtils;
-import org.apache.flink.util.Collector;
-
-import org.assertj.core.data.Percentage;
-import org.junit.jupiter.api.Test;
-
-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.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-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.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link OggJsonSerializationSchema} and {@link OggJsonDeserializationSchema}. */
-class OggJsonSerDeSchemaTest {
-
-    private static final DataType PHYSICAL_DATA_TYPE =
-            ROW(
-                    FIELD("id", INT().notNull()),
-                    FIELD("name", STRING()),
-                    FIELD("description", STRING()),
-                    FIELD("weight", FLOAT()));
-
-    private static List<String> readLines(String resource) throws IOException {
-        final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource);
-        assert url != null;
-        Path path = new File(url.getFile()).toPath();
-        return Files.readAllLines(path);
-    }
-
-    @Test
-    void testSerializationAndDeserialization() throws Exception {
-        testSerializationDeserialization("ogg-data.txt");
-    }
-
-    @Test
-    void testDeserializationWithMetadata() throws Exception {
-        testDeserializationWithMetadata("ogg-data.txt");
-    }
-
-    @Test
-    void testTombstoneMessages() throws Exception {
-        OggJsonDeserializationSchema deserializationSchema =
-                new OggJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        Collections.emptyList(),
-                        InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()),
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-        SimpleCollector collector = new SimpleCollector();
-        deserializationSchema.deserialize(null, collector);
-        deserializationSchema.deserialize(new byte[] {}, collector);
-        assertThat(collector.list).isEmpty();
-    }
-
-    public void testDeserializationWithMetadata(String resourceFile) throws Exception {
-        // we only read the first line for keeping the test simple
-        final String firstLine = readLines(resourceFile).get(0);
-
-        final List<ReadableMetadata> requestedMetadata = Arrays.asList(ReadableMetadata.values());
-
-        final DataType producedDataTypes =
-                DataTypeUtils.appendRowFields(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata.stream()
-                                .map(m -> DataTypes.FIELD(m.key, m.dataType))
-                                .collect(Collectors.toList()));
-        final OggJsonDeserializationSchema deserializationSchema =
-                new OggJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        requestedMetadata,
-                        InternalTypeInfo.of(producedDataTypes.getLogicalType()),
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        final SimpleCollector collector = new SimpleCollector();
-        deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector);
-        assertThat(collector.list).hasSize(1);
-        assertThat(collector.list.get(0))
-                .satisfies(
-                        row -> {
-                            assertThat(row.getInt(0)).isEqualTo(101);
-                            assertThat(row.getString(1).toString()).isEqualTo("scooter");
-                            assertThat(row.getString(2).toString())
-                                    .isEqualTo("Small 2-wheel scooter");
-                            assertThat(row.getFloat(3))
-                                    .isCloseTo(
-                                            3.140000104904175f, Percentage.withPercentage(1e-15));
-                            assertThat(row.getString(4).toString()).isEqualTo("OGG.TBL_TEST");
-                            assertThat(row.getArray(5).getString(0).toString()).isEqualTo("id");
-                            assertThat(row.getTimestamp(6, 6).getMillisecond())
-                                    .isEqualTo(1589377175766L);
-                            assertThat(row.getTimestamp(7, 6).getMillisecond())
-                                    .isEqualTo(1589384406000L);
-                        });
-    }
-
-    private void testSerializationDeserialization(String resourceFile) throws Exception {
-        List<String> lines = readLines(resourceFile);
-        OggJsonDeserializationSchema deserializationSchema =
-                new OggJsonDeserializationSchema(
-                        PHYSICAL_DATA_TYPE,
-                        Collections.emptyList(),
-                        InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()),
-                        false,
-                        TimestampFormat.ISO_8601);
-        open(deserializationSchema);
-
-        SimpleCollector collector = new SimpleCollector();
-        for (String line : lines) {
-            deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector);
-        }
-
-        // Ogg captures change data (`ogg-data.txt`) on the `product`
-        // table:
-        //
-        // CREATE TABLE product (
-        //  id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
-        //  name VARCHAR(255),
-        //  description VARCHAR(512),
-        //  weight FLOAT
-        // );
-        // ALTER TABLE product AUTO_INCREMENT = 101;
-        //
-        // INSERT INTO product
-        // VALUES (default,"scooter","Small 2-wheel scooter",3.14),
-        //        (default,"car battery","12V car battery",8.1),
-        //        (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40
-        // to #3",0.8),
-        //        (default,"hammer","12oz carpenter's hammer",0.75),
-        //        (default,"hammer","14oz carpenter's hammer",0.875),
-        //        (default,"hammer","16oz carpenter's hammer",1.0),
-        //        (default,"rocks","box of assorted rocks",5.3),
-        //        (default,"jacket","water resistent black wind breaker",0.1),
-        //        (default,"spare tire","24 inch spare tire",22.2);
-        // UPDATE product SET description='18oz carpenter hammer' WHERE id=106;
-        // UPDATE product SET weight='5.1' WHERE id=107;
-        // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2);
-        // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
-        // UPDATE product SET description='new water resistent white wind breaker', weight='0.5'
-        // WHERE id=110;
-        // UPDATE product SET weight='5.17' WHERE id=111;
-        // DELETE FROM product WHERE id=111;
-        List<String> expected =
-                Arrays.asList(
-                        "+I(101,scooter,Small 2-wheel scooter,3.14)",
-                        "+I(102,car battery,12V car battery,8.1)",
-                        "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)",
-                        "+I(104,hammer,12oz carpenter's hammer,0.75)",
-                        "+I(105,hammer,14oz carpenter's hammer,0.875)",
-                        "+I(106,hammer,16oz carpenter's hammer,1.0)",
-                        "+I(107,rocks,box of assorted rocks,5.3)",
-                        "+I(108,jacket,water resistent black wind breaker,0.1)",
-                        "+I(109,spare tire,24 inch spare tire,22.2)",
-                        "-U(106,hammer,16oz carpenter's hammer,1.0)",
-                        "+U(106,hammer,18oz carpenter hammer,1.0)",
-                        "-U(107,rocks,box of assorted rocks,5.3)",
-                        "+U(107,rocks,box of assorted rocks,5.1)",
-                        "+I(110,jacket,water resistent white wind breaker,0.2)",
-                        "+I(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "-U(110,jacket,water resistent white wind breaker,0.2)",
-                        "+U(110,jacket,new water resistent white wind breaker,0.5)",
-                        "-U(111,scooter,Big 2-wheel scooter ,5.18)",
-                        "+U(111,scooter,Big 2-wheel scooter ,5.17)",
-                        "-D(111,scooter,Big 2-wheel scooter ,5.17)");
-        List<String> actual =
-                collector.list.stream().map(Object::toString).collect(Collectors.toList());
-        assertThat(expected).containsExactlyElementsOf(actual);
-
-        OggJsonSerializationSchema serializationSchema =
-                new OggJsonSerializationSchema(
-                        (RowType) PHYSICAL_DATA_TYPE.getLogicalType(),
-                        TimestampFormat.SQL,
-                        JsonFormatOptions.MapNullKeyMode.LITERAL,
-                        "null",
-                        true);
-
-        open(serializationSchema);
-        actual = new ArrayList<>();
-        for (RowData rowData : collector.list) {
-            actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8));
-        }
-
-        expected =
-                Arrays.asList(
-                        "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op_type\":\"I\"}",
-                        "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op_type\":\"D\"}",
-                        "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op_type\":\"I\"}",
-                        "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op_type\":\"D\"}",
-                        "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"I\"}",
-                        "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"I\"}",
-                        "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op_type\":\"D\"}",
-                        "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op_type\":\"I\"}",
-                        "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op_type\":\"D\"}",
-                        "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\"}",
-                        "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op_type\":\"D\"}");
-        assertThat(expected).containsExactlyElementsOf(actual);
-    }
-
-    private static class SimpleCollector implements Collector<RowData> {
-
-        private final List<RowData> list = new ArrayList<>();
-
-        @Override
-        public void collect(RowData record) {
-            list.add(record);
-        }
-
-        @Override
-        public void close() {
-            // do nothing
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java
deleted file mode 100644
index c6d9e62..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * 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.utils;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.types.Row;
-
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-
-import java.io.IOException;
-import java.util.Objects;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-import static org.apache.flink.util.InstantiationUtil.deserializeObject;
-import static org.apache.flink.util.InstantiationUtil.serializeObject;
-
-/**
- * Matcher that provides a common way for asserting results of {@link DeserializationSchema}. It
- * takes into account e.g. the fact that serialization schema during runtime might be used after
- * serializing it over a wire. Usage:
- *
- * <ul>
- *   <li>when asserting for result after deserializing a row
- *       <pre>{@code
- * byte[] jsonBytes = ...
- * Row expectedRow = ...
- * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
- *
- * assertThat(jsonBytes, whenDeserializedWith(deserializationSchema)
- *     .matches(expectedRow));
- * }</pre>
- *   <li>to check if an exception is thrown during serialization:
- *       <pre>{@code
- * assertThat(serializedJson,
- *     whenDeserializedWith(deserializationSchema)
- *         .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
- * }</pre>
- * </ul>
- */
-public abstract class DeserializationSchemaMatcher extends TypeSafeMatcher<byte[]> {
-
-    final DeserializationSchema<Row> deserializationSchema;
-
-    private DeserializationSchemaMatcher(DeserializationSchema<Row> deserializationSchema) {
-        this.deserializationSchema = deserializationSchema;
-    }
-
-    public static DeserializationSchemaMatcherBuilder whenDeserializedWith(
-            DeserializationSchema<Row> deserializationSchema) {
-        return new DeserializationSchemaMatcherBuilder(deserializationSchema);
-    }
-
-    private static class DeserializationSchemaResultMatcher extends DeserializationSchemaMatcher {
-
-        private final Row expected;
-
-        private DeserializationSchemaResultMatcher(
-                DeserializationSchema<Row> deserializationSchema, Row expected) {
-            super(deserializationSchema);
-
-            this.expected = expected;
-        }
-
-        @Override
-        protected boolean matchesSafely(byte[] item) {
-            try {
-                return Objects.deepEquals(deserializationSchema.deserialize(item), expected);
-            } catch (IOException e) {
-                throw new AssertionError("Could not deserialize", e);
-            }
-        }
-
-        @Override
-        public void describeTo(Description description) {
-            description.appendValue(expected);
-        }
-    }
-
-    private static class DeserializationSchemaExceptionMatcher
-            extends DeserializationSchemaMatcher {
-
-        private final Matcher<? extends Throwable> exceptionMatcher;
-        private Throwable thrownException = null;
-
-        private DeserializationSchemaExceptionMatcher(
-                DeserializationSchema<Row> deserializationSchema,
-                Matcher<? extends Throwable> exceptionMatcher) {
-            super(deserializationSchema);
-            this.exceptionMatcher = exceptionMatcher;
-        }
-
-        @Override
-        protected boolean matchesSafely(byte[] item) {
-            try {
-                deserializationSchema.deserialize(item);
-            } catch (IOException e) {
-                thrownException = e;
-            }
-            return exceptionMatcher.matches(thrownException);
-        }
-
-        @Override
-        public void describeTo(Description description) {
-            exceptionMatcher.describeTo(description);
-        }
-
-        @Override
-        protected void describeMismatchSafely(byte[] item, Description mismatchDescription) {
-            exceptionMatcher.describeMismatch(thrownException, mismatchDescription);
-        }
-    }
-
-    /** Builder for {@link DeserializationSchemaMatcher}. */
-    public static class DeserializationSchemaMatcherBuilder {
-
-        private DeserializationSchema<Row> deserializationSchema;
-
-        private DeserializationSchemaMatcherBuilder(
-                DeserializationSchema<Row> deserializationSchema) {
-            try {
-                // we serialize and deserialize the schema to test runtime behavior
-                // when the schema is shipped to the cluster
-                this.deserializationSchema =
-                        deserializeObject(
-                                serializeObject(deserializationSchema),
-                                this.getClass().getClassLoader());
-                open(this.deserializationSchema);
-            } catch (IOException | ClassNotFoundException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        public DeserializationSchemaMatcher equalsTo(Row row) {
-            return new DeserializationSchemaResultMatcher(deserializationSchema, row);
-        }
-
-        public DeserializationSchemaMatcher failsWithException(
-                Matcher<? extends Throwable> exceptionMatcher) {
-            return new DeserializationSchemaExceptionMatcher(
-                    deserializationSchema, exceptionMatcher);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java
deleted file mode 100644
index 1cf6f6d..0000000
--- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * 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.utils;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.types.Row;
-
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-
-import java.io.IOException;
-import java.util.Objects;
-
-import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open;
-import static org.apache.flink.util.InstantiationUtil.deserializeObject;
-import static org.apache.flink.util.InstantiationUtil.serializeObject;
-
-/**
- * Matcher that provides a common way for asserting results of {@link SerializationSchema}. It takes
- * into account e.g. the fact that serialization schema during runtime might be used after
- * serializing and deserializing it over a wire. Usage:
- *
- * <ul>
- *   <li>when asserting for result after serializing and deserializing a row
- *       <pre>{@code
- * final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema);
- * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
- *
- * assertThat(row, whenSerializedWith(serializationSchema)
- *     .andDeserializedWith(deserializationSchema)
- *     .matches(row));
- * }</pre>
- *   <li>to check if an exception is thrown during serialization:
- *       <pre>{@code
- * assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class)));
- * }</pre>
- * </ul>
- */
-public abstract class SerializationSchemaMatcher extends TypeSafeMatcher<Row> {
-
-    final SerializationSchema<Row> serializationSchema;
-
-    private SerializationSchemaMatcher(SerializationSchema<Row> serializationSchema) {
-        this.serializationSchema = serializationSchema;
-    }
-
-    public static SerializationSchemaMatcherBuilder whenSerializedWith(
-            SerializationSchema<Row> serializationSchema) {
-        return new SerializationSchemaMatcherBuilder(serializationSchema);
-    }
-
-    private static class SerializationSchemaResultMatcher extends SerializationSchemaMatcher {
-
-        private final Row expected;
-        private final DeserializationSchema<Row> deserializationSchema;
-
-        private SerializationSchemaResultMatcher(
-                SerializationSchema<Row> serializationSchema,
-                DeserializationSchema<Row> deserializationSchema,
-                Row expected) {
-            super(serializationSchema);
-
-            this.expected = expected;
-            this.deserializationSchema = deserializationSchema;
-        }
-
-        @Override
-        protected boolean matchesSafely(Row item) {
-            try {
-                return Objects.deepEquals(
-                        deserializationSchema.deserialize(serializationSchema.serialize(item)),
-                        expected);
-            } catch (IOException e) {
-                throw new AssertionError("Could not deserialize", e);
-            }
-        }
-
-        @Override
-        public void describeTo(Description description) {
-            description.appendValue(expected);
-        }
-    }
-
-    private static class SerializationSchemaExceptionMatcher extends SerializationSchemaMatcher {
-
-        private final Matcher<? extends Throwable> exceptionMatcher;
-        private Throwable thrownException = null;
-
-        private SerializationSchemaExceptionMatcher(
-                SerializationSchema<Row> serializationSchema,
-                Matcher<? extends Throwable> exceptionMatcher) {
-            super(serializationSchema);
-            this.exceptionMatcher = exceptionMatcher;
-        }
-
-        @Override
-        protected boolean matchesSafely(Row item) {
-            try {
-                serializationSchema.serialize(item);
-            } catch (Exception e) {
-                thrownException = e;
-            }
-            return exceptionMatcher.matches(thrownException);
-        }
-
-        @Override
-        public void describeTo(Description description) {
-            exceptionMatcher.describeTo(description);
-        }
-
-        @Override
-        protected void describeMismatchSafely(Row item, Description mismatchDescription) {
-            exceptionMatcher.describeMismatch(thrownException, mismatchDescription);
-        }
-    }
-
-    /**
-     * Builder for {@link SerializationSchemaMatcher} that can assert results after serialize and
-     * deserialize.
-     */
-    public static class SerializationWithDeserializationSchemaMatcherBuilder {
-
-        private SerializationSchema<Row> serializationSchema;
-        private DeserializationSchema<Row> deserializationSchema;
-
-        private SerializationWithDeserializationSchemaMatcherBuilder(
-                SerializationSchema<Row> serializationSchema,
-                DeserializationSchema<Row> deserializationSchema) {
-            try {
-                // we serialize and deserialize the schema to test runtime behavior
-                // when the schema is shipped to the cluster
-                this.serializationSchema =
-                        deserializeObject(
-                                serializeObject(serializationSchema),
-                                this.getClass().getClassLoader());
-                open(this.serializationSchema);
-                this.deserializationSchema =
-                        deserializeObject(
-                                serializeObject(deserializationSchema),
-                                this.getClass().getClassLoader());
-                open(this.deserializationSchema);
-            } catch (IOException | ClassNotFoundException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        public SerializationSchemaMatcher equalsTo(Row expected) {
-            return new SerializationSchemaResultMatcher(
-                    serializationSchema, deserializationSchema, expected);
-        }
-    }
-
-    /** Builder for {@link SerializationSchemaMatcher}. */
-    public static class SerializationSchemaMatcherBuilder {
-
-        private SerializationSchema<Row> serializationSchema;
-
-        private SerializationSchemaMatcherBuilder(SerializationSchema<Row> serializationSchema) {
-            this.serializationSchema = serializationSchema;
-        }
-
-        public SerializationWithDeserializationSchemaMatcherBuilder andDeserializedWith(
-                DeserializationSchema<Row> deserializationSchema) {
-            return new SerializationWithDeserializationSchemaMatcherBuilder(
-                    serializationSchema, deserializationSchema);
-        }
-
-        public SerializationSchemaMatcher failsWithException(
-                Matcher<? extends Throwable> exceptionMatcher) {
-            return new SerializationSchemaExceptionMatcher(serializationSchema, exceptionMatcher);
-        }
-    }
-}
diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml
index 10789e4..9ed0c21 100644
--- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml
+++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml
@@ -24,8 +24,8 @@
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-formats</artifactId>
-		<version>1.18-SNAPSHOT</version>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
 	</parent>
 
 	<artifactId>flink-sql-avro-confluent-registry</artifactId>
diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
index 6e85cfe..474ed02 100644
--- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
+++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
@@ -15,7 +15,7 @@
 - io.confluent:kafka-schema-registry-client:7.2.2
 - org.apache.avro:avro:1.11.1
 - org.apache.commons:commons-compress:1.21
-- org.apache.kafka:kafka-clients:7.2.2-ccs
+- org.apache.kafka:kafka-clients:3.4.0
 - org.glassfish.jersey.core:jersey-common:2.30
 - org.xerial.snappy:snappy-java:1.1.8.3
 
diff --git a/flink-formats-kafka/pom.xml b/flink-formats-kafka/pom.xml
new file mode 100644
index 0000000..dbe67f5
--- /dev/null
+++ b/flink-formats-kafka/pom.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-connector-kafka-parent</artifactId>
+    <version>4.1-SNAPSHOT</version>
+  </parent>
+
+  <packaging>pom</packaging>
+
+  <artifactId>flink-formats-kafka</artifactId>
+  <name>Flink : Formats : Kafka</name>
+
+  <modules>
+    <module>flink-avro-confluent-registry</module>
+    <module>flink-json-debezium</module>
+    <module>flink-sql-avro-confluent-registry</module>
+  </modules>
+
+</project>
diff --git a/pom.xml b/pom.xml
index 2557a58..68a20c9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,6 +46,8 @@
         <module>flink-connector-kafka</module>
         <module>flink-sql-connector-kafka</module>
         <module>flink-connector-kafka-e2e-tests</module>
+        <module>flink-formats-kafka</module>
+        <module>flink-confluent-schema-registry-e2e-tests</module>
     </modules>
 
     <properties>
@@ -53,6 +55,7 @@
         <flink.shaded.version>16.1</flink.shaded.version>
         <kafka.version>3.4.0</kafka.version>
         <zookeeper.version>3.5.9</zookeeper.version>
+        <confluent.version>7.2.2</confluent.version>
 
         <jackson-bom.version>2.13.4.20221013</jackson-bom.version>
         <junit4.version>4.13.2</junit4.version>
@@ -60,9 +63,11 @@
         <assertj.version>3.23.1</assertj.version>
         <testcontainers.version>1.17.2</testcontainers.version>
         <mockito.version>3.4.6</mockito.version>
+        <powermock.version>2.0.9</powermock.version>
         <hamcrest.version>1.3</hamcrest.version>
         <byte-buddy.version>1.12.10</byte-buddy.version>
         <commons-cli.version>1.5.0</commons-cli.version>
+        <scala.binary.version>2.12</scala.binary.version>
         <scala-reflect.version>2.12.7</scala-reflect.version>
         <scala-library.version>2.12.7</scala-library.version>
         <snappy-java.version>1.1.8.3</snappy-java.version>