Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36858][pipeline-connector][kafka] Fix compatibility with Flink 1.20 JsonRowDataSerializationSchema #3784

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,19 @@ limitations under the License.
</execution>
</executions>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<id>test-jar</id>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.connectors.kafka.json.canal.CanalJsonSerializationSchema;
import org.apache.flink.cdc.connectors.kafka.json.debezium.DebeziumJsonSerializationSchema;
import org.apache.flink.cdc.connectors.kafka.utils.JsonRowDataSerializationSchemaUtils;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
Expand Down Expand Up @@ -57,6 +58,9 @@ public static SerializationSchema<Event> createSerializationSchema(
final boolean encodeDecimalAsPlainNumber =
formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);

final boolean ignoreNullFields =
JsonRowDataSerializationSchemaUtils.enableIgnoreNullFields(formatOptions);

switch (type) {
case DEBEZIUM_JSON:
{
Expand All @@ -65,7 +69,8 @@ public static SerializationSchema<Event> createSerializationSchema(
mapNullKeyMode,
mapNullKeyLiteral,
zoneId,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
}
case CANAL_JSON:
{
Expand All @@ -74,7 +79,8 @@ public static SerializationSchema<Event> createSerializationSchema(
mapNullKeyMode,
mapNullKeyLiteral,
zoneId,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
}
default:
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.cdc.common.types.utils.DataTypeUtils;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.connectors.kafka.json.TableSchemaInfo;
import org.apache.flink.cdc.connectors.kafka.utils.JsonRowDataSerializationSchemaUtils;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
Expand Down Expand Up @@ -73,6 +74,8 @@ public class CanalJsonSerializationSchema implements SerializationSchema<Event>

private final boolean encodeDecimalAsPlainNumber;

private final boolean ignoreNullFields;

private final ZoneId zoneId;

private InitializationContext context;
Expand All @@ -82,13 +85,15 @@ public CanalJsonSerializationSchema(
JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
String mapNullKeyLiteral,
ZoneId zoneId,
boolean encodeDecimalAsPlainNumber) {
boolean encodeDecimalAsPlainNumber,
boolean ignoreNullFields) {
this.timestampFormat = timestampFormat;
this.mapNullKeyMode = mapNullKeyMode;
this.mapNullKeyLiteral = mapNullKeyLiteral;
this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber;
this.zoneId = zoneId;
jsonSerializers = new HashMap<>();
this.ignoreNullFields = ignoreNullFields;
}

@Override
Expand All @@ -114,12 +119,13 @@ public byte[] serialize(Event event) {
LogicalType rowType =
DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType();
JsonRowDataSerializationSchema jsonSerializer =
new JsonRowDataSerializationSchema(
JsonRowDataSerializationSchemaUtils.createSerializationSchema(
createJsonRowType(fromLogicalToDataType(rowType)),
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
try {
jsonSerializer.open(context);
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.cdc.common.types.utils.DataTypeUtils;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.connectors.kafka.json.TableSchemaInfo;
import org.apache.flink.cdc.connectors.kafka.utils.JsonRowDataSerializationSchemaUtils;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
Expand Down Expand Up @@ -72,6 +73,8 @@ public class DebeziumJsonSerializationSchema implements SerializationSchema<Even

private final boolean encodeDecimalAsPlainNumber;

private final boolean ignoreNullFields;

private final ZoneId zoneId;

private InitializationContext context;
Expand All @@ -81,13 +84,15 @@ public DebeziumJsonSerializationSchema(
JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
String mapNullKeyLiteral,
ZoneId zoneId,
boolean encodeDecimalAsPlainNumber) {
boolean encodeDecimalAsPlainNumber,
boolean ignoreNullFields) {
this.timestampFormat = timestampFormat;
this.mapNullKeyMode = mapNullKeyMode;
this.mapNullKeyLiteral = mapNullKeyLiteral;
this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber;
this.zoneId = zoneId;
jsonSerializers = new HashMap<>();
this.ignoreNullFields = ignoreNullFields;
}

@Override
Expand All @@ -113,12 +118,13 @@ public byte[] serialize(Event event) {
LogicalType rowType =
DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType();
JsonRowDataSerializationSchema jsonSerializer =
new JsonRowDataSerializationSchema(
JsonRowDataSerializationSchemaUtils.createSerializationSchema(
createJsonRowType(fromLogicalToDataType(rowType)),
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
try {
jsonSerializer.open(context);
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.flink.cdc.common.types.utils.DataTypeUtils;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.connectors.kafka.json.TableSchemaInfo;
import org.apache.flink.cdc.connectors.kafka.utils.JsonRowDataSerializationSchemaUtils;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
Expand Down Expand Up @@ -61,6 +62,8 @@ public class JsonSerializationSchema implements SerializationSchema<Event> {

private final boolean encodeDecimalAsPlainNumber;

private final boolean ignoreNullFields;

private final ZoneId zoneId;

private InitializationContext context;
Expand All @@ -70,13 +73,15 @@ public JsonSerializationSchema(
JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
String mapNullKeyLiteral,
ZoneId zoneId,
boolean encodeDecimalAsPlainNumber) {
boolean encodeDecimalAsPlainNumber,
boolean ignoreNullFields) {
this.timestampFormat = timestampFormat;
this.mapNullKeyMode = mapNullKeyMode;
this.mapNullKeyLiteral = mapNullKeyLiteral;
this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber;
this.zoneId = zoneId;
jsonSerializers = new HashMap<>();
this.ignoreNullFields = ignoreNullFields;
}

@Override
Expand Down Expand Up @@ -131,11 +136,12 @@ private JsonRowDataSerializationSchema buildSerializationForPrimaryKey(Schema sc
// the row should never be null
DataType dataType = DataTypes.ROW(fields).notNull();
LogicalType rowType = DataTypeUtils.toFlinkDataType(dataType).getLogicalType();
return new JsonRowDataSerializationSchema(
return JsonRowDataSerializationSchemaUtils.createSerializationSchema(
(RowType) rowType,
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.connectors.kafka.serialization.CsvSerializationSchema;
import org.apache.flink.cdc.connectors.kafka.serialization.JsonSerializationSchema;
import org.apache.flink.cdc.connectors.kafka.utils.JsonRowDataSerializationSchemaUtils;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
Expand Down Expand Up @@ -54,12 +55,16 @@ public static SerializationSchema<Event> createSerializationSchema(

final boolean encodeDecimalAsPlainNumber =
formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
final boolean ignoreNullFields =
JsonRowDataSerializationSchemaUtils.enableIgnoreNullFields(
formatOptions);
return new JsonSerializationSchema(
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
zoneId,
encodeDecimalAsPlainNumber);
encodeDecimalAsPlainNumber,
ignoreNullFields);
}
case CSV:
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
/*
* 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.cdc.connectors.kafka.utils;

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.JsonRowDataSerializationSchema;
import org.apache.flink.table.types.logical.RowType;

import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.util.Arrays;

/**
* Utils for creating JsonRowDataSerializationSchema.TODO: Remove this class after bump to Flink
* 1.20 or higher.
*/
public class JsonRowDataSerializationSchemaUtils {

/**
* In flink>=1.20, the constructor of JsonRowDataSerializationSchema has 6 parameters, and in
* flink<1.20, the constructor of JsonRowDataSerializationSchema has 5 parameters.
MOBIN-F marked this conversation as resolved.
Show resolved Hide resolved
*/
public static JsonRowDataSerializationSchema createSerializationSchema(
RowType rowType,
TimestampFormat timestampFormat,
JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
String mapNullKeyLiteral,
boolean encodeDecimalAsPlainNumber,
boolean ignoreNullFields) {
try {
Class<?>[] fullParams =
new Class[] {
RowType.class,
TimestampFormat.class,
JsonFormatOptions.MapNullKeyMode.class,
String.class,
boolean.class,
boolean.class
};

Object[] fullParamValues =
new Object[] {
rowType,
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
encodeDecimalAsPlainNumber,
ignoreNullFields
};

for (int i = fullParams.length; i >= 5; i--) {
try {
Constructor<?> constructor =
JsonRowDataSerializationSchema.class.getConstructor(
Arrays.copyOfRange(fullParams, 0, i));

return (JsonRowDataSerializationSchema)
constructor.newInstance(Arrays.copyOfRange(fullParamValues, 0, i));
} catch (NoSuchMethodException ignored) {
}
}
} catch (Exception e) {
throw new RuntimeException(
"Failed to create JsonRowDataSerializationSchema,please check your Flink version is 1.19 or 1.20.",
e);
}
throw new RuntimeException(
"Failed to find appropriate constructor for JsonRowDataSerializationSchema,please check your Flink version is 1.19 or 1.20.");
}

/** flink>=1.20 only has the ENCODE_IGNORE_NULL_FIELDS parameter. */
MOBIN-F marked this conversation as resolved.
Show resolved Hide resolved
public static boolean enableIgnoreNullFields(ReadableConfig formatOptions) {
try {
Field field = JsonFormatOptions.class.getField("ENCODE_IGNORE_NULL_FIELDS");
ConfigOption<Boolean> encodeOption = (ConfigOption<Boolean>) field.get(null);
return formatOptions.get(encodeOption);
} catch (NoSuchFieldException | IllegalAccessException e) {
return false;
}
}
}
24 changes: 24 additions & 0 deletions flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,13 @@ limitations under the License.
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-pipeline-connector-kafka</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-test-util</artifactId>
Expand Down Expand Up @@ -126,6 +133,13 @@ limitations under the License.
<version>${scala.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>kafka</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down Expand Up @@ -245,6 +259,16 @@ limitations under the License.
</outputDirectory>
</artifactItem>

<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-pipeline-connector-kafka</artifactId>
<version>${project.version}</version>
<destFileName>kafka-cdc-pipeline-connector.jar</destFileName>
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies
</outputDirectory>
</artifactItem>

<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-pipeline-udf-examples</artifactId>
Expand Down
Loading
Loading