[FLINK-35442][cdc-connect][kafka] add key.format and partition.strategy option to make sure the same record sending to the same partition. (#3522)
parent
4bf5a395a5
commit
e0d6d1d1a8
@ -0,0 +1,117 @@
|
||||
/*
|
||||
* 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.serialization;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
import org.apache.flink.cdc.common.data.RecordData;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DataChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.Event;
|
||||
import org.apache.flink.cdc.common.event.OperationType;
|
||||
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Column;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.types.DataField;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
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.formats.csv.CsvRowDataSerializationSchema;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** A {@link SerializationSchema} to convert {@link Event} into byte of csv format. */
|
||||
public class CsvSerializationSchema implements SerializationSchema<Event> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* A map of {@link TableId} and its {@link SerializationSchema} to serialize Debezium JSON data.
|
||||
*/
|
||||
private final Map<TableId, TableSchemaInfo> csvSerializers;
|
||||
|
||||
private final ZoneId zoneId;
|
||||
|
||||
private InitializationContext context;
|
||||
|
||||
public CsvSerializationSchema(ZoneId zoneId) {
|
||||
this.zoneId = zoneId;
|
||||
csvSerializers = new HashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(InitializationContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] serialize(Event event) {
|
||||
if (event instanceof SchemaChangeEvent) {
|
||||
Schema schema;
|
||||
SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event;
|
||||
if (event instanceof CreateTableEvent) {
|
||||
CreateTableEvent createTableEvent = (CreateTableEvent) event;
|
||||
schema = createTableEvent.getSchema();
|
||||
} else {
|
||||
schema =
|
||||
SchemaUtils.applySchemaChangeEvent(
|
||||
csvSerializers.get(schemaChangeEvent.tableId()).getSchema(),
|
||||
schemaChangeEvent);
|
||||
}
|
||||
CsvRowDataSerializationSchema csvSerializer = buildSerializationForPrimaryKey(schema);
|
||||
try {
|
||||
csvSerializer.open(context);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
csvSerializers.put(
|
||||
schemaChangeEvent.tableId(),
|
||||
new TableSchemaInfo(
|
||||
schemaChangeEvent.tableId(), schema, csvSerializer, zoneId));
|
||||
return null;
|
||||
}
|
||||
DataChangeEvent dataChangeEvent = (DataChangeEvent) event;
|
||||
RecordData recordData =
|
||||
dataChangeEvent.op().equals(OperationType.DELETE)
|
||||
? dataChangeEvent.before()
|
||||
: dataChangeEvent.after();
|
||||
TableSchemaInfo tableSchemaInfo = csvSerializers.get(dataChangeEvent.tableId());
|
||||
return tableSchemaInfo
|
||||
.getSerializationSchema()
|
||||
.serialize(tableSchemaInfo.getRowDataFromRecordData(recordData, true));
|
||||
}
|
||||
|
||||
private CsvRowDataSerializationSchema buildSerializationForPrimaryKey(Schema schema) {
|
||||
DataField[] fields = new DataField[schema.primaryKeys().size() + 1];
|
||||
fields[0] = DataTypes.FIELD("TableId", DataTypes.STRING());
|
||||
for (int i = 0; i < schema.primaryKeys().size(); i++) {
|
||||
Column column = schema.getColumn(schema.primaryKeys().get(i)).get();
|
||||
fields[i + 1] = DataTypes.FIELD(column.getName(), column.getType());
|
||||
}
|
||||
// the row should never be null
|
||||
DataType dataType = DataTypes.ROW(fields).notNull();
|
||||
LogicalType rowType = DataTypeUtils.toFlinkDataType(dataType).getLogicalType();
|
||||
return new CsvRowDataSerializationSchema.Builder((RowType) rowType).build();
|
||||
}
|
||||
}
|
@ -0,0 +1,141 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.flink.cdc.connectors.kafka.serialization;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
import org.apache.flink.cdc.common.data.RecordData;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DataChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.Event;
|
||||
import org.apache.flink.cdc.common.event.OperationType;
|
||||
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Column;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.types.DataField;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
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.formats.common.TimestampFormat;
|
||||
import org.apache.flink.formats.json.JsonFormatOptions;
|
||||
import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** A {@link SerializationSchema} to convert {@link Event} into byte of json format. */
|
||||
public class JsonSerializationSchema implements SerializationSchema<Event> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* A map of {@link TableId} and its {@link SerializationSchema} to serialize Debezium JSON data.
|
||||
*/
|
||||
private final Map<TableId, TableSchemaInfo> jsonSerializers;
|
||||
|
||||
private final TimestampFormat timestampFormat;
|
||||
|
||||
private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode;
|
||||
|
||||
private final String mapNullKeyLiteral;
|
||||
|
||||
private final boolean encodeDecimalAsPlainNumber;
|
||||
|
||||
private final ZoneId zoneId;
|
||||
|
||||
private InitializationContext context;
|
||||
|
||||
public JsonSerializationSchema(
|
||||
TimestampFormat timestampFormat,
|
||||
JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
|
||||
String mapNullKeyLiteral,
|
||||
ZoneId zoneId,
|
||||
boolean encodeDecimalAsPlainNumber) {
|
||||
this.timestampFormat = timestampFormat;
|
||||
this.mapNullKeyMode = mapNullKeyMode;
|
||||
this.mapNullKeyLiteral = mapNullKeyLiteral;
|
||||
this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber;
|
||||
this.zoneId = zoneId;
|
||||
jsonSerializers = new HashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(InitializationContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] serialize(Event event) {
|
||||
if (event instanceof SchemaChangeEvent) {
|
||||
Schema schema;
|
||||
SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event;
|
||||
if (event instanceof CreateTableEvent) {
|
||||
CreateTableEvent createTableEvent = (CreateTableEvent) event;
|
||||
schema = createTableEvent.getSchema();
|
||||
} else {
|
||||
schema =
|
||||
SchemaUtils.applySchemaChangeEvent(
|
||||
jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(),
|
||||
schemaChangeEvent);
|
||||
}
|
||||
JsonRowDataSerializationSchema jsonSerializer = buildSerializationForPrimaryKey(schema);
|
||||
try {
|
||||
jsonSerializer.open(context);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
jsonSerializers.put(
|
||||
schemaChangeEvent.tableId(),
|
||||
new TableSchemaInfo(
|
||||
schemaChangeEvent.tableId(), schema, jsonSerializer, zoneId));
|
||||
return null;
|
||||
}
|
||||
DataChangeEvent dataChangeEvent = (DataChangeEvent) event;
|
||||
RecordData recordData =
|
||||
dataChangeEvent.op().equals(OperationType.DELETE)
|
||||
? dataChangeEvent.before()
|
||||
: dataChangeEvent.after();
|
||||
TableSchemaInfo tableSchemaInfo = jsonSerializers.get(dataChangeEvent.tableId());
|
||||
return tableSchemaInfo
|
||||
.getSerializationSchema()
|
||||
.serialize(tableSchemaInfo.getRowDataFromRecordData(recordData, true));
|
||||
}
|
||||
|
||||
private JsonRowDataSerializationSchema buildSerializationForPrimaryKey(Schema schema) {
|
||||
DataField[] fields = new DataField[schema.primaryKeys().size() + 1];
|
||||
fields[0] = DataTypes.FIELD("TableId", DataTypes.STRING());
|
||||
for (int i = 0; i < schema.primaryKeys().size(); i++) {
|
||||
Column column = schema.getColumn(schema.primaryKeys().get(i)).get();
|
||||
fields[i + 1] = DataTypes.FIELD(column.getName(), column.getType());
|
||||
}
|
||||
// the row should never be null
|
||||
DataType dataType = DataTypes.ROW(fields).notNull();
|
||||
LogicalType rowType = DataTypeUtils.toFlinkDataType(dataType).getLogicalType();
|
||||
return new JsonRowDataSerializationSchema(
|
||||
(RowType) rowType,
|
||||
timestampFormat,
|
||||
mapNullKeyMode,
|
||||
mapNullKeyLiteral,
|
||||
encodeDecimalAsPlainNumber);
|
||||
}
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.sink;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
|
||||
/** Enum class for building {@link SerializationSchema} for {@link ProducerRecord}. */
|
||||
public enum KeyFormat {
|
||||
JSON("json"),
|
||||
|
||||
CSV("csv");
|
||||
|
||||
private final String value;
|
||||
|
||||
KeyFormat(String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return value;
|
||||
}
|
||||
}
|
@ -0,0 +1,74 @@
|
||||
/*
|
||||
* 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.sink;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
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.configuration.ReadableConfig;
|
||||
import org.apache.flink.formats.common.TimestampFormat;
|
||||
import org.apache.flink.formats.json.JsonFormatOptions;
|
||||
import org.apache.flink.formats.json.JsonFormatOptionsUtil;
|
||||
|
||||
import java.time.ZoneId;
|
||||
|
||||
import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
|
||||
import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;
|
||||
|
||||
/**
|
||||
* Format factory for providing configured instances of {@link SerializationSchema} to convert
|
||||
* {@link Event} to byte.
|
||||
*/
|
||||
public class KeySerializationFactory {
|
||||
|
||||
/**
|
||||
* Creates a configured instance of {@link SerializationSchema} to convert {@link Event} to
|
||||
* byte.
|
||||
*/
|
||||
public static SerializationSchema<Event> createSerializationSchema(
|
||||
ReadableConfig formatOptions, KeyFormat keyFormat, ZoneId zoneId) {
|
||||
switch (keyFormat) {
|
||||
case JSON:
|
||||
{
|
||||
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 JsonSerializationSchema(
|
||||
timestampFormat,
|
||||
mapNullKeyMode,
|
||||
mapNullKeyLiteral,
|
||||
zoneId,
|
||||
encodeDecimalAsPlainNumber);
|
||||
}
|
||||
case CSV:
|
||||
{
|
||||
return new CsvSerializationSchema(zoneId);
|
||||
}
|
||||
default:
|
||||
{
|
||||
throw new IllegalArgumentException("UnSupport key format of " + keyFormat);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.sink;
|
||||
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
|
||||
/** Partition Strategy for sending {@link ProducerRecord} to kafka partition. */
|
||||
public enum PartitionStrategy {
|
||||
|
||||
/** All {@link ProducerRecord} will be sent to partition 0. */
|
||||
ALL_TO_ZERO("all-to-zero"),
|
||||
|
||||
/** {@link ProducerRecord} will be sent to specific partition by primary key. */
|
||||
HASH_BY_KEY("hash-by-key");
|
||||
|
||||
private final String value;
|
||||
|
||||
PartitionStrategy(String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return value;
|
||||
}
|
||||
}
|
@ -0,0 +1,126 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.flink.cdc.connectors.kafka.serialization;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DataChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.Event;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
import org.apache.flink.cdc.common.types.RowType;
|
||||
import org.apache.flink.cdc.connectors.kafka.json.MockInitializationContext;
|
||||
import org.apache.flink.cdc.connectors.kafka.sink.KeyFormat;
|
||||
import org.apache.flink.cdc.connectors.kafka.sink.KeySerializationFactory;
|
||||
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
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.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.ZoneId;
|
||||
|
||||
/** Tests for {@link CsvSerializationSchema}. */
|
||||
public class CsvSerializationSchemaTest {
|
||||
|
||||
public static final TableId TABLE_1 =
|
||||
TableId.tableId("default_namespace", "default_schema", "table1");
|
||||
|
||||
@Test
|
||||
public void testSerialize() throws Exception {
|
||||
ObjectMapper mapper =
|
||||
JacksonMapperFactory.createObjectMapper()
|
||||
.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false);
|
||||
SerializationSchema<Event> serializationSchema =
|
||||
KeySerializationFactory.createSerializationSchema(
|
||||
new Configuration(), KeyFormat.CSV, ZoneId.systemDefault());
|
||||
serializationSchema.open(new MockInitializationContext());
|
||||
|
||||
// create table
|
||||
Schema schema =
|
||||
Schema.newBuilder()
|
||||
.physicalColumn("col1", DataTypes.STRING())
|
||||
.physicalColumn("col2", DataTypes.STRING())
|
||||
.primaryKey("col1")
|
||||
.build();
|
||||
CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema);
|
||||
Assertions.assertNull(serializationSchema.serialize(createTableEvent));
|
||||
|
||||
// insert
|
||||
BinaryRecordDataGenerator generator =
|
||||
new BinaryRecordDataGenerator(RowType.of(DataTypes.STRING(), DataTypes.STRING()));
|
||||
DataChangeEvent insertEvent1 =
|
||||
DataChangeEvent.insertEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("1")
|
||||
}));
|
||||
String expected = "\"default_namespace.default_schema.table1\",1";
|
||||
String actual = new String(serializationSchema.serialize(insertEvent1));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent insertEvent2 =
|
||||
DataChangeEvent.insertEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("2"),
|
||||
BinaryStringData.fromString("2")
|
||||
}));
|
||||
expected = "\"default_namespace.default_schema.table1\",2";
|
||||
actual = new String(serializationSchema.serialize(insertEvent2));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent deleteEvent =
|
||||
DataChangeEvent.deleteEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("2"),
|
||||
BinaryStringData.fromString("2")
|
||||
}));
|
||||
expected = "\"default_namespace.default_schema.table1\",2";
|
||||
actual = new String(serializationSchema.serialize(deleteEvent));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent updateEvent =
|
||||
DataChangeEvent.updateEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("1")
|
||||
}),
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("x")
|
||||
}));
|
||||
expected = "\"default_namespace.default_schema.table1\",1";
|
||||
actual = new String(serializationSchema.serialize(updateEvent));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
}
|
||||
}
|
@ -0,0 +1,135 @@
|
||||
/*
|
||||
* 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.serialization;
|
||||
|
||||
import org.apache.flink.api.common.serialization.SerializationSchema;
|
||||
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DataChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.Event;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
import org.apache.flink.cdc.common.types.RowType;
|
||||
import org.apache.flink.cdc.connectors.kafka.json.MockInitializationContext;
|
||||
import org.apache.flink.cdc.connectors.kafka.sink.KeyFormat;
|
||||
import org.apache.flink.cdc.connectors.kafka.sink.KeySerializationFactory;
|
||||
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
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.JsonNode;
|
||||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.ZoneId;
|
||||
|
||||
/** Tests for {@link JsonSerializationSchema}. */
|
||||
public class JsonSerializationSchemaTest {
|
||||
|
||||
public static final TableId TABLE_1 =
|
||||
TableId.tableId("default_namespace", "default_schema", "table1");
|
||||
|
||||
@Test
|
||||
public void testSerialize() throws Exception {
|
||||
ObjectMapper mapper =
|
||||
JacksonMapperFactory.createObjectMapper()
|
||||
.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false);
|
||||
SerializationSchema<Event> serializationSchema =
|
||||
KeySerializationFactory.createSerializationSchema(
|
||||
new Configuration(), KeyFormat.JSON, ZoneId.systemDefault());
|
||||
serializationSchema.open(new MockInitializationContext());
|
||||
|
||||
// create table
|
||||
Schema schema =
|
||||
Schema.newBuilder()
|
||||
.physicalColumn("col1", DataTypes.STRING())
|
||||
.physicalColumn("col2", DataTypes.STRING())
|
||||
.primaryKey("col1")
|
||||
.build();
|
||||
CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema);
|
||||
Assertions.assertNull(serializationSchema.serialize(createTableEvent));
|
||||
|
||||
// insert
|
||||
BinaryRecordDataGenerator generator =
|
||||
new BinaryRecordDataGenerator(RowType.of(DataTypes.STRING(), DataTypes.STRING()));
|
||||
DataChangeEvent insertEvent1 =
|
||||
DataChangeEvent.insertEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("1")
|
||||
}));
|
||||
JsonNode expected =
|
||||
mapper.readTree(
|
||||
"{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"1\"}");
|
||||
JsonNode actual = mapper.readTree(serializationSchema.serialize(insertEvent1));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent insertEvent2 =
|
||||
DataChangeEvent.insertEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("2"),
|
||||
BinaryStringData.fromString("2")
|
||||
}));
|
||||
expected =
|
||||
mapper.readTree(
|
||||
"{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"2\"}");
|
||||
actual = mapper.readTree(serializationSchema.serialize(insertEvent2));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent deleteEvent =
|
||||
DataChangeEvent.deleteEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("2"),
|
||||
BinaryStringData.fromString("2")
|
||||
}));
|
||||
expected =
|
||||
mapper.readTree(
|
||||
"{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"2\"}");
|
||||
actual = mapper.readTree(serializationSchema.serialize(deleteEvent));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
|
||||
DataChangeEvent updateEvent =
|
||||
DataChangeEvent.updateEvent(
|
||||
TABLE_1,
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("1")
|
||||
}),
|
||||
generator.generate(
|
||||
new Object[] {
|
||||
BinaryStringData.fromString("1"),
|
||||
BinaryStringData.fromString("x")
|
||||
}));
|
||||
expected =
|
||||
mapper.readTree(
|
||||
"{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"1\"}");
|
||||
actual = mapper.readTree(serializationSchema.serialize(updateEvent));
|
||||
Assertions.assertEquals(expected, actual);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue