[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(<name>).as(<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>