From cdcf1ddc2a7a86e6050cdd8ac88e95baa8ec5c17 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Tue, 30 Apr 2024 10:48:29 +0800 Subject: [PATCH 01/12] add debezium avro format --- paimon-flink/paimon-flink-cdc/pom.xml | 8 + .../flink/action/cdc/format/DataFormat.java | 6 +- .../format/DeserializationSchemaFactory.java | 15 + .../debezium/DebeziumAvroRecordParser.java | 331 ++++++++++++++++++ ...ser.java => DebeziumJsonRecordParser.java} | 4 +- .../CdcDebeziumAvroDeserializationSchema.java | 106 ++++++ 6 files changed, 466 insertions(+), 4 deletions(-) create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java rename paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/{DebeziumRecordParser.java => DebeziumJsonRecordParser.java} (98%) create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 868eac71f313..8f67c13f596e 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -42,6 +42,7 @@ under the License. 2.9.0 1.19.1 4.0.0-1.17 + 7.5.0 @@ -124,6 +125,13 @@ under the License. provided + + org.apache.flink + flink-avro-confluent-registry + ${flink.version} + provided + + com.esri.geometry esri-geometry-api diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java index 28dc3e457c09..00f097d7a549 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java @@ -21,7 +21,8 @@ import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; import org.apache.paimon.flink.action.cdc.format.canal.CanalRecordParser; -import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumRecordParser; +import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumAvroRecordParser; +import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumJsonRecordParser; import org.apache.paimon.flink.action.cdc.format.maxwell.MaxwellRecordParser; import org.apache.paimon.flink.action.cdc.format.ogg.OggRecordParser; @@ -38,7 +39,8 @@ public enum DataFormat { CANAL_JSON(CanalRecordParser::new), OGG_JSON(OggRecordParser::new), MAXWELL_JSON(MaxwellRecordParser::new), - DEBEZIUM_JSON(DebeziumRecordParser::new); + DEBEZIUM_JSON(DebeziumJsonRecordParser::new), + DEBEZIUM_AVRO(DebeziumAvroRecordParser::new); // Add more data formats here if needed private final RecordParserFactory parser; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java new file mode 100644 index 000000000000..3bf372c63f89 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java @@ -0,0 +1,15 @@ +package org.apache.paimon.flink.action.cdc.format; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.flink.action.cdc.ComputedColumn; +import org.apache.paimon.flink.action.cdc.TypeMapping; + +import java.util.List; + +@FunctionalInterface +public interface DeserializationSchemaFactory { + + DeserializationSchema createDeserializationSchema( + boolean caseSensitive, TypeMapping typeMapping, List computedColumns); +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java new file mode 100644 index 000000000000..4ca71c8f1580 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java @@ -0,0 +1,331 @@ +/* + * 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.paimon.flink.action.cdc.format.debezium; + +import org.apache.paimon.flink.action.cdc.ComputedColumn; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.RecordParser; +import org.apache.paimon.flink.action.cdc.kafka.format.RecordParser; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.RowKind; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; + +import com.esri.core.geometry.ogc.OGCGeometry; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; +import io.confluent.kafka.serializers.GenericContainerWithVersion; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnCaseConvertAndDuplicateCheck; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnDuplicateErrMsg; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.listCaseConvert; +import static org.apache.paimon.utils.Preconditions.checkNotNull; + +/** + * Implementation of {@link RecordParser} for parsing messages in the Debezium avro format. + * + *

This parser handles records in the Debezium avro format and extracts relevant information to + * produce {@link RichCdcMultiplexRecord} objects. + */ +public class DebeziumAvroRecordParser extends RecordParser { + private static final Logger LOG = LoggerFactory.getLogger(DebeziumAvroRecordParser.class); + + private static final String FIELD_OP = "op"; + private static final String FIELD_SOURCE = "source"; + private static final String FIELD_SOURCE_DB = "db"; + private static final String FIELD_SOURCE_TABLE = "table"; + private static final String OP_READ = "r"; + private static final String OP_CREATE = "c"; + private static final String OP_UPDATE = "u"; + private static final String OP_DELETE = "d"; + private static final String OP_TRUNCATE = "t"; + private static final String OP_MESSAGE = "m"; + private static final Schema NULL_AVRO_SCHEMA = Schema.create(Schema.Type.NULL); + + public DebeziumAvroRecordParser( + boolean caseSensitive, TypeMapping typeMapping, List computedColumns) { + super(caseSensitive, typeMapping, computedColumns); + } + + @Override + protected List extractRecords() { + return Collections.emptyList(); + } + + @Override + protected String primaryField() { + return ""; + } + + @Override + protected String dataField() { + return ""; + } + + @Override + public List extractRecords(String topic, byte[] key, byte[] value) + throws IOException { + parseKeyValueRecord(topic, key, value); + // Skip debezium tombstone event + if (valueContainerWithVersion == null) { + return Collections.emptyList(); + } + validateFormat(); + extractPrimaryKeys(); + + return doExtractRecords(); + } + + @Override + protected List doExtractRecords() { + List records = new ArrayList<>(); + GenericRecord valueRecord = (GenericRecord) valueContainerWithVersion.container(); + String op = valueRecord.get(FIELD_OP).toString(); + GenericRecord source = (GenericRecord) valueRecord.get(FIELD_SOURCE); + databaseName = source.get(FIELD_SOURCE_DB).toString(); + tableName = source.get(FIELD_SOURCE_TABLE).toString(); + + switch (op) { + case OP_READ: + case OP_CREATE: + processRecord( + (GenericRecord) valueRecord.get(FIELD_AFTER), RowKind.INSERT, records); + break; + case OP_UPDATE: + processRecord( + (GenericRecord) valueRecord.get(FIELD_BEFORE), RowKind.DELETE, records); + processRecord( + (GenericRecord) valueRecord.get(FIELD_AFTER), RowKind.INSERT, records); + break; + case OP_DELETE: + processRecord( + (GenericRecord) valueRecord.get(FIELD_BEFORE), RowKind.DELETE, records); + break; + case OP_TRUNCATE: + case OP_MESSAGE: + LOG.info("Skip record operation: {}", op); + break; + default: + throw new UnsupportedOperationException("Unknown record operation: " + op); + } + + return records; + } + + private void processRecord( + GenericRecord payload, RowKind kind, List records) { + Map resultMap = new HashMap<>(); + LinkedHashMap paimonFieldTypes = setPaimonFieldType(); + + fieldDescriptors.forEach( + field -> { + String key = field.getColumnName(); + resultMap.put( + key, + payload.get(key) == null + ? null + : transformValue(payload.get(key), field)); + }); + + // generate values for computed columns + computedColumns.forEach( + computedColumn -> { + resultMap.put( + computedColumn.columnName(), + computedColumn.eval(resultMap.get(computedColumn.fieldReference()))); + paimonFieldTypes.put(computedColumn.columnName(), computedColumn.columnType()); + }); + + records.add(createRecord(kind, resultMap, paimonFieldTypes)); + } + + @Override + protected void extractPrimaryKeys() { + if (keyContainerWithVersion != null) { + GenericRecord key = (GenericRecord) keyContainerWithVersion.container(); + Schema keySchema = sanitizedSchema(key.getSchema()); + keySchema.getFields().stream().map(Schema.Field::name).forEach(primaryKeys::add); + } + } + + @Override + protected String format() { + return "debezium-avro"; + } + + @Override + protected LinkedHashMap setPaimonFieldType() { + LinkedHashMap paimonFieldTypes = new LinkedHashMap<>(); + GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); + Schema payloadSchema = + sanitizedSchema(((GenericRecord) value.get(FIELD_AFTER)).getSchema()); + + fieldDescriptors = + payloadSchema.getFields().stream() + .map( + filed -> + new DebeziumAvroFieldDescriptor( + sanitizedSchema(filed.schema()), + filed.name(), + primaryKeys.stream() + .anyMatch(key -> key.equals(filed.name())))) + .collect(Collectors.toList()); + + fieldDescriptors.forEach( + field -> paimonFieldTypes.put(field.getColumnName(), field.getPaimonType())); + + return paimonFieldTypes; + } + + @Override + public void validateFormat() { + String errorMessageTemplate = + "Didn't find '%s' property in avro. Only supports debezium-avro format," + + "please make sure your topic's format is correct."; + GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); + GenericRecord source = (GenericRecord) value.get(FIELD_SOURCE); + checkNotNull(source, errorMessageTemplate, FIELD_SOURCE); + checkNotNull(source.get(FIELD_SOURCE_DB), errorMessageTemplate, FIELD_SOURCE_DB); + checkNotNull(source.get(FIELD_SOURCE_TABLE), errorMessageTemplate, FIELD_SOURCE_TABLE); + } + + @Override + public org.apache.paimon.schema.Schema getKafkaSchema(String topic, byte[] key, byte[] value) { + parseKeyValueRecord(topic, key, value); + // Skip debezium tombstone event + if (valueContainerWithVersion == null) { + return null; + } + validateFormat(); + extractPrimaryKeys(); + GenericRecord source = + (GenericRecord) + ((GenericRecord) valueContainerWithVersion.container()).get(FIELD_SOURCE); + databaseName = source.get(FIELD_SOURCE_DB).toString(); + tableName = source.get(FIELD_SOURCE_TABLE).toString(); + LinkedHashMap paimonFieldTypes = setPaimonFieldType(); + + org.apache.paimon.schema.Schema.Builder builder = + org.apache.paimon.schema.Schema.newBuilder(); + Set existedFields = new HashSet<>(); + Function columnDuplicateErrMsg = columnDuplicateErrMsg(tableName); + for (Map.Entry entry : paimonFieldTypes.entrySet()) { + builder.column( + columnCaseConvertAndDuplicateCheck( + entry.getKey(), existedFields, caseSensitive, columnDuplicateErrMsg), + entry.getValue()); + } + builder.primaryKey(listCaseConvert(primaryKeys, caseSensitive)); + + return builder.build(); + } + + private Schema sanitizedSchema(Schema schema) { + if (schema.getType() == Schema.Type.UNION + && schema.getTypes().size() == 2 + && schema.getTypes().contains(NULL_AVRO_SCHEMA)) { + for (Schema memberSchema : schema.getTypes()) { + if (!memberSchema.equals(NULL_AVRO_SCHEMA)) { + return memberSchema; + } + } + } + return schema; + } + + private void parseKeyValueRecord(String topic, byte[] key, byte[] value) { + keyContainerWithVersion = deserializer.deserialize(topic, true, key); + valueContainerWithVersion = deserializer.deserialize(topic, false, value); + } + + private String transformValue( + Object originalValue, DebeziumAvroFieldDescriptor fieldDescriptor) { + if (fieldDescriptor.isSetType()) { + return String.format("[%s]", originalValue); + } else if (fieldDescriptor.isGeoType()) { + return convertWkbArray((GenericRecord) originalValue); + } else if (originalValue instanceof ByteBuffer) { + byte[] value = ((ByteBuffer) originalValue).array(); + if (fieldDescriptor.isDecimalPreciseType()) { + return new BigDecimal(new BigInteger(value), fieldDescriptor.getScale()) + .toPlainString(); + } + return new String(value, StandardCharsets.UTF_8); + } + return originalValue.toString(); + } + + public static String convertWkbArray(GenericRecord record) { + try { + String geoJson = OGCGeometry.fromBinary((ByteBuffer) record.get("wkb")).asGeoJson(); + JsonNode originGeoNode = OBJECT_MAPPER.readTree(geoJson); + + Map geometryInfo = new HashMap<>(); + String geometryType = originGeoNode.get("type").asText(); + geometryInfo.put("type", geometryType); + if (geometryType.equalsIgnoreCase("GeometryCollection")) { + geometryInfo.put("geometries", originGeoNode.get("geometries")); + } else { + geometryInfo.put("coordinates", originGeoNode.get("coordinates")); + } + int srid = record.get("srid") != null ? (int) record.get("srid") : 0; + geometryInfo.put("srid", srid); + + return OBJECT_MAPPER.writer().writeValueAsString(geometryInfo); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to convert %s to geometry JSON.", record), e); + } + } + + private static class Deserializer extends AbstractKafkaAvroDeserializer + implements Serializable { + + public Deserializer(SchemaRegistryClient client) { + this.schemaRegistry = client; + } + + public GenericContainerWithVersion deserialize( + String topic, boolean isKey, byte[] payload) { + return deserializeWithSchemaAndVersion(topic, isKey, payload); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java similarity index 98% rename from paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java rename to paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java index 4718564620a4..d6b666918264 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java @@ -63,7 +63,7 @@ * operation type, and primary key field names are used to construct the details of each record * event. */ -public class DebeziumRecordParser extends RecordParser { +public class DebeziumJsonRecordParser extends RecordParser { private static final String FIELD_SCHEMA = "schema"; protected static final String FIELD_PAYLOAD = "payload"; @@ -83,7 +83,7 @@ public class DebeziumRecordParser extends RecordParser { private final Map classNames = new HashMap<>(); private final Map> parameters = new HashMap<>(); - public DebeziumRecordParser( + public DebeziumJsonRecordParser( boolean caseSensitive, TypeMapping typeMapping, List computedColumns) { super(caseSensitive, typeMapping, computedColumns); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java new file mode 100644 index 000000000000..19146cd8d466 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java @@ -0,0 +1,106 @@ +/* + * 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.paimon.flink.action.cdc.serialization; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.format.avro.AvroSchemaConverter; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroDeserializationSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Map; + +import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; + +/** + * A simple deserialization schema for {@link CdcSourceRecord}. reference from {@link + * DebeziumAvroDeserializationSchema}. + */ +public class CdcDebeziumAvroDeserializationSchema + implements DeserializationSchema { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(CdcJsonDeserializationSchema.class); + + /** The deserializer to deserialize Debezium Avro data. */ + private final DeserializationSchema avroDeserializer; + + public CdcDebeziumAvroDeserializationSchema( + RowType rowType, + String schemaRegistryUrl, + @Nullable String schemaString, + @Nullable Map registryConfigs) { + Schema schema = + schemaString == null + ? AvroSchemaConverter.convertToSchema(rowType) + : new Parser().parse(schemaString); + avroDeserializer = + ConfluentRegistryAvroDeserializationSchema.forGeneric( + schema, schemaRegistryUrl, registryConfigs); + } + + @Override + public void open(InitializationContext context) throws Exception { + avroDeserializer.open(context); + } + + @Override + public CdcSourceRecord deserialize(byte[] message) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return null; + } + + GenericRecord record = this.avroDeserializer.deserialize(message); + return new CdcSourceRecord(record); + } + + @Override + public boolean isEndOfStream(CdcSourceRecord nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return getForClass(CdcSourceRecord.class); + } + + public static RowType createDebeziumAvroRowType(DataType databaseSchema) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + return DataTypes.ROW( + DataTypes.FIELD(0, "before", databaseSchema.nullable()), + DataTypes.FIELD(1, "after", databaseSchema.nullable()), + DataTypes.FIELD(2, "op", DataTypes.STRING())); + } +} From 200bcacc5ae2a153e7844ce24d59d8a114a74fb8 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Sat, 11 May 2024 17:01:19 +0800 Subject: [PATCH 02/12] Support debezium avro format in cdc action. --- paimon-flink/paimon-flink-cdc/pom.xml | 19 +- .../action/cdc/MessageQueueSchemaUtils.java | 10 + .../flink/action/cdc/SyncJobHandler.java | 16 +- .../flink/action/cdc/format/DataFormat.java | 56 +++- .../format/DeserializationSchemaFactory.java | 15 - .../action/cdc/format/JsonRecordParser.java | 185 +++++++++++ .../flink/action/cdc/format/RecordParser.java | 145 +------- .../cdc/format/canal/CanalRecordParser.java | 4 +- .../debezium/DebeziumAvroRecordParser.java | 310 +++++------------- .../debezium/DebeziumJsonRecordParser.java | 41 ++- .../format/debezium/DebeziumSchemaUtils.java | 125 ++++++- .../format/maxwell/MaxwellRecordParser.java | 14 +- .../cdc/format/ogg/OggRecordParser.java | 14 +- .../action/cdc/kafka/KafkaActionUtils.java | 66 ++-- ...afkaDebeziumAvroDeserializationSchema.java | 87 +++++ ...afkaDebeziumJsonDeserializationSchema.java | 83 +++++ .../action/cdc/mysql/MySqlTypeUtils.java | 6 +- .../action/cdc/pulsar/PulsarActionUtils.java | 112 ++++--- ...lsarDebeziumAvroDeserializationSchema.java | 82 +++++ .../CdcDebeziumAvroDeserializationSchema.java | 106 ------ .../CdcJsonDeserializationSchema.java | 5 + .../ConfluentAvroDeserializationSchema.java | 38 +++ .../action/cdc/kafka/KafkaSchemaITCase.java | 3 +- .../cdc/kafka/KafkaSyncTableActionITCase.java | 3 +- .../action/cdc/pulsar/PulsarSchemaITCase.java | 3 +- 25 files changed, 947 insertions(+), 601 deletions(-) delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/JsonRecordParser.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumJsonDeserializationSchema.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/ConfluentAvroDeserializationSchema.java diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 8f67c13f596e..a8e77ca59c4b 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -45,6 +45,13 @@ under the License. 7.5.0 + + + confluent + https://packages.confluent.io/maven/ + + + org.apache.paimon @@ -126,10 +133,16 @@ under the License. - org.apache.flink - flink-avro-confluent-registry - ${flink.version} + io.confluent + kafka-avro-serializer + ${confluent.platform.version} provided + + + org.apache.avro + avro + + diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSchemaUtils.java index 73e63bcd029d..59295e57ef48 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSchemaUtils.java @@ -23,6 +23,9 @@ import org.apache.paimon.flink.action.cdc.format.RecordParser; import org.apache.paimon.schema.Schema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + import java.util.Collections; import java.util.List; import java.util.Objects; @@ -31,6 +34,13 @@ /** Utility class to build schema by trying to read and parse records from message queue. */ public class MessageQueueSchemaUtils { + public static final ConfigOption SCHEMA_REGISTRY_URL = + ConfigOptions.key("schema.registry.url") + .stringType() + .noDefaultValue() + .withDescription( + "To use the Confluence schema registry model for Apache Avro serialization, you need to provide the schema registry URL."); + private static final int MAX_RETRY = 5; private static final int POLL_TIMEOUT_MILLIS = 1000; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java index c674e560b11f..2e3897bf14b7 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java @@ -183,9 +183,13 @@ public void checkRequiredOption() { public Source provideSource() { switch (sourceType) { case KAFKA: - return KafkaActionUtils.buildKafkaSource(cdcSourceConfig); + return KafkaActionUtils.buildKafkaSource( + cdcSourceConfig, + provideDataFormat().createKafkaDeserializer(cdcSourceConfig)); case PULSAR: - return PulsarActionUtils.buildPulsarSource(cdcSourceConfig); + return PulsarActionUtils.buildPulsarSource( + cdcSourceConfig, + provideDataFormat().createPulsarDeserializer(cdcSourceConfig)); default: throw new UnsupportedOperationException( "Cannot get source from source type" + sourceType); @@ -238,9 +242,13 @@ public DataFormat provideDataFormat() { public MessageQueueSchemaUtils.ConsumerWrapper provideConsumer() { switch (sourceType) { case KAFKA: - return KafkaActionUtils.getKafkaEarliestConsumer(cdcSourceConfig); + return KafkaActionUtils.getKafkaEarliestConsumer( + cdcSourceConfig, + provideDataFormat().createKafkaDeserializer(cdcSourceConfig)); case PULSAR: - return PulsarActionUtils.createPulsarConsumer(cdcSourceConfig); + return PulsarActionUtils.createPulsarConsumer( + cdcSourceConfig, + provideDataFormat().createPulsarDeserializer(cdcSourceConfig)); default: throw new UnsupportedOperationException( "Cannot get consumer from source type" + sourceType); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java index 00f097d7a549..003d382259ca 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.action.cdc.format; +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; import org.apache.paimon.flink.action.cdc.format.canal.CanalRecordParser; @@ -25,8 +26,17 @@ import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumJsonRecordParser; import org.apache.paimon.flink.action.cdc.format.maxwell.MaxwellRecordParser; import org.apache.paimon.flink.action.cdc.format.ogg.OggRecordParser; +import org.apache.paimon.flink.action.cdc.kafka.KafkaDebeziumAvroDeserializationSchema; +import org.apache.paimon.flink.action.cdc.kafka.KafkaDebeziumJsonDeserializationSchema; +import org.apache.paimon.flink.action.cdc.pulsar.PulsarDebeziumAvroDeserializationSchema; +import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; import java.util.List; +import java.util.function.Function; /** * Enumerates the supported data formats for message queue and provides a mechanism to create their @@ -36,17 +46,41 @@ * which can be used to create instances of {@link RecordParser} for that format. */ public enum DataFormat { - CANAL_JSON(CanalRecordParser::new), - OGG_JSON(OggRecordParser::new), - MAXWELL_JSON(MaxwellRecordParser::new), - DEBEZIUM_JSON(DebeziumJsonRecordParser::new), - DEBEZIUM_AVRO(DebeziumAvroRecordParser::new); + CANAL_JSON( + CanalRecordParser::new, + KafkaDebeziumJsonDeserializationSchema::new, + CdcJsonDeserializationSchema::new), + OGG_JSON( + OggRecordParser::new, + KafkaDebeziumJsonDeserializationSchema::new, + CdcJsonDeserializationSchema::new), + MAXWELL_JSON( + MaxwellRecordParser::new, + KafkaDebeziumJsonDeserializationSchema::new, + CdcJsonDeserializationSchema::new), + DEBEZIUM_JSON( + DebeziumJsonRecordParser::new, + KafkaDebeziumJsonDeserializationSchema::new, + CdcJsonDeserializationSchema::new), + DEBEZIUM_AVRO( + DebeziumAvroRecordParser::new, + KafkaDebeziumAvroDeserializationSchema::new, + PulsarDebeziumAvroDeserializationSchema::new); // Add more data formats here if needed private final RecordParserFactory parser; + private final Function> + kafkaDeserializer; + private final Function> + pulsarDeserializer; - DataFormat(RecordParserFactory parser) { + DataFormat( + RecordParserFactory parser, + Function> kafkaDeserializer, + Function> pulsarDeserializer) { this.parser = parser; + this.kafkaDeserializer = kafkaDeserializer; + this.pulsarDeserializer = pulsarDeserializer; } /** @@ -62,6 +96,16 @@ public RecordParser createParser( return parser.createParser(caseSensitive, typeMapping, computedColumns); } + public KafkaDeserializationSchema createKafkaDeserializer( + Configuration cdcSourceConfig) { + return kafkaDeserializer.apply(cdcSourceConfig); + } + + public DeserializationSchema createPulsarDeserializer( + Configuration cdcSourceConfig) { + return pulsarDeserializer.apply(cdcSourceConfig); + } + public static DataFormat fromConfigString(String format) { try { return DataFormat.valueOf(format.replace("-", "_").toUpperCase()); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java deleted file mode 100644 index 3bf372c63f89..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DeserializationSchemaFactory.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.paimon.flink.action.cdc.format; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.paimon.flink.action.cdc.CdcSourceRecord; -import org.apache.paimon.flink.action.cdc.ComputedColumn; -import org.apache.paimon.flink.action.cdc.TypeMapping; - -import java.util.List; - -@FunctionalInterface -public interface DeserializationSchemaFactory { - - DeserializationSchema createDeserializationSchema( - boolean caseSensitive, TypeMapping typeMapping, List computedColumns); -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/JsonRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/JsonRecordParser.java new file mode 100644 index 000000000000..55cf22238b5c --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/JsonRecordParser.java @@ -0,0 +1,185 @@ +/* + * 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.paimon.flink.action.cdc.format; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.flink.action.cdc.ComputedColumn; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.utils.TypeUtils; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.paimon.utils.JsonSerdeUtil.convertValue; +import static org.apache.paimon.utils.JsonSerdeUtil.getNodeAs; +import static org.apache.paimon.utils.JsonSerdeUtil.isNull; +import static org.apache.paimon.utils.JsonSerdeUtil.writeValueAsString; + +/** + * Provides a base implementation for parsing messages of various formats into {@link + * RichCdcMultiplexRecord} objects. + * + *

This abstract class defines common functionalities and fields required for parsing messages. + * Subclasses are expected to provide specific implementations for extracting records, validating + * message formats, and other format-specific operations. + */ +public abstract class JsonRecordParser extends RecordParser { + private static final Logger LOG = LoggerFactory.getLogger(JsonRecordParser.class); + + protected JsonNode root; + + public JsonRecordParser( + boolean caseSensitive, TypeMapping typeMapping, List computedColumns) { + super(caseSensitive, typeMapping, computedColumns); + } + + protected void setRoot(CdcSourceRecord record) { + root = (JsonNode) record.getValue(); + } + + protected abstract String primaryField(); + + protected abstract String dataField(); + + // use STRING type in default when we cannot get origin data types (most cases) + protected LinkedHashMap fillDefaultTypes(JsonNode record) { + LinkedHashMap fieldTypes = new LinkedHashMap<>(); + record.fieldNames().forEachRemaining(name -> fieldTypes.put(name, DataTypes.STRING())); + return fieldTypes; + } + + protected Map extractRowData( + JsonNode record, LinkedHashMap paimonFieldTypes) { + paimonFieldTypes.putAll(fillDefaultTypes(record)); + Map recordMap = + convertValue(record, new TypeReference>() {}); + Map rowData = + recordMap.entrySet().stream() + .filter(entry -> Objects.nonNull(entry.getKey())) + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> { + if (Objects.nonNull(entry.getValue()) + && !TypeUtils.isBasicType(entry.getValue())) { + try { + return writeValueAsString(entry.getValue()); + } catch (JsonProcessingException e) { + LOG.error("Failed to deserialize record.", e); + return Objects.toString(entry.getValue()); + } + } + return Objects.toString(entry.getValue()); + })); + evalComputedColumns(rowData, paimonFieldTypes); + return rowData; + } + + @Override + protected List extractPrimaryKeys() { + ArrayNode pkNames = getNodeAs(root, primaryField(), ArrayNode.class); + if (pkNames == null) { + return Collections.emptyList(); + } + + return StreamSupport.stream(pkNames.spliterator(), false) + .map(JsonNode::asText) + .collect(Collectors.toList()); + } + + protected void processRecord( + JsonNode jsonNode, RowKind rowKind, List records) { + LinkedHashMap paimonFieldTypes = new LinkedHashMap<>(jsonNode.size()); + Map rowData = this.extractRowData(jsonNode, paimonFieldTypes); + records.add(createRecord(rowKind, rowData, paimonFieldTypes)); + } + + protected JsonNode mergeOldRecord(JsonNode data, JsonNode oldNode) { + JsonNode oldFullRecordNode = data.deepCopy(); + oldNode.fieldNames() + .forEachRemaining( + fieldName -> + ((ObjectNode) oldFullRecordNode) + .set(fieldName, oldNode.get(fieldName))); + return oldFullRecordNode; + } + + @Nullable + @Override + protected String getTableName() { + JsonNode node = root.get(FIELD_TABLE); + return isNull(node) ? null : node.asText(); + } + + @Nullable + @Override + protected String getDatabaseName() { + JsonNode node = root.get(FIELD_DATABASE); + return isNull(node) ? null : node.asText(); + } + + protected void checkNotNull(JsonNode node, String key) { + if (isNull(node)) { + throw new RuntimeException( + String.format("Invalid %s format: missing '%s' field.", format(), key)); + } + } + + protected void checkNotNull( + JsonNode node, String key, String conditionKey, String conditionValue) { + if (isNull(node)) { + throw new RuntimeException( + String.format( + "Invalid %s format: missing '%s' field when '%s' is '%s'.", + format(), key, conditionKey, conditionValue)); + } + } + + protected JsonNode getAndCheck(String key) { + JsonNode node = root.get(key); + checkNotNull(node, key); + return node; + } + + protected JsonNode getAndCheck(String key, String conditionKey, String conditionValue) { + JsonNode node = root.get(key); + checkNotNull(node, key, conditionKey, conditionValue); + return node; + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParser.java index f4543759aa0d..c987937ebf4e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParser.java @@ -25,15 +25,7 @@ import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; -import org.apache.paimon.utils.TypeUtils; - -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; @@ -42,23 +34,15 @@ import javax.annotation.Nullable; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnDuplicateErrMsg; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.listCaseConvert; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.mapKeyCaseConvert; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.recordKeyDuplicateErrMsg; -import static org.apache.paimon.utils.JsonSerdeUtil.convertValue; -import static org.apache.paimon.utils.JsonSerdeUtil.getNodeAs; -import static org.apache.paimon.utils.JsonSerdeUtil.isNull; -import static org.apache.paimon.utils.JsonSerdeUtil.writeValueAsString; /** * Provides a base implementation for parsing messages of various formats into {@link @@ -75,12 +59,10 @@ public abstract class RecordParser protected static final String FIELD_TABLE = "table"; protected static final String FIELD_DATABASE = "database"; - private final boolean caseSensitive; + protected final boolean caseSensitive; protected final TypeMapping typeMapping; protected final List computedColumns; - protected JsonNode root; - public RecordParser( boolean caseSensitive, TypeMapping typeMapping, List computedColumns) { this.caseSensitive = caseSensitive; @@ -111,23 +93,6 @@ public Schema buildSchema(CdcSourceRecord record) { } } - protected abstract List extractRecords(); - - protected abstract String primaryField(); - - protected abstract String dataField(); - - protected boolean isDDL() { - return false; - } - - // use STRING type in default when we cannot get origin data types (most cases) - protected LinkedHashMap fillDefaultTypes(JsonNode record) { - LinkedHashMap fieldTypes = new LinkedHashMap<>(); - record.fieldNames().forEachRemaining(name -> fieldTypes.put(name, DataTypes.STRING())); - return fieldTypes; - } - @Override public void flatMap(CdcSourceRecord value, Collector out) { try { @@ -139,34 +104,17 @@ public void flatMap(CdcSourceRecord value, Collector out } } - protected Map extractRowData( - JsonNode record, LinkedHashMap paimonFieldTypes) { - paimonFieldTypes.putAll(fillDefaultTypes(record)); - Map recordMap = - convertValue(record, new TypeReference>() {}); - Map rowData = - recordMap.entrySet().stream() - .filter(entry -> Objects.nonNull(entry.getKey())) - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> { - if (Objects.nonNull(entry.getValue()) - && !TypeUtils.isBasicType(entry.getValue())) { - try { - return writeValueAsString(entry.getValue()); - } catch (JsonProcessingException e) { - LOG.error("Failed to deserialize record.", e); - return Objects.toString(entry.getValue()); - } - } - return Objects.toString(entry.getValue()); - })); - evalComputedColumns(rowData, paimonFieldTypes); - return rowData; + protected abstract void setRoot(CdcSourceRecord record); + + protected abstract List extractRecords(); + + protected boolean isDDL() { + return false; } - // generate values for computed columns + protected abstract List extractPrimaryKeys(); + + /** generate values for computed columns. */ protected void evalComputedColumns( Map rowData, LinkedHashMap paimonFieldTypes) { computedColumns.forEach( @@ -178,26 +126,8 @@ protected void evalComputedColumns( }); } - private List extractPrimaryKeys() { - ArrayNode pkNames = getNodeAs(root, primaryField(), ArrayNode.class); - if (pkNames == null) { - return Collections.emptyList(); - } - - return StreamSupport.stream(pkNames.spliterator(), false) - .map(JsonNode::asText) - .collect(Collectors.toList()); - } - - protected void processRecord( - JsonNode jsonNode, RowKind rowKind, List records) { - LinkedHashMap paimonFieldTypes = new LinkedHashMap<>(jsonNode.size()); - Map rowData = this.extractRowData(jsonNode, paimonFieldTypes); - records.add(createRecord(rowKind, rowData, paimonFieldTypes)); - } - /** Handle case sensitivity here. */ - private RichCdcMultiplexRecord createRecord( + protected RichCdcMultiplexRecord createRecord( RowKind rowKind, Map data, LinkedHashMap paimonFieldTypes) { @@ -219,64 +149,15 @@ private RichCdcMultiplexRecord createRecord( new CdcRecord(rowKind, data)); } - protected void setRoot(CdcSourceRecord record) { - root = (JsonNode) record.getValue(); - } - - protected JsonNode mergeOldRecord(JsonNode data, JsonNode oldNode) { - JsonNode oldFullRecordNode = data.deepCopy(); - oldNode.fieldNames() - .forEachRemaining( - fieldName -> - ((ObjectNode) oldFullRecordNode) - .set(fieldName, oldNode.get(fieldName))); - return oldFullRecordNode; - } - @Nullable - protected String getTableName() { - JsonNode node = root.get(FIELD_TABLE); - return isNull(node) ? null : node.asText(); - } + protected abstract String getTableName(); @Nullable - protected String getDatabaseName() { - JsonNode node = root.get(FIELD_DATABASE); - return isNull(node) ? null : node.asText(); - } + protected abstract String getDatabaseName(); private void logInvalidSourceRecord(CdcSourceRecord record) { LOG.error("Invalid source record:\n{}", record.toString()); } - protected void checkNotNull(JsonNode node, String key) { - if (isNull(node)) { - throw new RuntimeException( - String.format("Invalid %s format: missing '%s' field.", format(), key)); - } - } - - protected void checkNotNull( - JsonNode node, String key, String conditionKey, String conditionValue) { - if (isNull(node)) { - throw new RuntimeException( - String.format( - "Invalid %s format: missing '%s' field when '%s' is '%s'.", - format(), key, conditionKey, conditionValue)); - } - } - - protected JsonNode getAndCheck(String key) { - JsonNode node = root.get(key); - checkNotNull(node, key); - return node; - } - - protected JsonNode getAndCheck(String key, String conditionKey, String conditionValue) { - JsonNode node = root.get(key); - checkNotNull(node, key, conditionKey, conditionValue); - return node; - } - protected abstract String format(); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/canal/CanalRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/canal/CanalRecordParser.java index c7cd06c18fb0..93e1aa41fc80 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/canal/CanalRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/canal/CanalRecordParser.java @@ -20,7 +20,7 @@ import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.RecordParser; +import org.apache.paimon.flink.action.cdc.format.JsonRecordParser; import org.apache.paimon.flink.action.cdc.mysql.MySqlTypeUtils; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.DataType; @@ -63,7 +63,7 @@ *

Additionally, the parser supports schema extraction, which can be used to understand the * structure of the incoming data and its corresponding field types. */ -public class CanalRecordParser extends RecordParser { +public class CanalRecordParser extends JsonRecordParser { private static final Logger LOG = LoggerFactory.getLogger(CanalRecordParser.class); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java index 4ca71c8f1580..19915fa6475b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java @@ -18,45 +18,44 @@ package org.apache.paimon.flink.action.cdc.format.debezium; +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; import org.apache.paimon.flink.action.cdc.format.RecordParser; -import org.apache.paimon.flink.action.cdc.kafka.format.RecordParser; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowKind; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import com.esri.core.geometry.ogc.OGCGeometry; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; -import io.confluent.kafka.serializers.GenericContainerWithVersion; +import io.debezium.data.geometry.Geometry; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Serializable; -import java.math.BigDecimal; -import java.math.BigInteger; +import javax.annotation.Nullable; + import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.function.Function; +import java.util.Objects; import java.util.stream.Collectors; -import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnCaseConvertAndDuplicateCheck; -import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnDuplicateErrMsg; -import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.listCaseConvert; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_AFTER; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_BEFORE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_DB; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_SOURCE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_TYPE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_DELETE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_INSERT; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_MESSAGE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_READE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_TRUNCATE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_UPDATE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.avroToPaimonDataType; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** @@ -68,122 +67,91 @@ public class DebeziumAvroRecordParser extends RecordParser { private static final Logger LOG = LoggerFactory.getLogger(DebeziumAvroRecordParser.class); - private static final String FIELD_OP = "op"; - private static final String FIELD_SOURCE = "source"; - private static final String FIELD_SOURCE_DB = "db"; - private static final String FIELD_SOURCE_TABLE = "table"; - private static final String OP_READ = "r"; - private static final String OP_CREATE = "c"; - private static final String OP_UPDATE = "u"; - private static final String OP_DELETE = "d"; - private static final String OP_TRUNCATE = "t"; - private static final String OP_MESSAGE = "m"; private static final Schema NULL_AVRO_SCHEMA = Schema.create(Schema.Type.NULL); + private GenericRecord keyRecord; + private GenericRecord valueRecord; + public DebeziumAvroRecordParser( boolean caseSensitive, TypeMapping typeMapping, List computedColumns) { super(caseSensitive, typeMapping, computedColumns); } @Override - protected List extractRecords() { - return Collections.emptyList(); - } - - @Override - protected String primaryField() { - return ""; + protected void setRoot(CdcSourceRecord record) { + keyRecord = (GenericRecord) record.getKey(); + valueRecord = (GenericRecord) record.getValue(); } @Override - protected String dataField() { - return ""; - } - - @Override - public List extractRecords(String topic, byte[] key, byte[] value) - throws IOException { - parseKeyValueRecord(topic, key, value); - // Skip debezium tombstone event - if (valueContainerWithVersion == null) { - return Collections.emptyList(); - } - validateFormat(); - extractPrimaryKeys(); - - return doExtractRecords(); - } - - @Override - protected List doExtractRecords() { + protected List extractRecords() { + String operation = getAndCheck(FIELD_TYPE).toString(); List records = new ArrayList<>(); - GenericRecord valueRecord = (GenericRecord) valueContainerWithVersion.container(); - String op = valueRecord.get(FIELD_OP).toString(); - GenericRecord source = (GenericRecord) valueRecord.get(FIELD_SOURCE); - databaseName = source.get(FIELD_SOURCE_DB).toString(); - tableName = source.get(FIELD_SOURCE_TABLE).toString(); - - switch (op) { - case OP_READ: - case OP_CREATE: - processRecord( - (GenericRecord) valueRecord.get(FIELD_AFTER), RowKind.INSERT, records); + switch (operation) { + case OP_READE: + case OP_INSERT: + processRecord((GenericRecord) getAndCheck(FIELD_AFTER), RowKind.INSERT, records); break; case OP_UPDATE: - processRecord( - (GenericRecord) valueRecord.get(FIELD_BEFORE), RowKind.DELETE, records); - processRecord( - (GenericRecord) valueRecord.get(FIELD_AFTER), RowKind.INSERT, records); + processRecord((GenericRecord) getAndCheck(FIELD_BEFORE), RowKind.DELETE, records); + processRecord((GenericRecord) getAndCheck(FIELD_AFTER), RowKind.INSERT, records); break; case OP_DELETE: - processRecord( - (GenericRecord) valueRecord.get(FIELD_BEFORE), RowKind.DELETE, records); + processRecord((GenericRecord) getAndCheck(FIELD_BEFORE), RowKind.DELETE, records); break; case OP_TRUNCATE: case OP_MESSAGE: - LOG.info("Skip record operation: {}", op); + LOG.info("Skip record operation: {}", operation); break; default: - throw new UnsupportedOperationException("Unknown record operation: " + op); + throw new UnsupportedOperationException("Unknown record operation: " + operation); } return records; } private void processRecord( - GenericRecord payload, RowKind kind, List records) { - Map resultMap = new HashMap<>(); - LinkedHashMap paimonFieldTypes = setPaimonFieldType(); - - fieldDescriptors.forEach( - field -> { - String key = field.getColumnName(); - resultMap.put( - key, - payload.get(key) == null - ? null - : transformValue(payload.get(key), field)); - }); - - // generate values for computed columns - computedColumns.forEach( - computedColumn -> { - resultMap.put( - computedColumn.columnName(), - computedColumn.eval(resultMap.get(computedColumn.fieldReference()))); - paimonFieldTypes.put(computedColumn.columnName(), computedColumn.columnType()); - }); - - records.add(createRecord(kind, resultMap, paimonFieldTypes)); + GenericRecord record, RowKind rowKind, List records) { + LinkedHashMap paimonFieldTypes = new LinkedHashMap<>(); + Map rowData = this.extractRowData(record, paimonFieldTypes); + records.add(createRecord(rowKind, rowData, paimonFieldTypes)); } @Override - protected void extractPrimaryKeys() { - if (keyContainerWithVersion != null) { - GenericRecord key = (GenericRecord) keyContainerWithVersion.container(); - Schema keySchema = sanitizedSchema(key.getSchema()); - keySchema.getFields().stream().map(Schema.Field::name).forEach(primaryKeys::add); + protected List extractPrimaryKeys() { + if (keyRecord == null) { + return Collections.emptyList(); } + Schema keySchema = sanitizedSchema(keyRecord.getSchema()); + return keySchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + } + + private Map extractRowData( + GenericRecord record, LinkedHashMap paimonFieldTypes) { + Schema payloadSchema = sanitizedSchema(record.getSchema()); + + LinkedHashMap resultMap = new LinkedHashMap<>(); + for (Schema.Field field : payloadSchema.getFields()) { + Schema schema = sanitizedSchema(field.schema()); + String fieldName = field.name(); + String rawValue = Objects.toString(record.get(fieldName), null); + String transformed = + DebeziumSchemaUtils.transformRawValue( + rawValue, + schema.getLogicalType().getName(), + schema.getFullName(), + typeMapping, + () -> + (ByteBuffer) + ((GenericRecord) record.get(fieldName)) + .get(Geometry.WKB_FIELD), + ZoneOffset.UTC); + resultMap.put(fieldName, transformed); + paimonFieldTypes.put(fieldName, avroToPaimonDataType(schema)); + } + + evalComputedColumns(resultMap, paimonFieldTypes); + return resultMap; } @Override @@ -191,73 +159,6 @@ protected String format() { return "debezium-avro"; } - @Override - protected LinkedHashMap setPaimonFieldType() { - LinkedHashMap paimonFieldTypes = new LinkedHashMap<>(); - GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); - Schema payloadSchema = - sanitizedSchema(((GenericRecord) value.get(FIELD_AFTER)).getSchema()); - - fieldDescriptors = - payloadSchema.getFields().stream() - .map( - filed -> - new DebeziumAvroFieldDescriptor( - sanitizedSchema(filed.schema()), - filed.name(), - primaryKeys.stream() - .anyMatch(key -> key.equals(filed.name())))) - .collect(Collectors.toList()); - - fieldDescriptors.forEach( - field -> paimonFieldTypes.put(field.getColumnName(), field.getPaimonType())); - - return paimonFieldTypes; - } - - @Override - public void validateFormat() { - String errorMessageTemplate = - "Didn't find '%s' property in avro. Only supports debezium-avro format," - + "please make sure your topic's format is correct."; - GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); - GenericRecord source = (GenericRecord) value.get(FIELD_SOURCE); - checkNotNull(source, errorMessageTemplate, FIELD_SOURCE); - checkNotNull(source.get(FIELD_SOURCE_DB), errorMessageTemplate, FIELD_SOURCE_DB); - checkNotNull(source.get(FIELD_SOURCE_TABLE), errorMessageTemplate, FIELD_SOURCE_TABLE); - } - - @Override - public org.apache.paimon.schema.Schema getKafkaSchema(String topic, byte[] key, byte[] value) { - parseKeyValueRecord(topic, key, value); - // Skip debezium tombstone event - if (valueContainerWithVersion == null) { - return null; - } - validateFormat(); - extractPrimaryKeys(); - GenericRecord source = - (GenericRecord) - ((GenericRecord) valueContainerWithVersion.container()).get(FIELD_SOURCE); - databaseName = source.get(FIELD_SOURCE_DB).toString(); - tableName = source.get(FIELD_SOURCE_TABLE).toString(); - LinkedHashMap paimonFieldTypes = setPaimonFieldType(); - - org.apache.paimon.schema.Schema.Builder builder = - org.apache.paimon.schema.Schema.newBuilder(); - Set existedFields = new HashSet<>(); - Function columnDuplicateErrMsg = columnDuplicateErrMsg(tableName); - for (Map.Entry entry : paimonFieldTypes.entrySet()) { - builder.column( - columnCaseConvertAndDuplicateCheck( - entry.getKey(), existedFields, caseSensitive, columnDuplicateErrMsg), - entry.getValue()); - } - builder.primaryKey(listCaseConvert(primaryKeys, caseSensitive)); - - return builder.build(); - } - private Schema sanitizedSchema(Schema schema) { if (schema.getType() == Schema.Type.UNION && schema.getTypes().size() == 2 @@ -271,61 +172,30 @@ private Schema sanitizedSchema(Schema schema) { return schema; } - private void parseKeyValueRecord(String topic, byte[] key, byte[] value) { - keyContainerWithVersion = deserializer.deserialize(topic, true, key); - valueContainerWithVersion = deserializer.deserialize(topic, false, value); + @Nullable + @Override + protected String getTableName() { + return getFromSourceField(FIELD_TABLE); } - private String transformValue( - Object originalValue, DebeziumAvroFieldDescriptor fieldDescriptor) { - if (fieldDescriptor.isSetType()) { - return String.format("[%s]", originalValue); - } else if (fieldDescriptor.isGeoType()) { - return convertWkbArray((GenericRecord) originalValue); - } else if (originalValue instanceof ByteBuffer) { - byte[] value = ((ByteBuffer) originalValue).array(); - if (fieldDescriptor.isDecimalPreciseType()) { - return new BigDecimal(new BigInteger(value), fieldDescriptor.getScale()) - .toPlainString(); - } - return new String(value, StandardCharsets.UTF_8); - } - return originalValue.toString(); + @Nullable + @Override + protected String getDatabaseName() { + return getFromSourceField(FIELD_DB); } - public static String convertWkbArray(GenericRecord record) { - try { - String geoJson = OGCGeometry.fromBinary((ByteBuffer) record.get("wkb")).asGeoJson(); - JsonNode originGeoNode = OBJECT_MAPPER.readTree(geoJson); - - Map geometryInfo = new HashMap<>(); - String geometryType = originGeoNode.get("type").asText(); - geometryInfo.put("type", geometryType); - if (geometryType.equalsIgnoreCase("GeometryCollection")) { - geometryInfo.put("geometries", originGeoNode.get("geometries")); - } else { - geometryInfo.put("coordinates", originGeoNode.get("coordinates")); - } - int srid = record.get("srid") != null ? (int) record.get("srid") : 0; - geometryInfo.put("srid", srid); - - return OBJECT_MAPPER.writer().writeValueAsString(geometryInfo); - } catch (Exception e) { - throw new RuntimeException( - String.format("Failed to convert %s to geometry JSON.", record), e); + @Nullable + private String getFromSourceField(String key) { + GenericRecord source = (GenericRecord) valueRecord.get(FIELD_SOURCE); + if (Objects.isNull(source)) { + return null; } + return (String) source.get(key); } - private static class Deserializer extends AbstractKafkaAvroDeserializer - implements Serializable { - - public Deserializer(SchemaRegistryClient client) { - this.schemaRegistry = client; - } - - public GenericContainerWithVersion deserialize( - String topic, boolean isKey, byte[] payload) { - return deserializeWithSchemaAndVersion(topic, isKey, payload); - } + protected Object getAndCheck(String key) { + Object node = valueRecord.get(key); + checkNotNull(node, key); + return node; } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java index d6b666918264..5d7e11baf9fc 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java @@ -21,7 +21,7 @@ import org.apache.paimon.flink.action.cdc.CdcSourceRecord; import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.RecordParser; +import org.apache.paimon.flink.action.cdc.format.JsonRecordParser; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowKind; @@ -43,14 +43,26 @@ import java.util.Map; import java.util.Objects; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_AFTER; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_BEFORE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_DB; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_PAYLOAD; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_PRIMARY; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_SCHEMA; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_SOURCE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_TYPE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_DELETE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_INSERT; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_READE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_UPDATE; import static org.apache.paimon.utils.JsonSerdeUtil.getNodeAs; import static org.apache.paimon.utils.JsonSerdeUtil.isNull; /** - * The {@code DebeziumRecordParser} class extends the abstract {@link RecordParser} and is designed - * to parse records from Debezium's JSON change data capture (CDC) format. Debezium is a CDC - * solution for MySQL databases that captures row-level changes to database tables and outputs them - * in JSON format. This parser extracts relevant information from the Debezium-JSON format and + * The {@code DebeziumRecordParser} class extends the abstract {@link JsonRecordParser} and is + * designed to parse records from Debezium's JSON change data capture (CDC) format. Debezium is a + * CDC solution for MySQL databases that captures row-level changes to database tables and outputs + * them in JSON format. This parser extracts relevant information from the Debezium-JSON format and * converts it into a list of {@link RichCdcMultiplexRecord} objects. * *

The class supports various database operations such as INSERT, UPDATE, DELETE, and READ @@ -63,20 +75,7 @@ * operation type, and primary key field names are used to construct the details of each record * event. */ -public class DebeziumJsonRecordParser extends RecordParser { - - private static final String FIELD_SCHEMA = "schema"; - protected static final String FIELD_PAYLOAD = "payload"; - private static final String FIELD_BEFORE = "before"; - private static final String FIELD_AFTER = "after"; - private static final String FIELD_SOURCE = "source"; - private static final String FIELD_PRIMARY = "pkNames"; - private static final String FIELD_DB = "db"; - private static final String FIELD_TYPE = "op"; - private static final String OP_INSERT = "c"; - private static final String OP_UPDATE = "u"; - private static final String OP_DELETE = "d"; - private static final String OP_READE = "r"; +public class DebeziumJsonRecordParser extends JsonRecordParser { private boolean hasSchema; private final Map debeziumTypes = new HashMap<>(); @@ -147,10 +146,10 @@ private void parseSchema(JsonNode schema) { ArrayNode fields = null; for (int i = 0; i < schemaFields.size(); i++) { JsonNode node = schemaFields.get(i); - if (getString(node, "field").equals("after")) { + if ("after".equals(getString(node, "field"))) { fields = getNodeAs(node, "fields", ArrayNode.class); break; - } else if (getString(node, "field").equals("before")) { + } else if ("before".equals(getString(node, "field"))) { if (fields == null) { fields = getNodeAs(node, "fields", ArrayNode.class); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java index 1aab6653d4d4..97a0ba43462d 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java @@ -36,17 +36,23 @@ import io.debezium.time.MicroTimestamp; import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; +import org.apache.avro.Schema; import org.apache.kafka.connect.json.JsonConverterConfig; import javax.annotation.Nullable; +import java.io.IOException; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZoneOffset; import java.util.Base64; import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Supplier; import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING; @@ -56,7 +62,23 @@ */ public class DebeziumSchemaUtils { - /** Transform raw string value according to schema. */ + public static final String FIELD_SCHEMA = "schema"; + public static final String FIELD_PAYLOAD = "payload"; + public static final String FIELD_SOURCE = "source"; + public static final String FIELD_PRIMARY = "pkNames"; + public static final String FIELD_DB = "db"; + + public static final String FIELD_BEFORE = "before"; + public static final String FIELD_AFTER = "after"; + + public static final String FIELD_TYPE = "op"; + public static final String OP_READE = "r"; + public static final String OP_INSERT = "c"; + public static final String OP_UPDATE = "u"; + public static final String OP_DELETE = "d"; + public static final String OP_TRUNCATE = "t"; + public static final String OP_MESSAGE = "m"; + public static String transformRawValue( @Nullable String rawValue, String debeziumType, @@ -64,6 +86,31 @@ public static String transformRawValue( TypeMapping typeMapping, JsonNode origin, ZoneId serverTimeZone) { + return transformRawValue( + rawValue, + debeziumType, + className, + typeMapping, + () -> { + try { + return ByteBuffer.wrap(origin.get(Geometry.WKB_FIELD).binaryValue()); + } catch (IOException e) { + throw new IllegalArgumentException( + String.format("Failed to convert %s to geometry JSON.", rawValue), + e); + } + }, + serverTimeZone); + } + + /** Transform raw string value according to schema. */ + public static String transformRawValue( + @Nullable String rawValue, + String debeziumType, + @Nullable String className, + TypeMapping typeMapping, + Supplier geometryGetter, + ZoneId serverTimeZone) { if (rawValue == null) { return null; } @@ -158,8 +205,7 @@ else if (Date.SCHEMA_NAME.equals(className)) { } else if (Point.LOGICAL_NAME.equals(className) || Geometry.LOGICAL_NAME.equals(className)) { try { - byte[] wkb = origin.get(Geometry.WKB_FIELD).binaryValue(); - transformed = MySqlTypeUtils.convertWkbArray(wkb); + transformed = MySqlTypeUtils.convertWkbArray(geometryGetter.get()); } catch (Exception e) { throw new IllegalArgumentException( String.format("Failed to convert %s to geometry JSON.", rawValue), e); @@ -251,4 +297,77 @@ private static DataType fromDebeziumType(String dbzType) { public static String decimalLogicalName() { return "org.apache.#.connect.data.Decimal".replace("#", "kafka"); } + + protected static final String CONNECT_PARAMETERS_PROP = "connect.parameters"; + protected static final String CONNECT_NAME_PROP = "connect.name"; + + private static final String POINT_LOGICAL_NAME = "io.debezium.data.geometry.Point"; + private static final String GEOMETRY_LOGICAL_NAME = "io.debezium.data.geometry.Geometry"; + private static final String ENUM_SET_LOGICAL_NAME = "io.debezium.data.EnumSet"; + private static final String DATE_SCHEMA_NAME = "io.debezium.time.Date"; + private static final String TIMESTAMP_SCHEMA_NAME = "io.debezium.time.Timestamp"; + private static final String MICRO_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.MicroTimestamp"; + private static final String NANO_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.NanoTimestamp"; + private static final String TIME_SCHEMA_NAME = "io.debezium.time.Time"; + private static final String MICRO_TIME_SCHEMA_NAME = "io.debezium.time.MicroTime"; + private static final String NANO_TIME_SCHEMA_NAME = "io.debezium.time.NanoTime"; + private static final String ZONED_TIME_SCHEMA_NAME = "io.debezium.time.ZonedTime"; + private static final String ZONED_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.ZonedTimestamp"; + private static final String DECIMAL_PRECISE_SCHEMA_NAME = + "org.apache.kafka.connect.data.Decimal"; + private static final String SCHEMA_PARAMETER_COLUMN_TYPE = "__debezium.source.column.type"; + private static final String SCHEMA_PARAMETER_COLUMN_SIZE = "__debezium.source.column.length"; + private static final String SCHEMA_PARAMETER_COLUMN_PRECISION = + "__debezium.source.column.scale"; + private static final String SCHEMA_PARAMETER_COLUMN_NAME = "__debezium.source.column.name"; + + public static DataType avroToPaimonDataType(Schema schema) { + // Mapping by mysql types + // Parse actual source column type from connect.parameters if enable debezium property + // "column.propagate.source.type", otherwise will infer avro schema type mapping to paimon + Map connectParameters = + (Map) schema.getObjectProp(CONNECT_PARAMETERS_PROP); + if (Objects.nonNull(connectParameters)) { + String typeName = + connectParameters.getOrDefault( + SCHEMA_PARAMETER_COLUMN_TYPE, schema.getType().name()); + Integer length = + Optional.ofNullable(connectParameters.get(SCHEMA_PARAMETER_COLUMN_SIZE)) + .map(Integer::valueOf) + .orElse(null); + Integer scale = + Optional.ofNullable(connectParameters.get(SCHEMA_PARAMETER_COLUMN_PRECISION)) + .map(Integer::valueOf) + .orElse(null); + return MySqlTypeUtils.toDataType(typeName, length, scale, TypeMapping.defaultMapping()); + } + + // Mapping by avro schema type + return fromDebeziumAvroType(schema); + } + + private static DataType fromDebeziumAvroType(Schema schema) { + Schema.Type avroType = schema.getType(); + switch (avroType) { + case BOOLEAN: + return DataTypes.BOOLEAN(); + case BYTES: + case FIXED: + return DataTypes.BYTES(); + case DOUBLE: + return DataTypes.DOUBLE(); + case FLOAT: + return DataTypes.FLOAT(); + case INT: + return DataTypes.INT(); + case LONG: + return DataTypes.BIGINT(); + case STRING: + case RECORD: + return DataTypes.STRING(); + default: + throw new UnsupportedOperationException( + String.format("Don't support avro type '%s' yet.", avroType)); + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/maxwell/MaxwellRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/maxwell/MaxwellRecordParser.java index 7379f81a55ee..9b3c6b70651c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/maxwell/MaxwellRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/maxwell/MaxwellRecordParser.java @@ -20,7 +20,7 @@ import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.RecordParser; +import org.apache.paimon.flink.action.cdc.format.JsonRecordParser; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.RowKind; @@ -30,11 +30,11 @@ import java.util.List; /** - * The {@code MaxwellRecordParser} class extends the abstract {@link RecordParser} and is designed - * to parse records from Maxwell's JSON change data capture (CDC) format. Maxwell is a CDC solution - * for MySQL databases that captures row-level changes to database tables and outputs them in JSON - * format. This parser extracts relevant information from the Maxwell-JSON format and converts it - * into a list of {@link RichCdcMultiplexRecord} objects. + * The {@code MaxwellRecordParser} class extends the abstract {@link JsonRecordParser} and is + * designed to parse records from Maxwell's JSON change data capture (CDC) format. Maxwell is a CDC + * solution for MySQL databases that captures row-level changes to database tables and outputs them + * in JSON format. This parser extracts relevant information from the Maxwell-JSON format and + * converts it into a list of {@link RichCdcMultiplexRecord} objects. * *

The class supports various database operations such as INSERT, UPDATE, and DELETE, and creates * corresponding {@link RichCdcMultiplexRecord} objects to represent these changes. @@ -42,7 +42,7 @@ *

Validation is performed to ensure that the JSON records contain all necessary fields, and the * class also supports schema extraction for the Kafka topic. */ -public class MaxwellRecordParser extends RecordParser { +public class MaxwellRecordParser extends JsonRecordParser { private static final String FIELD_OLD = "old"; private static final String FIELD_TYPE = "type"; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/ogg/OggRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/ogg/OggRecordParser.java index 019ae1562db9..988a82f98290 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/ogg/OggRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/ogg/OggRecordParser.java @@ -21,7 +21,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.RecordParser; +import org.apache.paimon.flink.action.cdc.format.JsonRecordParser; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.RowKind; @@ -35,11 +35,11 @@ import static org.apache.paimon.utils.JsonSerdeUtil.isNull; /** - * The {@code OggRecordParser} class extends the abstract {@link RecordParser} and is responsible - * for parsing records from the Oracle GoldenGate (OGG) JSON format. Oracle GoldenGate is a software - * application used for real-time data integration and replication in heterogeneous IT environments. - * This parser extracts relevant information from the OGG JSON records and transforms them into a - * list of {@link RichCdcMultiplexRecord} objects. + * The {@code OggRecordParser} class extends the abstract {@link JsonRecordParser} and is + * responsible for parsing records from the Oracle GoldenGate (OGG) JSON format. Oracle GoldenGate + * is a software application used for real-time data integration and replication in heterogeneous IT + * environments. This parser extracts relevant information from the OGG JSON records and transforms + * them into a list of {@link RichCdcMultiplexRecord} objects. * *

The class handles three types of database operations, represented by "U" for UPDATE, "I" for * INSERT, and "D" for DELETE. It then generates corresponding {@link RichCdcMultiplexRecord} @@ -50,7 +50,7 @@ * providing a way to understand the structure of the incoming records and their corresponding field * types. */ -public class OggRecordParser extends RecordParser { +public class OggRecordParser extends JsonRecordParser { private static final String FIELD_BEFORE = "before"; private static final String FIELD_TYPE = "op_type"; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java index 4f0be0ef221e..e1c1acfdd820 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java @@ -21,13 +21,14 @@ import org.apache.paimon.flink.action.cdc.CdcSourceRecord; import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.format.DataFormat; -import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; import org.apache.paimon.utils.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; @@ -42,7 +43,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import java.io.IOException; import java.time.Duration; import java.util.Arrays; import java.util.Collection; @@ -70,7 +70,9 @@ public class KafkaActionUtils { private static final String PARTITION = "partition"; private static final String OFFSET = "offset"; - public static KafkaSource buildKafkaSource(Configuration kafkaConfig) { + public static KafkaSource buildKafkaSource( + Configuration kafkaConfig, + KafkaDeserializationSchema deserializationSchema) { KafkaSourceBuilder kafkaSourceBuilder = KafkaSource.builder(); if (kafkaConfig.contains(KafkaConnectorOptions.TOPIC)) { @@ -85,7 +87,7 @@ public static KafkaSource buildKafkaSource(Configuration kafkaC } kafkaSourceBuilder - .setValueOnlyDeserializer(new CdcJsonDeserializationSchema()) + .setDeserializer(KafkaRecordDeserializationSchema.of(deserializationSchema)) .setGroupId(kafkaPropertiesGroupId(kafkaConfig)); Properties properties = createKafkaProperties(kafkaConfig); @@ -244,7 +246,8 @@ public static DataFormat getDataFormat(Configuration kafkaConfig) { } public static MessageQueueSchemaUtils.ConsumerWrapper getKafkaEarliestConsumer( - Configuration kafkaConfig) { + Configuration kafkaConfig, + KafkaDeserializationSchema deserializationSchema) { Properties props = createKafkaProperties(kafkaConfig); props.put( @@ -263,12 +266,7 @@ public static MessageQueueSchemaUtils.ConsumerWrapper getKafkaEarliestConsumer( KafkaConsumer consumer = new KafkaConsumer<>(props); - String topic; - if (kafkaConfig.contains(KafkaConnectorOptions.TOPIC)) { - topic = kafkaConfig.get(KafkaConnectorOptions.TOPIC).get(0); - } else { - topic = findOneTopic(props, kafkaConfig.get(KafkaConnectorOptions.TOPIC_PATTERN)); - } + String topic = findOneTopic(kafkaConfig, props); // the return may be null in older versions of the Kafka client List partitionInfos = consumer.partitionsFor(topic); @@ -286,7 +284,7 @@ public static MessageQueueSchemaUtils.ConsumerWrapper getKafkaEarliestConsumer( consumer.assign(topicPartitions); consumer.seekToBeginning(topicPartitions); - return new KafkaConsumerWrapper(consumer, topic); + return new KafkaConsumerWrapper(consumer, topic, deserializationSchema); } private static Properties createKafkaProperties(Configuration kafkaConfig) { @@ -295,44 +293,56 @@ private static Properties createKafkaProperties(Configuration kafkaConfig) { return props; } - private static String findOneTopic(Properties properties, String pattern) { - Pattern topicPattern = Pattern.compile(pattern); - try (AdminClient adminClient = AdminClient.create(properties)) { - Set allTopicNames = adminClient.listTopics().names().get(); - for (String topicName : allTopicNames) { - if (topicPattern.matcher(topicName).matches()) { - return topicName; + public static String findOneTopic(Configuration kafkaConfig) { + return findOneTopic(kafkaConfig, createKafkaProperties(kafkaConfig)); + } + + private static String findOneTopic(Configuration kafkaConfig, Properties properties) { + if (kafkaConfig.contains(KafkaConnectorOptions.TOPIC)) { + return kafkaConfig.get(KafkaConnectorOptions.TOPIC).get(0); + } else { + String pattern = kafkaConfig.get(KafkaConnectorOptions.TOPIC_PATTERN); + Pattern topicPattern = Pattern.compile(pattern); + try (AdminClient adminClient = AdminClient.create(properties)) { + Set allTopicNames = adminClient.listTopics().names().get(); + for (String topicName : allTopicNames) { + if (topicPattern.matcher(topicName).matches()) { + return topicName; + } } + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); } - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); - } - throw new RuntimeException("Cannot find topics match the topic-pattern " + pattern); + throw new RuntimeException("Cannot find topics match the topic-pattern " + pattern); + } } private static class KafkaConsumerWrapper implements MessageQueueSchemaUtils.ConsumerWrapper { private final KafkaConsumer consumer; private final String topic; + private final KafkaDeserializationSchema deserializationSchema; - KafkaConsumerWrapper(KafkaConsumer kafkaConsumer, String topic) { + KafkaConsumerWrapper( + KafkaConsumer kafkaConsumer, + String topic, + KafkaDeserializationSchema deserializationSchema) { this.consumer = kafkaConsumer; this.topic = topic; + this.deserializationSchema = deserializationSchema; } @Override public List getRecords(int pollTimeOutMills) { ConsumerRecords consumerRecords = consumer.poll(Duration.ofMillis(pollTimeOutMills)); - CdcJsonDeserializationSchema deserializationSchema = new CdcJsonDeserializationSchema(); return StreamSupport.stream(consumerRecords.records(topic).spliterator(), false) .map( consumerRecord -> { try { - return deserializationSchema.deserialize( - consumerRecord.value()); - } catch (IOException e) { + return deserializationSchema.deserialize(consumerRecord); + } catch (Exception e) { throw new RuntimeException(e); } }) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java new file mode 100644 index 000000000000..0cbca3ffd7ff --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -0,0 +1,87 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.flink.action.cdc.serialization.ConfluentAvroDeserializationSchema; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; +import static org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils.SCHEMA_REGISTRY_URL; + +/** A simple deserialization schema for {@link CdcSourceRecord}. */ +public class KafkaDebeziumAvroDeserializationSchema + implements KafkaDeserializationSchema { + + private static final long serialVersionUID = 1L; + + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + + private final String topic; + private final String schemaRegistryUrl; + + /** The deserializer to deserialize Debezium Avro data. */ + private ConfluentAvroDeserializationSchema avroDeserializer; + + public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { + this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); + this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + } + + @Override + public void open(DeserializationSchema.InitializationContext context) throws Exception { + avroDeserializer = + new ConfluentAvroDeserializationSchema( + new CachedSchemaRegistryClient( + schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } + + @Override + public CdcSourceRecord deserialize(ConsumerRecord message) throws IOException { + if (message.value() == null) { + // skip tombstone messages + return null; + } + + GenericRecord key = + (GenericRecord) this.avroDeserializer.deserialize(topic, true, message.key()); + GenericRecord value = + (GenericRecord) this.avroDeserializer.deserialize(topic, false, message.value()); + return new CdcSourceRecord(topic, key, value); + } + + @Override + public boolean isEndOfStream(CdcSourceRecord nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return getForClass(CdcSourceRecord.class); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumJsonDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumJsonDeserializationSchema.java new file mode 100644 index 000000000000..76211cf56dad --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumJsonDeserializationSchema.java @@ -0,0 +1,83 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; + +/** A simple deserialization schema for {@link CdcSourceRecord}. */ +public class KafkaDebeziumJsonDeserializationSchema + implements KafkaDeserializationSchema { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = + LoggerFactory.getLogger(KafkaDebeziumJsonDeserializationSchema.class); + + private final ObjectMapper objectMapper = new ObjectMapper(); + + public KafkaDebeziumJsonDeserializationSchema(Configuration cdcSourceConfig) { + this(); + } + + public KafkaDebeziumJsonDeserializationSchema() { + objectMapper + .configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + } + + @Override + public CdcSourceRecord deserialize(ConsumerRecord message) throws IOException { + if (message.value() == null) { + // skip tombstone messages + return null; + } + + try { + return new CdcSourceRecord(objectMapper.readValue(message.value(), JsonNode.class)); + } catch (Exception e) { + LOG.error("Invalid Json:\n{}", new String(message.value())); + throw e; + } + } + + @Override + public boolean isEndOfStream(CdcSourceRecord nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return getForClass(CdcSourceRecord.class); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlTypeUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlTypeUtils.java index 5b5efa1cf846..1af47667d67e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlTypeUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlTypeUtils.java @@ -329,7 +329,11 @@ public static boolean isGeoType(String type) { } public static String convertWkbArray(byte[] wkb) throws JsonProcessingException { - String geoJson = OGCGeometry.fromBinary(ByteBuffer.wrap(wkb)).asGeoJson(); + return convertWkbArray(ByteBuffer.wrap(wkb)); + } + + public static String convertWkbArray(ByteBuffer wkbByteBuffer) throws JsonProcessingException { + String geoJson = OGCGeometry.fromBinary(wkbByteBuffer).asGeoJson(); JsonNode originGeoNode = objectMapper.readTree(geoJson); Optional srid = diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionUtils.java index 66bdc1847c57..7e2398ef7300 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionUtils.java @@ -21,8 +21,8 @@ import org.apache.paimon.flink.action.cdc.CdcSourceRecord; import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.format.DataFormat; -import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; @@ -57,6 +57,7 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import java.util.regex.Pattern; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; @@ -164,7 +165,9 @@ public class PulsarActionUtils { .defaultValue(true) .withDescription("To specify the boundedness of a stream."); - public static PulsarSource buildPulsarSource(Configuration pulsarConfig) { + public static PulsarSource buildPulsarSource( + Configuration pulsarConfig, + DeserializationSchema deserializationSchema) { PulsarSourceBuilder pulsarSourceBuilder = PulsarSource.builder(); // the minimum setup @@ -172,7 +175,7 @@ public static PulsarSource buildPulsarSource(Configuration puls .setServiceUrl(pulsarConfig.get(PULSAR_SERVICE_URL)) .setAdminUrl(pulsarConfig.get(PULSAR_ADMIN_URL)) .setSubscriptionName(pulsarConfig.get(PULSAR_SUBSCRIPTION_NAME)) - .setDeserializationSchema(new CdcJsonDeserializationSchema()); + .setDeserializationSchema(deserializationSchema); pulsarConfig.getOptional(TOPIC).ifPresent(pulsarSourceBuilder::setTopics); pulsarConfig.getOptional(TOPIC_PATTERN).ifPresent(pulsarSourceBuilder::setTopicPattern); @@ -293,7 +296,8 @@ public static DataFormat getDataFormat(Configuration pulsarConfig) { /** Referenced to {@link PulsarPartitionSplitReader#createPulsarConsumer}. */ public static MessageQueueSchemaUtils.ConsumerWrapper createPulsarConsumer( - Configuration pulsarConfig) { + Configuration pulsarConfig, + DeserializationSchema deserializationSchema) { try { SourceConfiguration pulsarSourceConfiguration = new SourceConfiguration(pulsarConfig); PulsarClient pulsarClient = PulsarClientFactory.createClient(pulsarSourceConfiguration); @@ -307,10 +311,8 @@ public static MessageQueueSchemaUtils.ConsumerWrapper createPulsarConsumer( // The default position is Latest consumerBuilder.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest); - String topic = - pulsarConfig.contains(TOPIC) - ? pulsarConfig.get(TOPIC).get(0) - : findOneTopic(pulsarClient, pulsarConfig.get(TOPIC_PATTERN)); + String topic = findOneTopic(pulsarConfig, () -> pulsarClient); + TopicPartition topicPartition = new TopicPartition(topic); consumerBuilder.topic(topicPartition.getFullTopicName()); @@ -329,47 +331,68 @@ public static MessageQueueSchemaUtils.ConsumerWrapper createPulsarConsumer( // Create the consumer configuration by using common utils. Consumer consumer = consumerBuilder.subscribe(); - return new PulsarConsumerWrapper(consumer, topic); + return new PulsarConsumerWrapper(consumer, topic, deserializationSchema); } catch (PulsarClientException e) { throw new RuntimeException(e); } } + public static String findOneTopic(Configuration pulsarConfig) { + return findOneTopic( + pulsarConfig, + () -> { + try { + return PulsarClientFactory.createClient( + new SourceConfiguration(pulsarConfig)); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }); + } + /** Referenced to {@link TopicPatternSubscriber}. */ - private static String findOneTopic(PulsarClient pulsarClient, String topicPattern) { - TopicName destination = TopicName.get(topicPattern); - String pattern = destination.toString(); + private static String findOneTopic( + Configuration pulsarConfig, Supplier pulsarClientSupplier) { + if (pulsarConfig.contains(TOPIC)) { + return pulsarConfig.get(TOPIC).get(0); + } else { + String topicPattern = pulsarConfig.get(TOPIC_PATTERN); + TopicName destination = TopicName.get(topicPattern); + String pattern = destination.toString(); - Pattern shortenedPattern = Pattern.compile(pattern.split("://")[1]); - String namespace = destination.getNamespaceObject().toString(); + Pattern shortenedPattern = Pattern.compile(pattern.split("://")[1]); + String namespace = destination.getNamespaceObject().toString(); - LookupService lookupService = ((PulsarClientImpl) pulsarClient).getLookup(); - NamespaceName namespaceName = NamespaceName.get(namespace); - try { - // Pulsar 2.11.0 can filter regular expression on broker, but it has a bug which - // can only be used for wildcard filtering. - String queryPattern = shortenedPattern.toString(); - if (!queryPattern.endsWith(".*")) { - queryPattern = null; - } - - GetTopicsResult topicsResult = - lookupService - .getTopicsUnderNamespace( - namespaceName, - CommandGetTopicsOfNamespace.Mode.ALL, - queryPattern, - null) - .get(); - List topics = topicsResult.getTopics(); - - if (topics == null || topics.isEmpty()) { - throw new RuntimeException("Cannot find topics match the topic-pattern " + pattern); + LookupService lookupService = + ((PulsarClientImpl) pulsarClientSupplier.get()).getLookup(); + NamespaceName namespaceName = NamespaceName.get(namespace); + try { + // Pulsar 2.11.0 can filter regular expression on broker, but it has a bug which + // can only be used for wildcard filtering. + String queryPattern = shortenedPattern.toString(); + if (!queryPattern.endsWith(".*")) { + queryPattern = null; + } + + GetTopicsResult topicsResult = + lookupService + .getTopicsUnderNamespace( + namespaceName, + CommandGetTopicsOfNamespace.Mode.ALL, + queryPattern, + null) + .get(); + List topics = topicsResult.getTopics(); + + if (topics == null || topics.isEmpty()) { + throw new RuntimeException( + "Cannot find topics match the topic-pattern " + pattern); + } + + return topics.get(0); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); } - - return topics.get(0); - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); } } @@ -377,18 +400,21 @@ private static class PulsarConsumerWrapper implements MessageQueueSchemaUtils.Co private final Consumer consumer; private final String topic; + private final DeserializationSchema deserializationSchema; - PulsarConsumerWrapper(Consumer consumer, String topic) { + PulsarConsumerWrapper( + Consumer consumer, + String topic, + DeserializationSchema deserializationSchema) { this.consumer = consumer; this.topic = topic; + this.deserializationSchema = deserializationSchema; } @Override public List getRecords(int pollTimeOutMills) { try { Message message = consumer.receive(pollTimeOutMills, TimeUnit.MILLISECONDS); - CdcJsonDeserializationSchema deserializationSchema = - new CdcJsonDeserializationSchema(); return message == null ? Collections.emptyList() : Collections.singletonList( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java new file mode 100644 index 000000000000..042c7574d87a --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java @@ -0,0 +1,82 @@ +/* + * 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.paimon.flink.action.cdc.pulsar; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.flink.action.cdc.serialization.ConfluentAvroDeserializationSchema; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; +import static org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils.SCHEMA_REGISTRY_URL; + +/** A simple deserialization schema for {@link CdcSourceRecord}. */ +public class PulsarDebeziumAvroDeserializationSchema + implements DeserializationSchema { + + private static final long serialVersionUID = 1L; + + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + + private final String topic; + private final String schemaRegistryUrl; + + /** The deserializer to deserialize Debezium Avro data. */ + private ConfluentAvroDeserializationSchema avroDeserializer; + + public PulsarDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { + this.topic = PulsarActionUtils.findOneTopic(cdcSourceConfig); + this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + } + + @Override + public void open(InitializationContext context) throws Exception { + avroDeserializer = + new ConfluentAvroDeserializationSchema( + new CachedSchemaRegistryClient( + schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } + + @Override + public CdcSourceRecord deserialize(byte[] message) throws IOException { + if (message == null) { + return null; + } + + GenericRecord value = + (GenericRecord) this.avroDeserializer.deserialize(topic, false, message); + return new CdcSourceRecord(topic, null, value); + } + + @Override + public boolean isEndOfStream(CdcSourceRecord nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return getForClass(CdcSourceRecord.class); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java deleted file mode 100644 index 19146cd8d466..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcDebeziumAvroDeserializationSchema.java +++ /dev/null @@ -1,106 +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.paimon.flink.action.cdc.serialization; - -import org.apache.paimon.flink.action.cdc.CdcSourceRecord; -import org.apache.paimon.format.avro.AvroSchemaConverter; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Parser; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroDeserializationSchema; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.Map; - -import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; - -/** - * A simple deserialization schema for {@link CdcSourceRecord}. reference from {@link - * DebeziumAvroDeserializationSchema}. - */ -public class CdcDebeziumAvroDeserializationSchema - implements DeserializationSchema { - - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(CdcJsonDeserializationSchema.class); - - /** The deserializer to deserialize Debezium Avro data. */ - private final DeserializationSchema avroDeserializer; - - public CdcDebeziumAvroDeserializationSchema( - RowType rowType, - String schemaRegistryUrl, - @Nullable String schemaString, - @Nullable Map registryConfigs) { - Schema schema = - schemaString == null - ? AvroSchemaConverter.convertToSchema(rowType) - : new Parser().parse(schemaString); - avroDeserializer = - ConfluentRegistryAvroDeserializationSchema.forGeneric( - schema, schemaRegistryUrl, registryConfigs); - } - - @Override - public void open(InitializationContext context) throws Exception { - avroDeserializer.open(context); - } - - @Override - public CdcSourceRecord deserialize(byte[] message) throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return null; - } - - GenericRecord record = this.avroDeserializer.deserialize(message); - return new CdcSourceRecord(record); - } - - @Override - public boolean isEndOfStream(CdcSourceRecord nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return getForClass(CdcSourceRecord.class); - } - - public static RowType createDebeziumAvroRowType(DataType databaseSchema) { - // Debezium Avro contains other information, e.g. "source", "ts_ms" - // but we don't need them - return DataTypes.ROW( - DataTypes.FIELD(0, "before", databaseSchema.nullable()), - DataTypes.FIELD(1, "after", databaseSchema.nullable()), - DataTypes.FIELD(2, "op", DataTypes.STRING())); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcJsonDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcJsonDeserializationSchema.java index 07461cd49257..be39ef695d78 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcJsonDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/serialization/CdcJsonDeserializationSchema.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,6 +43,10 @@ public class CdcJsonDeserializationSchema implements DeserializationSchema fields = new ArrayList<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 5bb63b6a892b..3d66a1da8464 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -485,7 +485,8 @@ public void testKafkaBuildSchemaWithDelete(String format) throws Exception { Schema kafkaSchema = MessageQueueSchemaUtils.getSchema( - getKafkaEarliestConsumer(kafkaConfig), + getKafkaEarliestConsumer( + kafkaConfig, new KafkaDebeziumJsonDeserializationSchema()), getDataFormat(kafkaConfig), TypeMapping.defaultMapping()); List fields = new ArrayList<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java index 33d854fb8eb0..255d43bfbc81 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; @@ -56,7 +57,7 @@ public void testPulsarSchema() throws Exception { Schema pulsarSchema = MessageQueueSchemaUtils.getSchema( - createPulsarConsumer(pulsarConfig), + createPulsarConsumer(pulsarConfig, new CdcJsonDeserializationSchema()), getDataFormat(pulsarConfig), TypeMapping.defaultMapping()); List fields = new ArrayList<>(); From 1d91ccefa4096837ed09c1104eb4b677de3b9927 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Mon, 13 May 2024 09:23:49 +0800 Subject: [PATCH 03/12] fix ut --- .../flink/action/cdc/format/DataFormat.java | 2 + .../KafkaCanalSyncDatabaseActionITCase.java | 77 ++++++++----- .../KafkaCanalSyncTableActionITCase.java | 108 +++++++++--------- ...KafkaDebeziumSyncDatabaseActionITCase.java | 18 +-- .../KafkaDebeziumSyncTableActionITCase.java | 34 +++--- .../KafkaMaxwellSyncDatabaseActionITCase.java | 18 +-- .../KafkaMaxwellSyncTableActionITCase.java | 24 ++-- .../KafkaOggSyncDatabaseActionITCase.java | 20 ++-- .../kafka/KafkaOggSyncTableActionITCase.java | 24 ++-- .../action/cdc/kafka/KafkaSchemaITCase.java | 20 ++-- .../cdc/kafka/KafkaSyncTableActionITCase.java | 37 +++--- .../cdc/pulsar/PulsarActionITCaseBase.java | 8 +- .../action/cdc/pulsar/PulsarSchemaITCase.java | 6 +- .../PulsarSyncDatabaseActionITCase.java | 25 ++-- .../pulsar/PulsarSyncTableActionITCase.java | 22 ++-- .../case-insensitive/canal-json-data-1.txt} | 0 .../include/topic0/canal-json-data-1.txt} | 0 .../incomplete/canal-json-data-1.txt} | 0 .../topic0/canal-json-data-1.txt} | 0 .../topic0/canal-json-data-2.txt} | 0 .../topic0/canal-json-data-3.txt} | 0 .../topic1/canal-json-data-1.txt} | 0 .../topic1/canal-json-data-2.txt} | 0 .../topic1/canal-json-data-3.txt} | 0 .../topic0/canal-json-data-1.txt} | 0 .../topic0/canal-json-data-2.txt} | 0 .../topic0/canal-json-data-3.txt} | 0 .../topic1/canal-json-data-1.txt} | 0 .../topic1/canal-json-data-2.txt} | 0 .../topic1/canal-json-data-3.txt} | 0 .../topic2/canal-json-data-1.txt} | 0 .../topic2/canal-json-data-2.txt} | 0 .../topic2/canal-json-data-3.txt} | 0 .../database/tostring/canal-json-data-1.txt} | 0 .../table/alltype/canal-json-data.txt} | 0 .../computedcolumn/canal-json-data-1.txt} | 0 .../computedcolumn/canal-json-data-2.txt} | 0 .../table/event/event-delete.txt | 0 .../table/event/event-insert.txt | 0 .../table/event/event-row.txt | 0 .../table/event/event-update.txt | 0 .../table/incomplete/canal-json-data-1.txt} | 0 .../table/incomplete/canal-json-data-2.txt} | 0 .../initialemptytopic/canal-json-data-1.txt} | 0 .../table/nononddldata/canal-json-data-1.txt} | 0 .../table/nonpk/canal-json-data-1.txt} | 0 .../optionschange/canal-json-data-1.txt} | 0 .../optionschange/canal-json-data-2.txt} | 0 .../schemaevolution/canal-json-data-1.txt} | 0 .../schemaevolution/canal-json-data-2.txt} | 0 .../schemaevolution/canal-json-data-3.txt} | 0 .../schemaevolution/canal-json-data-4.txt} | 0 .../schemaevolution/canal-json-data-5.txt} | 0 .../canal-json-data-1.txt} | 0 .../canal-json-data-2.txt} | 0 .../canal-json-data-3.txt} | 0 .../canal-json-data-4.txt} | 0 .../canal-json-data-5.txt} | 0 .../canal-json-data-1.txt} | 0 .../canal-json-data-2.txt} | 0 .../table/startupmode/canal-json-data-1.txt} | 0 .../table/startupmode/canal-json-data-2.txt} | 0 .../table/tostring/canal-json-data-1.txt} | 0 .../table/watermark/canal-json-data-1.txt} | 0 .../include/topic0/debezium-json-data-1.txt} | 0 .../topic0/debezium-json-data-1.txt} | 0 .../topic0/debezium-json-data-2.txt} | 0 .../topic1/debezium-json-data-1.txt} | 0 .../topic1/debezium-json-data-2.txt} | 0 .../topic0/debezium-json-data-1.txt} | 0 .../topic0/debezium-json-data-2.txt} | 0 .../topic1/debezium-json-data-1.txt} | 0 .../topic1/debezium-json-data-2.txt} | 0 .../computedcolumn/debezium-json-data-1.txt} | 0 .../nestedtype/debezium-json-data-1.txt} | 0 .../table/nullvalue/debezium-json-data-1.txt} | 0 .../table/nullvalue/debezium-json-data-2.txt} | 0 .../schema/alltype/debezium-json-data-1.txt} | 0 .../computedcolumn/debezium-json-data-1.txt} | 0 .../schema/include/debezium-json-data-1.txt} | 0 .../schemaevolution/debezium-json-data-1.txt} | 0 .../schemaevolution/debezium-json-data-2.txt} | 0 .../schemaevolution/debezium-json-data-3.txt} | 0 .../schemaevolution/debezium-json-data-4.txt} | 0 .../schemaevolution/debezium-json-data-1.txt} | 0 .../schemaevolution/debezium-json-data-2.txt} | 0 .../schemaevolution/debezium-json-data-3.txt} | 0 .../startupmode/debezium-json-data-1.txt} | 0 .../startupmode/debezium-json-data-2.txt} | 0 .../table/watermark/debezium-json-data-1.txt} | 0 .../include/topic0/maxwell-json-data-1.txt} | 0 .../topic0/maxwell-json-data-1.txt} | 0 .../topic0/maxwell-json-data-2.txt} | 0 .../topic1/maxwell-json-data-1.txt} | 0 .../topic1/maxwell-json-data-2.txt} | 0 .../topic0/maxwell-json-data-1.txt} | 0 .../topic0/maxwell-json-data-2.txt} | 0 .../topic1/maxwell-json-data-1.txt} | 0 .../topic1/maxwell-json-data-2.txt} | 0 .../computedcolumn/maxwell-json-data-1.txt} | 0 .../schemaevolution/maxwell-json-data-1.txt} | 0 .../schemaevolution/maxwell-json-data-2.txt} | 0 .../schemaevolution/maxwell-json-data-3.txt} | 0 .../schemaevolution/maxwell-json-data-4.txt} | 0 .../startupmode/maxwell-json-data-1.txt} | 0 .../startupmode/maxwell-json-data-2.txt} | 0 .../table/watermark/maxwell-json-data-1.txt} | 0 .../case-insensitive/ogg-json-data-1.txt} | 0 .../include/topic0/ogg-json-data-1.txt} | 0 .../prefixsuffix/topic0/ogg-json-data-1.txt} | 0 .../prefixsuffix/topic0/ogg-json-data-2.txt} | 0 .../prefixsuffix/topic1/ogg-json-data-1.txt} | 0 .../prefixsuffix/topic1/ogg-json-data-2.txt} | 0 .../topic0/ogg-json-data-1.txt} | 0 .../topic0/ogg-json-data-2.txt} | 0 .../topic1/ogg-json-data-1.txt} | 0 .../topic1/ogg-json-data-2.txt} | 0 .../table/computedcolumn/ogg-json-data-1.txt} | 0 .../table/event/event-delete.txt | 0 .../table/event/event-insert.txt | 0 .../table/event/event-update.txt | 0 .../schemaevolution/ogg-json-data-1.txt} | 0 .../schemaevolution/ogg-json-data-2.txt} | 0 .../schemaevolution/ogg-json-data-3.txt} | 0 .../table/startupmode/ogg-json-data-1.txt} | 0 .../table/startupmode/ogg-json-data-2.txt} | 0 .../table/watermark/ogg-json-data-1.txt} | 0 127 files changed, 240 insertions(+), 203 deletions(-) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/case-insensitive/canal-data-1.txt => canal-json/database/case-insensitive/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/include/topic0/canal-data-1.txt => canal-json/database/include/topic0/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/incomplete/canal-data-1.txt => canal-json/database/incomplete/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic0/canal-data-1.txt => canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic0/canal-data-2.txt => canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic0/canal-data-3.txt => canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic1/canal-data-1.txt => canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic1/canal-data-2.txt => canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/prefixsuffix/topic1/canal-data-3.txt => canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic0/canal-data-1.txt => canal-json/database/schemaevolution/topic0/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic0/canal-data-2.txt => canal-json/database/schemaevolution/topic0/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic0/canal-data-3.txt => canal-json/database/schemaevolution/topic0/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic1/canal-data-1.txt => canal-json/database/schemaevolution/topic1/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic1/canal-data-2.txt => canal-json/database/schemaevolution/topic1/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic1/canal-data-3.txt => canal-json/database/schemaevolution/topic1/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic2/canal-data-1.txt => canal-json/database/schemaevolution/topic2/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic2/canal-data-2.txt => canal-json/database/schemaevolution/topic2/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/schemaevolution/topic2/canal-data-3.txt => canal-json/database/schemaevolution/topic2/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/database/tostring/canal-data-1.txt => canal-json/database/tostring/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/alltype/canal-data.txt => canal-json/table/alltype/canal-json-data.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/computedcolumn/canal-data-1.txt => canal-json/table/computedcolumn/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/computedcolumn/canal-data-2.txt => canal-json/table/computedcolumn/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal => canal-json}/table/event/event-delete.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal => canal-json}/table/event/event-insert.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal => canal-json}/table/event/event-row.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal => canal-json}/table/event/event-update.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/incomplete/canal-data-1.txt => canal-json/table/incomplete/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/incomplete/canal-data-2.txt => canal-json/table/incomplete/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/initialemptytopic/canal-data-1.txt => canal-json/table/initialemptytopic/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/nononddldata/canal-data-1.txt => canal-json/table/nononddldata/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/nonpk/canal-data-1.txt => canal-json/table/nonpk/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/optionschange/canal-data-1.txt => canal-json/table/optionschange/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/optionschange/canal-data-2.txt => canal-json/table/optionschange/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolution/canal-data-1.txt => canal-json/table/schemaevolution/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolution/canal-data-2.txt => canal-json/table/schemaevolution/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolution/canal-data-3.txt => canal-json/table/schemaevolution/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolution/canal-data-4.txt => canal-json/table/schemaevolution/canal-json-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolution/canal-data-5.txt => canal-json/table/schemaevolution/canal-json-data-5.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmissingddl/canal-data-1.txt => canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmissingddl/canal-data-2.txt => canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmissingddl/canal-data-3.txt => canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmissingddl/canal-data-4.txt => canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmissingddl/canal-data-5.txt => canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmultiple/canal-data-1.txt => canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/schemaevolutionmultiple/canal-data-2.txt => canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/startupmode/canal-data-1.txt => canal-json/table/startupmode/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/startupmode/canal-data-2.txt => canal-json/table/startupmode/canal-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/tostring/canal-data-1.txt => canal-json/table/tostring/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal/table/watermark/canal-data-1.txt => canal-json/table/watermark/canal-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/include/topic0/debezium-data-1.txt => debezium-json/database/include/topic0/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/prefixsuffix/topic0/debezium-data-1.txt => debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/prefixsuffix/topic0/debezium-data-2.txt => debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/prefixsuffix/topic1/debezium-data-1.txt => debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/prefixsuffix/topic1/debezium-data-2.txt => debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/schemaevolution/topic0/debezium-data-1.txt => debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/schemaevolution/topic0/debezium-data-2.txt => debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/schemaevolution/topic1/debezium-data-1.txt => debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/database/schemaevolution/topic1/debezium-data-2.txt => debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/computedcolumn/debezium-data-1.txt => debezium-json/table/computedcolumn/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/nestedtype/debezium-data-1.txt => debezium-json/table/nestedtype/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/nullvalue/debezium-data-1.txt => debezium-json/table/nullvalue/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/nullvalue/debezium-data-2.txt => debezium-json/table/nullvalue/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/alltype/debezium-data-1.txt => debezium-json/table/schema/alltype/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/computedcolumn/debezium-data-1.txt => debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/include/debezium-data-1.txt => debezium-json/table/schema/include/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/schemaevolution/debezium-data-1.txt => debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/schemaevolution/debezium-data-2.txt => debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/schemaevolution/debezium-data-3.txt => debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schema/schemaevolution/debezium-data-4.txt => debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schemaevolution/debezium-data-1.txt => debezium-json/table/schemaevolution/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schemaevolution/debezium-data-2.txt => debezium-json/table/schemaevolution/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/schemaevolution/debezium-data-3.txt => debezium-json/table/schemaevolution/debezium-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/startupmode/debezium-data-1.txt => debezium-json/table/startupmode/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/startupmode/debezium-data-2.txt => debezium-json/table/startupmode/debezium-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium/table/watermark/debezium-data-1.txt => debezium-json/table/watermark/debezium-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/include/topic0/maxwell-data-1.txt => maxwell-json/database/include/topic0/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt => maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt => maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt => maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt => maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/schemaevolution/topic0/maxwell-data-1.txt => maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/schemaevolution/topic0/maxwell-data-2.txt => maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/schemaevolution/topic1/maxwell-data-1.txt => maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/database/schemaevolution/topic1/maxwell-data-2.txt => maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/computedcolumn/maxwell-data-1.txt => maxwell-json/table/computedcolumn/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/schemaevolution/maxwell-data-1.txt => maxwell-json/table/schemaevolution/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/schemaevolution/maxwell-data-2.txt => maxwell-json/table/schemaevolution/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/schemaevolution/maxwell-data-3.txt => maxwell-json/table/schemaevolution/maxwell-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/schemaevolution/maxwell-data-4.txt => maxwell-json/table/schemaevolution/maxwell-json-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/startupmode/maxwell-data-1.txt => maxwell-json/table/startupmode/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/startupmode/maxwell-data-2.txt => maxwell-json/table/startupmode/maxwell-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell/table/watermark/maxwell-data-1.txt => maxwell-json/table/watermark/maxwell-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/case-insensitive/ogg-data-1.txt => ogg-json/database/case-insensitive/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/include/topic0/ogg-data-1.txt => ogg-json/database/include/topic0/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/prefixsuffix/topic0/ogg-data-1.txt => ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/prefixsuffix/topic0/ogg-data-2.txt => ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/prefixsuffix/topic1/ogg-data-1.txt => ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/prefixsuffix/topic1/ogg-data-2.txt => ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/schemaevolution/topic0/ogg-data-1.txt => ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/schemaevolution/topic0/ogg-data-2.txt => ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/schemaevolution/topic1/ogg-data-1.txt => ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/database/schemaevolution/topic1/ogg-data-2.txt => ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/computedcolumn/ogg-data-1.txt => ogg-json/table/computedcolumn/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg => ogg-json}/table/event/event-delete.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg => ogg-json}/table/event/event-insert.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg => ogg-json}/table/event/event-update.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/schemaevolution/ogg-data-1.txt => ogg-json/table/schemaevolution/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/schemaevolution/ogg-data-2.txt => ogg-json/table/schemaevolution/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/schemaevolution/ogg-data-3.txt => ogg-json/table/schemaevolution/ogg-json-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/startupmode/ogg-data-1.txt => ogg-json/table/startupmode/ogg-json-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/startupmode/ogg-data-2.txt => ogg-json/table/startupmode/ogg-json-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg/table/watermark/ogg-data-1.txt => ogg-json/table/watermark/ogg-json-data-1.txt} (100%) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java index 003d382259ca..6fae18a723d8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java @@ -69,8 +69,10 @@ public enum DataFormat { // Add more data formats here if needed private final RecordParserFactory parser; + // Deserializer for Kafka private final Function> kafkaDeserializer; + // Deserializer for Pulsar private final Function> pulsarDeserializer; diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java index 96070048b8a3..89c4a0019bbe 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java @@ -46,6 +46,7 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaCanalSyncDatabaseActionITCase extends KafkaActionITCaseBase { + private static final String FORMAT = "canal-json"; @Test @Timeout(60) @@ -61,12 +62,14 @@ public void testSchemaEvolutionMultiTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( topics.get(i), - "kafka/canal/database/schemaevolution/topic%s/canal-data-1.txt", - i); + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + FORMAT, + i, + FORMAT); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); if (ThreadLocalRandom.current().nextBoolean()) { kafkaConfig.put(TOPIC.key(), String.join(";", topics)); } else { @@ -94,12 +97,14 @@ public void testSchemaEvolutionOneTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( topics.get(0), - "kafka/canal/database/schemaevolution/topic%s/canal-data-1.txt", - i); + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + FORMAT, + i, + FORMAT); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = @@ -141,8 +146,10 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/canal/database/schemaevolution/topic%s/canal-data-2.txt", - i); + "kafka/%s/database/schemaevolution/topic%s/%s-data-2.txt", + FORMAT, + i, + FORMAT); } rowType1 = @@ -180,8 +187,10 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/canal/database/schemaevolution/topic%s/canal-data-3.txt", - i); + "kafka/%s/database/schemaevolution/topic%s/%s-data-3.txt", + FORMAT, + i, + FORMAT); } rowType1 = @@ -222,7 +231,7 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int @Test public void testTopicIsEmpty() { Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); @@ -255,12 +264,16 @@ public void testTableAffixMultiTopic() throws Exception { for (int i = 0; i < topics.size(); i++) { writeRecordsToKafka( - topics.get(i), "kafka/canal/database/prefixsuffix/topic%s/canal-data-1.txt", i); + topics.get(i), + "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", + FORMAT, + i, + FORMAT); } // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -296,12 +309,16 @@ public void testTableAffixOneTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( - topics.get(0), "kafka/canal/database/prefixsuffix/topic%s/canal-data-1.txt", i); + topics.get(0), + "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", + FORMAT, + i, + FORMAT); } // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -342,8 +359,10 @@ private void testTableAffixImpl(List topics, boolean writeOne, int fileC for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/canal/database/prefixsuffix/topic%s/canal-data-2.txt", - i); + "kafka/%s/database/prefixsuffix/topic%s/%s-data-2.txt", + FORMAT, + i, + FORMAT); } rowType1 = RowType.of( @@ -376,8 +395,10 @@ private void testTableAffixImpl(List topics, boolean writeOne, int fileC for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/canal/database/prefixsuffix/topic%s/canal-data-3.txt", - i); + "kafka/%s/database/prefixsuffix/topic%s/%s-data-3.txt", + FORMAT, + i, + FORMAT); } rowType1 = @@ -450,11 +471,12 @@ private void includingAndExcludingTablesImpl( final String topic1 = "include_exclude" + UUID.randomUUID(); List topics = Collections.singletonList(topic1); topics.forEach(topic -> createTestTopic(topic, 1, 1)); - writeRecordsToKafka(topics.get(0), "kafka/canal/database/include/topic0/canal-data-1.txt"); + writeRecordsToKafka( + topics.get(0), "kafka/%s/database/include/topic0/%s-data-1.txt", FORMAT, FORMAT); // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -474,10 +496,10 @@ private void includingAndExcludingTablesImpl( public void testTypeMappingToString() throws Exception { final String topic = "map-to-string"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/database/tostring/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/database/tostring/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = @@ -520,10 +542,11 @@ public void testCatalogAndTableConfig() { public void testCaseInsensitive() throws Exception { final String topic = "case-insensitive"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/database/case-insensitive/canal-data-1.txt"); + writeRecordsToKafka( + topic, "kafka/%s/database/case-insensitive/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = @@ -555,10 +578,10 @@ public void testCaseInsensitive() throws Exception { public void testCannotSynchronizeIncompleteJson() throws Exception { final String topic = "incomplete"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/database/incomplete/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/database/incomplete/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index bb259cd0f34d..85c59738c208 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -57,7 +57,7 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaCanalSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String CANAL = "canal"; + private static final String FORMAT = "canal-json"; @Test @Timeout(60) @@ -74,10 +74,10 @@ public void testSchemaEvolutionWithMissingDdl() throws Exception { private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { final String topic = "schema_evolution"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-1.txt", sourceDir); + writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-1.txt", FORMAT, sourceDir, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -105,7 +105,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce List expected = Arrays.asList("+I[1, 1, one]", "+I[1, 2, two]", "+I[2, 4, four]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-2.txt", sourceDir); + writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-2.txt", FORMAT, sourceDir, FORMAT); rowType = RowType.of( @@ -126,7 +126,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[1, 6, six, 60]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-3.txt", sourceDir); + writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-3.txt", FORMAT, sourceDir, FORMAT); rowType = RowType.of( @@ -148,7 +148,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[2, 8, eight, 80000000000]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-4.txt", sourceDir); + writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-4.txt", FORMAT, sourceDir, FORMAT); rowType = RowType.of( @@ -174,7 +174,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110], 9.9]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-5.txt", sourceDir); + writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-5.txt", FORMAT, sourceDir, FORMAT); rowType = RowType.of( @@ -206,10 +206,11 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce public void testMultipleSchemaEvolutions() throws Exception { final String topic = "schema_evolution_multiple"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt"); + writeRecordsToKafka( + topic, "kafka/%s/table/schemaevolutionmultiple/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); if (ThreadLocalRandom.current().nextBoolean()) { kafkaConfig.put(TOPIC.key(), topic); @@ -239,7 +240,8 @@ private void testSchemaEvolutionMultipleImpl(String topic) throws Exception { List expected = Collections.singletonList("+I[1, one, 10, string_1]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt"); + writeRecordsToKafka( + topic, "kafka/%s/table/schemaevolutionmultiple/%s-data-2.txt", FORMAT, FORMAT); rowType = RowType.of( @@ -275,10 +277,10 @@ public void testAllTypes() throws Exception { private void testAllTypesOnce() throws Exception { final String topic = "all_type" + UUID.randomUUID(); createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/alltype/canal-data.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/alltype/%s-data.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -535,7 +537,7 @@ private void testAllTypesImpl() throws Exception { public void testNotSupportFormat() throws Exception { final String topic = "not_support"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); kafkaConfig.put(VALUE_FORMAT.key(), "togg-json"); @@ -558,10 +560,10 @@ public void testNotSupportFormat() throws Exception { public void testKafkaNoNonDdlData() throws Exception { final String topic = "no_non_ddl_data"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/nononddldata/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/nononddldata/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -581,10 +583,10 @@ public void testKafkaNoNonDdlData() throws Exception { public void testAssertSchemaCompatible() throws Exception { final String topic = "assert_schema_compatible"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); // create an incompatible table @@ -617,10 +619,10 @@ public void testAssertSchemaCompatible() throws Exception { public void testStarUpOptionSpecific() throws Exception { final String topic = "start_up_specific"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); @@ -653,10 +655,11 @@ public void testStarUpOptionSpecific() throws Exception { public void testStarUpOptionLatest() throws Exception { final String topic = "start_up_latest"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, true, "kafka/canal/table/startupmode/canal-data-1.txt"); + writeRecordsToKafka( + topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -670,7 +673,7 @@ public void testStarUpOptionLatest() throws Exception { // wait task running to commit LATEST_OFFSET Thread.sleep(5_000); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); FileStoreTable table = getFileStoreTable(tableName); @@ -693,10 +696,11 @@ public void testStarUpOptionLatest() throws Exception { public void testStarUpOptionTimestamp() throws Exception { final String topic = "start_up_timestamp"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, true, "kafka/canal/table/startupmode/canal-data-1.txt"); + writeRecordsToKafka( + topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); kafkaConfig.put( @@ -709,7 +713,7 @@ public void testStarUpOptionTimestamp() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); FileStoreTable table = getFileStoreTable(tableName); @@ -732,10 +736,10 @@ public void testStarUpOptionTimestamp() throws Exception { public void testStarUpOptionEarliest() throws Exception { final String topic = "start_up_earliest"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -746,7 +750,7 @@ public void testStarUpOptionEarliest() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); FileStoreTable table = getFileStoreTable(tableName); @@ -771,10 +775,10 @@ public void testStarUpOptionEarliest() throws Exception { public void testStarUpOptionGroup() throws Exception { final String topic = "start_up_group"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); KafkaSyncTableAction action = @@ -785,7 +789,7 @@ public void testStarUpOptionGroup() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); FileStoreTable table = getFileStoreTable(tableName); @@ -810,10 +814,10 @@ public void testStarUpOptionGroup() throws Exception { public void testComputedColumn() throws Exception { String topic = "computed_column"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/computedcolumn/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -842,10 +846,10 @@ public void testComputedColumn() throws Exception { public void testTypeMappingToString() throws Exception { final String topic = "map-to-string"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/tostring/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/tostring/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -887,10 +891,10 @@ public void testCatalogAndTableConfig() { public void testCDCOperations(boolean ignoreDelete) throws Exception { final String topic = "event-insert" + UUID.randomUUID(); createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/event/event-row.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/event/event-row.txt", FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = getBasicTableConfig(); @@ -921,7 +925,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110], 9.9]"); waitForResult(expectedRow, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/event/event-insert.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/event/event-insert.txt", FORMAT); // For the INSERT operation List expectedInsert = @@ -932,7 +936,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[2, 4, four, NULL, NULL, NULL, NULL]"); waitForResult(expectedInsert, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/event/event-update.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/event/event-update.txt", FORMAT); // For the UPDATE operation List expectedUpdate = @@ -943,7 +947,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[2, 4, four, NULL, NULL, NULL, NULL]"); waitForResult(expectedUpdate, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/canal/table/event/event-delete.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/event/event-delete.txt", FORMAT); // For the DELETE operation List expectedDelete = @@ -981,7 +985,7 @@ public void testSyncWithInitialEmptyTopic() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/canal/table/initialemptytopic/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/initialemptytopic/%s-data-1.txt", FORMAT); RowType rowType = RowType.of( @@ -1004,10 +1008,10 @@ public void testSyncWithInitialEmptyTopic() throws Exception { public void testSynchronizeIncompleteJson() throws Exception { String topic = "incomplete"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/incomplete/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/incomplete/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -1035,10 +1039,10 @@ public void testSynchronizeIncompleteJson() throws Exception { public void testSynchronizeNonPkTable() throws Exception { String topic = "non_pk"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/nonpk/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/nonpk/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig).withTableConfig(getBasicTableConfig()).build(); @@ -1063,10 +1067,10 @@ public void testSynchronizeNonPkTable() throws Exception { public void testMissingDecimalPrecision() throws Exception { String topic = "missing-decimal-precision"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/incomplete/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/incomplete/%s-data-2.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -1105,11 +1109,12 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval Collections.singletonList("_id"), Collections.emptyMap()); } else { - writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-2.txt"); + writeRecordsToKafka( + topic, "kafka/%s/table/computedcolumn/%s-data-2.txt", FORMAT, FORMAT); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -1122,7 +1127,8 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval runActionWithDefaultEnv(action); if (triggerSchemaRetrievalException) { - writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-2.txt"); + writeRecordsToKafka( + topic, "kafka/%s/table/computedcolumn/%s-data-2.txt", FORMAT, FORMAT); } RowType rowType = @@ -1141,6 +1147,6 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(CANAL); + testWaterMarkSyncTable(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java index 94efa60c71e4..4d5164f60f56 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java @@ -23,52 +23,52 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaDebeziumSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String DEBEZIUM = "debezium"; + private static final String FORMAT = "debezium-json"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(DEBEZIUM); + testSchemaEvolutionMultiTopic(FORMAT); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(DEBEZIUM); + testSchemaEvolutionOneTopic(FORMAT); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(DEBEZIUM); + testTopicIsEmpty(FORMAT); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(DEBEZIUM); + testTableAffixMultiTopic(FORMAT); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(DEBEZIUM); + testTableAffixOneTopic(FORMAT); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(DEBEZIUM); + testIncludingTables(FORMAT); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(DEBEZIUM); + testExcludingTables(FORMAT); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(DEBEZIUM); + testIncludingAndExcludingTables(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index 04dfb3769a61..245c871eebb8 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -36,84 +36,84 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaDebeziumSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String DEBEZIUM = "debezium"; + private static final String FORMAT = "debezium-json"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", DEBEZIUM); + runSingleTableSchemaEvolution("schemaevolution", FORMAT); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(DEBEZIUM); + testNotSupportFormat(FORMAT); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(DEBEZIUM); + testAssertSchemaCompatible(FORMAT); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(DEBEZIUM); + testStarUpOptionSpecific(FORMAT); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(DEBEZIUM); + testStarUpOptionLatest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(DEBEZIUM); + testStarUpOptionTimestamp(FORMAT); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(DEBEZIUM); + testStarUpOptionEarliest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(DEBEZIUM); + testStarUpOptionGroup(FORMAT); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(DEBEZIUM); + testComputedColumn(FORMAT); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(DEBEZIUM); + testWaterMarkSyncTable(FORMAT); } @Test @Timeout(60) public void testKafkaBuildSchemaWithDelete() throws Exception { - testKafkaBuildSchemaWithDelete(DEBEZIUM); + testKafkaBuildSchemaWithDelete(FORMAT); } @Test @Timeout(60) public void testSchemaIncludeRecord1() throws Exception { - testSchemaIncludeRecord(DEBEZIUM); + testSchemaIncludeRecord(FORMAT); } @Test @Timeout(60) public void testAllTypesWithSchema() throws Exception { - testAllTypesWithSchemaImpl(DEBEZIUM); + testAllTypesWithSchemaImpl(FORMAT); } @Test @@ -122,13 +122,13 @@ public void testMessageWithNullValue() throws Exception { final String topic = "test_null_value"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/debezium/table/nullvalue/debezium-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/nullvalue/%s-data-1.txt", FORMAT, FORMAT); // write null value kafkaProducer.send(new ProducerRecord<>(topic, null)); - writeRecordsToKafka(topic, "kafka/debezium/table/nullvalue/debezium-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/nullvalue/%s-data-2.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java index 189863f44c0d..afa03081050c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java @@ -24,52 +24,52 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaMaxwellSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String MAXWELL = "maxwell"; + private static final String FORMAT = "maxwell-json"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(MAXWELL); + testSchemaEvolutionMultiTopic(FORMAT); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(MAXWELL); + testSchemaEvolutionOneTopic(FORMAT); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(MAXWELL); + testTopicIsEmpty(FORMAT); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(MAXWELL); + testTableAffixMultiTopic(FORMAT); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(MAXWELL); + testTableAffixOneTopic(FORMAT); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(MAXWELL); + testIncludingTables(FORMAT); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(MAXWELL); + testExcludingTables(FORMAT); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(MAXWELL); + testIncludingAndExcludingTables(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java index f15c06c5eb87..f5154d404dbb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java @@ -24,71 +24,71 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaMaxwellSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String MAXWELL = "maxwell"; + private static final String FORMAT = "maxwell-json"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", MAXWELL); + runSingleTableSchemaEvolution("schemaevolution", FORMAT); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(MAXWELL); + testNotSupportFormat(FORMAT); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(MAXWELL); + testAssertSchemaCompatible(FORMAT); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(MAXWELL); + testStarUpOptionSpecific(FORMAT); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(MAXWELL); + testStarUpOptionLatest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(MAXWELL); + testStarUpOptionTimestamp(FORMAT); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(MAXWELL); + testStarUpOptionEarliest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(MAXWELL); + testStarUpOptionGroup(FORMAT); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(MAXWELL); + testComputedColumn(FORMAT); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(MAXWELL); + testWaterMarkSyncTable(FORMAT); } @Test @Timeout(60) public void testFieldValNullSyncTable() throws Exception { - testTableFiledValNull(MAXWELL); + testTableFiledValNull(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java index 9eacbfea2874..4ed7dd9935f9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java @@ -23,58 +23,58 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaOggSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String OGG = "ogg"; + private static final String FORMAT = "ogg-json"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(OGG); + testSchemaEvolutionMultiTopic(FORMAT); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(OGG); + testSchemaEvolutionOneTopic(FORMAT); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(OGG); + testTopicIsEmpty(FORMAT); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(OGG); + testTableAffixMultiTopic(FORMAT); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(OGG); + testTableAffixOneTopic(FORMAT); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(OGG); + testIncludingTables(FORMAT); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(OGG); + testExcludingTables(FORMAT); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(OGG); + testIncludingAndExcludingTables(FORMAT); } @Test @Timeout(60) public void testCaseInsensitive() throws Exception { - testCaseInsensitive(OGG); + testCaseInsensitive(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java index 6394b5b71978..54dc2ec14b97 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java @@ -24,71 +24,71 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaOggSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String OGG = "ogg"; + private static final String FORMAT = "ogg-json"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", OGG); + runSingleTableSchemaEvolution("schemaevolution", FORMAT); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(OGG); + testNotSupportFormat(FORMAT); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(OGG); + testAssertSchemaCompatible(FORMAT); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(OGG); + testStarUpOptionSpecific(FORMAT); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(OGG); + testStarUpOptionLatest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(OGG); + testStarUpOptionTimestamp(FORMAT); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(OGG); + testStarUpOptionEarliest(FORMAT); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(OGG); + testStarUpOptionGroup(FORMAT); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(OGG); + testComputedColumn(FORMAT); } @Test @Timeout(60) public void testCDCOperations() throws Exception { - testCDCOperations(OGG); + testCDCOperations(FORMAT); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(OGG); + testWaterMarkSyncTable(FORMAT); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java index 84bb802bc195..6e8b69320ef9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java @@ -43,15 +43,17 @@ /** Tests for building schema from Kafka. */ public class KafkaSchemaITCase extends KafkaActionITCaseBase { + private static final String FORMAT = "canal-json"; + @Test @Timeout(60) public void testKafkaSchema() throws Exception { final String topic = "test_kafka_schema"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); - kafkaConfig.setString(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.setString(VALUE_FORMAT.key(), FORMAT); kafkaConfig.setString(TOPIC.key(), topic); Schema kafkaSchema = @@ -72,10 +74,10 @@ kafkaConfig, new KafkaDebeziumJsonDeserializationSchema()), public void testTableOptionsChange() throws Exception { final String topic = "test_table_options_change"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/table/optionschange/canal-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/optionschange/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = new HashMap<>(); tableConfig.put("bucket", "1"); @@ -88,7 +90,7 @@ public void testTableOptionsChange() throws Exception { waitingTables(tableName); jobClient.cancel(); - writeRecordsToKafka(topic, "kafka/canal/table/optionschange/canal-data-2.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/optionschange/%s-data-2.txt", FORMAT, FORMAT); tableConfig.put("sink.savepoint.auto-tag", "true"); tableConfig.put("tag.num-retained-max", "5"); @@ -115,10 +117,11 @@ public void testTableOptionsChange() throws Exception { public void testNewlyAddedTablesOptionsChange() throws Exception { final String topic = "test_database_options_change"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/canal/database/schemaevolution/topic0/canal-data-1.txt"); + writeRecordsToKafka( + topic, "kafka/%s/database/schemaevolution/topic0/%s-data-1.txt", FORMAT, FORMAT); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); + kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = new HashMap<>(); tableConfig.put("bucket", "1"); @@ -140,7 +143,8 @@ public void testNewlyAddedTablesOptionsChange() throws Exception { tableConfig.put("snapshot.num-retained.max", "10"); tableConfig.put("changelog-producer", "input"); - writeRecordsToKafka(topic, "kafka/canal/database/schemaevolution/topic1/canal-data-1.txt"); + writeRecordsToKafka( + topic, "kafka/%s/database/schemaevolution/topic1/%s-data-1.txt", FORMAT, FORMAT); KafkaSyncDatabaseAction action2 = syncDatabaseActionBuilder(kafkaConfig).withTableConfig(tableConfig).build(); runActionWithDefaultEnv(action2); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 3d66a1da8464..6b8dc631df8d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -21,6 +21,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataField; @@ -61,7 +62,7 @@ protected void runSingleTableSchemaEvolution(String sourceDir, String format) th writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-1.txt", format, sourceDir, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -163,7 +164,7 @@ protected void testAssertSchemaCompatible(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); // create an incompatible table @@ -196,7 +197,7 @@ protected void testStarUpOptionSpecific(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); @@ -232,7 +233,7 @@ protected void testStarUpOptionLatest(String format) throws Exception { topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -272,7 +273,7 @@ public void testStarUpOptionTimestamp(String format) throws Exception { topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); kafkaConfig.put( @@ -312,7 +313,7 @@ public void testStarUpOptionEarliest(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -352,7 +353,7 @@ public void testStarUpOptionGroup(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); KafkaSyncTableAction action = @@ -392,7 +393,7 @@ public void testComputedColumn(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/computedcolumn/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -424,7 +425,7 @@ protected void testCDCOperations(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/event/event-insert.txt", format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -480,13 +481,15 @@ public void testKafkaBuildSchemaWithDelete(String format) throws Exception { topic, "kafka/%s/table/schema/schemaevolution/%s-data-4.txt", format, format); Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); - kafkaConfig.setString(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.setString(VALUE_FORMAT.key(), format); kafkaConfig.setString(TOPIC.key(), topic); + DataFormat dataFormat = DataFormat.fromConfigString(format); + Schema kafkaSchema = MessageQueueSchemaUtils.getSchema( getKafkaEarliestConsumer( - kafkaConfig, new KafkaDebeziumJsonDeserializationSchema()), + kafkaConfig, dataFormat.createKafkaDeserializer(kafkaConfig)), getDataFormat(kafkaConfig), TypeMapping.defaultMapping()); List fields = new ArrayList<>(); @@ -504,7 +507,7 @@ public void testWaterMarkSyncTable(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/watermark/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); Map config = getBasicTableConfig(); @@ -533,10 +536,10 @@ public void testWaterMarkSyncTable(String format) throws Exception { public void testSchemaIncludeRecord(String format) throws Exception { String topic = "schema_include"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/debezium/table/schema/include/debezium-data-1.txt"); + writeRecordsToKafka(topic, "kafka/debezium-json/table/schema/include/debezium-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -567,10 +570,10 @@ public void testSchemaIncludeRecord(String format) throws Exception { public void testAllTypesWithSchemaImpl(String format) throws Exception { String topic = "schema_include_all_type"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/debezium/table/schema/alltype/debezium-data-1.txt"); + writeRecordsToKafka(topic, "kafka/debezium-json/table/schema/alltype/debezium-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -812,7 +815,7 @@ protected void testTableFiledValNull(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-4.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java index 969a9416f6c1..d139ef2aa6b7 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java @@ -203,8 +203,12 @@ private void deleteTopics() throws Exception { } } - protected List getMessages(String resource) throws IOException { - URL url = PulsarActionITCaseBase.class.getClassLoader().getResource(resource); + protected List getMessages(String resourceDirFormat, Object... args) + throws IOException { + URL url = + PulsarActionITCaseBase.class + .getClassLoader() + .getResource(String.format(resourceDirFormat, args)); assertThat(url).isNotNull(); java.nio.file.Path path = new File(url.getFile()).toPath(); List lines = Files.readAllLines(path); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java index 255d43bfbc81..2087ad6081d8 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java @@ -40,6 +40,7 @@ /** Tests for building schema from Pulsar. */ public class PulsarSchemaITCase extends PulsarActionITCaseBase { + private static final String FORMAT = "canal-json"; @Test @Timeout(60) @@ -48,12 +49,13 @@ public void testPulsarSchema() throws Exception { createTopic(topic); // ---------- Write the Canal json into pulsar ------------------- - List messages = getMessages("kafka/canal/table/schemaevolution/canal-data-1.txt"); + List messages = + getMessages("kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); sendMessages(topic, messages); Configuration pulsarConfig = Configuration.fromMap(getBasicPulsarConfig()); pulsarConfig.setString(TOPIC.key(), topic); - pulsarConfig.set(VALUE_FORMAT, "canal-json"); + pulsarConfig.set(VALUE_FORMAT, FORMAT); Schema pulsarSchema = MessageQueueSchemaUtils.getSchema( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java index 37e6b66bf554..86f866a2d09c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java @@ -39,6 +39,7 @@ /** IT cases for {@link PulsarSyncDatabaseAction}. */ public class PulsarSyncDatabaseActionITCase extends PulsarActionITCaseBase { + private static final String FORMAT = "canal-json"; @Test @Timeout(60) @@ -58,9 +59,8 @@ public void testSchemaEvolutionMultiTopic() throws Exception { sendMessages( topics.get(i), getMessages( - "kafka/canal/database/schemaevolution/topic" - + i - + "/canal-data-1.txt")); + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + FORMAT, i, FORMAT)); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -68,7 +68,7 @@ public void testSchemaEvolutionMultiTopic() throws Exception { Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); - pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); + pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); if (ThreadLocalRandom.current().nextBoolean()) { pulsarConfig.put(TOPIC.key(), String.join(";", topics)); } else { @@ -100,9 +100,8 @@ public void testSchemaEvolutionOneTopic() throws Exception { sendMessages( topics.get(0), getMessages( - "kafka/canal/database/schemaevolution/topic" - + i - + "/canal-data-1.txt")); + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + FORMAT, i, FORMAT)); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -110,7 +109,7 @@ public void testSchemaEvolutionOneTopic() throws Exception { Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); - pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); + pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); pulsarConfig.put(TOPIC.key(), String.join(";", topics)); PulsarSyncDatabaseAction action = @@ -154,9 +153,8 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int sendMessages( writeOne ? topics.get(0) : topics.get(i), getMessages( - "kafka/canal/database/schemaevolution/topic" - + i - + "/canal-data-2.txt")); + "kafka/%s/database/schemaevolution/topic%s/%s-data-2.txt", + FORMAT, i, FORMAT)); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -199,9 +197,8 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int sendMessages( writeOne ? topics.get(0) : topics.get(i), getMessages( - "kafka/canal/database/schemaevolution/topic" - + i - + "/canal-data-3.txt")); + "kafka/%s/database/schemaevolution/topic%s/%s-data-3.txt", + FORMAT, i, FORMAT)); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java index a4fb332d5033..672a03c676c0 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java @@ -40,6 +40,7 @@ /** IT cases for {@link PulsarSyncTableAction}. */ public class PulsarSyncTableActionITCase extends PulsarActionITCaseBase { + private static final String FORMAT = "canal-json"; @Test @Timeout(120) @@ -53,8 +54,7 @@ private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { createTopic(topic, 1); // ---------- Write the Canal json into Pulsar ------------------- sendMessages( - topic, - getMessages(String.format("kafka/canal/table/%s/canal-data-1.txt", sourceDir))); + topic, getMessages("kafka/%s/table/%s/%s-data-1.txt", FORMAT, sourceDir, FORMAT)); Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); @@ -63,7 +63,7 @@ private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { } else { pulsarConfig.put(TOPIC_PATTERN.key(), "schema_.*"); } - pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); + pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); PulsarSyncTableAction action = syncTableActionBuilder(pulsarConfig) @@ -92,8 +92,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, - getMessages(String.format("kafka/canal/table/%s/canal-data-2.txt", sourceDir))); + topic, getMessages("kafka/%s/table/%s/%s-data-2.txt", FORMAT, sourceDir, FORMAT)); rowType = RowType.of( @@ -115,8 +114,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, - getMessages(String.format("kafka/canal/table/%s/canal-data-3.txt", sourceDir))); + topic, getMessages("kafka/%s/table/%s/%s-data-3.txt", FORMAT, sourceDir, FORMAT)); rowType = RowType.of( @@ -139,8 +137,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, - getMessages(String.format("kafka/canal/table/%s/canal-data-4.txt", sourceDir))); + topic, getMessages("kafka/%s/table/%s/%s-data-4.txt", FORMAT, sourceDir, FORMAT)); rowType = RowType.of( @@ -167,8 +164,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, - getMessages(String.format("kafka/canal/table/%s/canal-data-5.txt", sourceDir))); + topic, getMessages("kafka/%s/table/%s/%s-data-5.txt", FORMAT, sourceDir, FORMAT)); rowType = RowType.of( @@ -201,12 +197,12 @@ public void testWaterMarkSyncTable() throws Exception { String topic = "watermark"; topics = Collections.singletonList(topic); createTopic(topic, 1); - sendMessages(topic, getMessages("kafka/canal/table/watermark/canal-data-1.txt")); + sendMessages(topic, getMessages("kafka/%s/table/watermark/%s-data-1.txt", FORMAT, FORMAT)); Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); pulsarConfig.put(TOPIC.key(), topic); - pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); + pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); Map config = getBasicTableConfig(); config.put("tag.automatic-creation", "watermark"); config.put("tag.creation-period", "hourly"); diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/case-insensitive/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/case-insensitive/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/case-insensitive/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/case-insensitive/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/include/topic0/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/include/topic0/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/include/topic0/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/include/topic0/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/incomplete/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/incomplete/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/incomplete/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/incomplete/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/tostring/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/tostring/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/tostring/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/tostring/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/alltype/canal-data.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/alltype/canal-json-data.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/alltype/canal-data.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/alltype/canal-json-data.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-delete.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-delete.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-delete.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-insert.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-insert.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-insert.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-row.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-row.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-row.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-row.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-update.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-update.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-update.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-update.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/initialemptytopic/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/initialemptytopic/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/initialemptytopic/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/initialemptytopic/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nononddldata/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nononddldata/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nononddldata/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nononddldata/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nonpk/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nonpk/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nonpk/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nonpk/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-5.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-5.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-5.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-5.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-5.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-5.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/tostring/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/tostring/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/tostring/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/tostring/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/watermark/canal-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/watermark/canal-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/watermark/canal-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/watermark/canal-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/include/topic0/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/include/topic0/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/computedcolumn/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/computedcolumn/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nestedtype/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nestedtype/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nestedtype/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nestedtype/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/alltype/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/alltype/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/alltype/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/alltype/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/computedcolumn/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/computedcolumn/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/include/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/include/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/include/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/include/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/watermark/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/watermark/debezium-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/watermark/debezium-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/watermark/debezium-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/include/topic0/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/include/topic0/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/include/topic0/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/include/topic0/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/computedcolumn/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/computedcolumn/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/computedcolumn/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/computedcolumn/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/watermark/maxwell-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/watermark/maxwell-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/watermark/maxwell-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/watermark/maxwell-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/case-insensitive/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/case-insensitive/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/include/topic0/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/include/topic0/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/computedcolumn/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/computedcolumn/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/computedcolumn/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/computedcolumn/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-delete.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-delete.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-insert.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-insert.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-update.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-update.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/watermark/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/watermark/ogg-json-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/watermark/ogg-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/watermark/ogg-json-data-1.txt From d1e41fbf7f3308b5b8503d9a0f3129f0eebe8c05 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Mon, 13 May 2024 10:17:38 +0800 Subject: [PATCH 04/12] fix --- .../cdc/kafka/KafkaCanalSyncTableActionITCase.java | 3 ++- .../cdc/kafka/KafkaSyncDatabaseActionITCase.java | 14 +++++++------- .../cdc/kafka/KafkaSyncTableActionITCase.java | 4 ++-- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index 85c59738c208..6976c8570027 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -985,7 +985,8 @@ public void testSyncWithInitialEmptyTopic() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/%s/table/initialemptytopic/%s-data-1.txt", FORMAT); + writeRecordsToKafka( + topic, "kafka/%s/table/initialemptytopic/%s-data-1.txt", FORMAT, FORMAT); RowType rowType = RowType.of( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index 16aff6984098..9a3158907cb0 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -59,7 +59,7 @@ protected void testSchemaEvolutionMultiTopic(String format) throws Exception { } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -87,7 +87,7 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception { } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -194,7 +194,7 @@ private void testSchemaEvolutionImpl( protected void testTopicIsEmpty(String format) { Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); @@ -239,7 +239,7 @@ protected void testTableAffixMultiTopic(String format) throws Exception { // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -286,7 +286,7 @@ protected void testTableAffixOneTopic(String format) throws Exception { // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -442,7 +442,7 @@ private void includingAndExcludingTablesImpl( // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -465,7 +465,7 @@ protected void testCaseInsensitive(String format) throws Exception { topic, "kafka/%s/database/case-insensitive/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(VALUE_FORMAT.key(), format); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 6b8dc631df8d..1613a7beb53a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -536,7 +536,7 @@ public void testWaterMarkSyncTable(String format) throws Exception { public void testSchemaIncludeRecord(String format) throws Exception { String topic = "schema_include"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/debezium-json/table/schema/include/debezium-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/schema/include/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); kafkaConfig.put(VALUE_FORMAT.key(), format); @@ -570,7 +570,7 @@ public void testSchemaIncludeRecord(String format) throws Exception { public void testAllTypesWithSchemaImpl(String format) throws Exception { String topic = "schema_include_all_type"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/debezium-json/table/schema/alltype/debezium-data-1.txt"); + writeRecordsToKafka(topic, "kafka/%s/table/schema/alltype/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); kafkaConfig.put(VALUE_FORMAT.key(), format); From 93a71c13c875fa4597b3b37c11be08d9d5774c33 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Mon, 13 May 2024 11:37:43 +0800 Subject: [PATCH 05/12] fix --- .../paimon/flink/action/cdc/format/DataFormat.java | 5 +++++ .../cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java | 3 ++- .../cdc/kafka/KafkaCanalSyncTableActionITCase.java | 3 ++- .../kafka/KafkaDebeziumSyncDatabaseActionITCase.java | 4 +++- .../cdc/kafka/KafkaDebeziumSyncTableActionITCase.java | 3 ++- .../kafka/KafkaMaxwellSyncDatabaseActionITCase.java | 4 +++- .../cdc/kafka/KafkaMaxwellSyncTableActionITCase.java | 4 +++- .../cdc/kafka/KafkaOggSyncDatabaseActionITCase.java | 4 +++- .../cdc/kafka/KafkaOggSyncTableActionITCase.java | 4 +++- .../flink/action/cdc/kafka/KafkaSchemaITCase.java | 3 ++- .../cdc/kafka/KafkaSyncDatabaseActionITCase.java | 11 +++++++---- .../flink/action/cdc/pulsar/PulsarSchemaITCase.java | 3 ++- .../cdc/pulsar/PulsarSyncDatabaseActionITCase.java | 3 ++- .../cdc/pulsar/PulsarSyncTableActionITCase.java | 3 ++- 14 files changed, 41 insertions(+), 16 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java index 6fae18a723d8..896b71416ea1 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/DataFormat.java @@ -108,6 +108,11 @@ public DeserializationSchema createPulsarDeserializer( return pulsarDeserializer.apply(cdcSourceConfig); } + /** Returns the configuration string representation of this data format. */ + public String asConfigString() { + return this.name().toLowerCase().replace("_", "-"); + } + public static DataFormat fromConfigString(String format) { try { return DataFormat.valueOf(format.replace("-", "_").toUpperCase()); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java index 89c4a0019bbe..fa7c64cb92f3 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -46,7 +47,7 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaCanalSyncDatabaseActionITCase extends KafkaActionITCaseBase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index 6976c8570027..6732d8487eba 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -57,7 +58,7 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaCanalSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java index 4d5164f60f56..196230315f5a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java @@ -18,12 +18,14 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaDebeziumSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = "debezium-json"; + private static final String FORMAT = DataFormat.DEBEZIUM_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index 245c871eebb8..ccea411591ff 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -36,7 +37,7 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaDebeziumSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = "debezium-json"; + private static final String FORMAT = DataFormat.DEBEZIUM_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java index afa03081050c..c1d514eaa59d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java @@ -18,13 +18,15 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaMaxwellSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = "maxwell-json"; + private static final String FORMAT = DataFormat.MAXWELL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java index f5154d404dbb..d41fba70df60 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java @@ -18,13 +18,15 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaMaxwellSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = "maxwell-json"; + private static final String FORMAT = DataFormat.MAXWELL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java index 4ed7dd9935f9..ab132c366d64 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java @@ -18,12 +18,14 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaOggSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = "ogg-json"; + private static final String FORMAT = DataFormat.OGG_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java index 54dc2ec14b97..5f368236e6a9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java @@ -18,13 +18,15 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.flink.action.cdc.format.DataFormat; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaOggSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = "ogg-json"; + private static final String FORMAT = DataFormat.OGG_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java index 6e8b69320ef9..49d28774f045 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataField; @@ -43,7 +44,7 @@ /** Tests for building schema from Kafka. */ public class KafkaSchemaITCase extends KafkaActionITCaseBase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index 9a3158907cb0..92eff783b05c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -358,7 +359,7 @@ private void testTableAffixImpl( DataTypes.STRING() }, new String[] {"id", "name", "description", "weight", "address"}); - if (format.equals("debezium")) { + if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { expected = Arrays.asList( "+I[101, scooter, Small 2-wheel scooter, 3.14, Beijing]", @@ -383,7 +384,7 @@ private void testTableAffixImpl( DataTypes.STRING() }, new String[] {"id", "name", "description", "weight", "age"}); - if (format.equals("debezium")) { + if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { expected = Arrays.asList( "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", @@ -497,11 +498,13 @@ protected void testCaseInsensitive(String format) throws Exception { } private DataType getDataType(String format) { - return format.equals("debezium") ? DataTypes.STRING() : DataTypes.STRING().notNull(); + return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) + ? DataTypes.STRING() + : DataTypes.STRING().notNull(); } private List getPrimaryKey(String format) { - return format.equals("debezium") + return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) ? Collections.emptyList() : Collections.singletonList("id"); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java index 2087ad6081d8..fe8ea0a818b4 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; @@ -40,7 +41,7 @@ /** Tests for building schema from Pulsar. */ public class PulsarSchemaITCase extends PulsarActionITCaseBase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java index 86f866a2d09c..7b569a261b25 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.action.cdc.pulsar; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -39,7 +40,7 @@ /** IT cases for {@link PulsarSyncDatabaseAction}. */ public class PulsarSyncDatabaseActionITCase extends PulsarActionITCaseBase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java index 672a03c676c0..79a4b6c64efd 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc.pulsar; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -40,7 +41,7 @@ /** IT cases for {@link PulsarSyncTableAction}. */ public class PulsarSyncTableActionITCase extends PulsarActionITCaseBase { - private static final String FORMAT = "canal-json"; + private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(120) From 47c6879dfd1cda095893210d33f5cd3e1dc1ee13 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Mon, 13 May 2024 12:47:46 +0800 Subject: [PATCH 06/12] fix --- .../flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java | 4 ++-- .../flink/action/cdc/kafka/KafkaSyncTableActionITCase.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index 9e58d3867d2e..3201eab8e6d0 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -176,7 +176,7 @@ private void testSchemaEvolutionImpl( }, new String[] {"id", "name", "description", "weight", "address"}); - if (format.equals("debezium")) { + if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { expected = Arrays.asList( "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, NULL]", @@ -512,7 +512,7 @@ private List getPrimaryKey(String format) { } private List getBucketKey(String format) { - return format.equals("debezium") + return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) ? Collections.singletonList("id") : Collections.emptyList(); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 9a023716dcc0..1389a969ed55 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -512,7 +512,7 @@ public void testWaterMarkSyncTable(String format) throws Exception { kafkaConfig.put(TOPIC.key(), topic); Map config = getBasicTableConfig(); - if ("debezium".equals(format)) { + if (DataFormat.DEBEZIUM_JSON.asConfigString().equals(format)) { // debezium has no key // append mode never stop with compaction config.remove("bucket"); From bbac6dcf28ae8c24339c16a01eef4ce0d559e053 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Tue, 18 Jun 2024 11:49:40 +0800 Subject: [PATCH 07/12] Recovery cdc test --- .../KafkaCanalSyncDatabaseActionITCase.java | 78 +++++-------- .../KafkaCanalSyncTableActionITCase.java | 110 ++++++++---------- ...KafkaDebeziumSyncDatabaseActionITCase.java | 20 ++-- .../KafkaDebeziumSyncTableActionITCase.java | 35 +++--- .../KafkaMaxwellSyncDatabaseActionITCase.java | 20 ++-- .../KafkaMaxwellSyncTableActionITCase.java | 26 ++--- .../KafkaOggSyncDatabaseActionITCase.java | 22 ++-- .../kafka/KafkaOggSyncTableActionITCase.java | 26 ++--- .../action/cdc/kafka/KafkaSchemaITCase.java | 21 ++-- .../kafka/KafkaSyncDatabaseActionITCase.java | 29 +++-- .../cdc/kafka/KafkaSyncTableActionITCase.java | 39 +++---- .../cdc/mysql/MySqlSyncTableActionITCase.java | 15 ++- .../cdc/pulsar/PulsarActionITCaseBase.java | 8 +- .../action/cdc/pulsar/PulsarSchemaITCase.java | 7 +- .../PulsarSyncDatabaseActionITCase.java | 26 +++-- .../pulsar/PulsarSyncTableActionITCase.java | 23 ++-- .../flink/kafka/KafkaLogStoreFactoryTest.java | 27 ++++- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 4 +- .../case-insensitive/canal-data-1.txt} | 0 .../database/include/topic0/canal-data-1.txt} | 0 .../database/incomplete/canal-data-1.txt} | 0 .../prefixsuffix/topic0/canal-data-1.txt} | 0 .../prefixsuffix/topic0/canal-data-2.txt} | 0 .../prefixsuffix/topic0/canal-data-3.txt} | 0 .../prefixsuffix/topic1/canal-data-1.txt} | 0 .../prefixsuffix/topic1/canal-data-2.txt} | 0 .../prefixsuffix/topic1/canal-data-3.txt} | 0 .../schemaevolution/topic0/canal-data-1.txt} | 0 .../schemaevolution/topic0/canal-data-2.txt} | 0 .../schemaevolution/topic0/canal-data-3.txt} | 0 .../schemaevolution/topic1/canal-data-1.txt} | 0 .../schemaevolution/topic1/canal-data-2.txt} | 0 .../schemaevolution/topic1/canal-data-3.txt} | 0 .../schemaevolution/topic2/canal-data-1.txt} | 0 .../schemaevolution/topic2/canal-data-2.txt} | 0 .../schemaevolution/topic2/canal-data-3.txt} | 0 .../database/tostring/canal-data-1.txt} | 0 .../table/alltype/canal-data.txt} | 0 .../table/computedcolumn/canal-data-1.txt} | 0 .../table/computedcolumn/canal-data-2.txt} | 0 .../table/event/event-delete.txt | 0 .../table/event/event-insert.txt | 0 .../table/event/event-row.txt | 0 .../table/event/event-update.txt | 0 .../table/incomplete/canal-data-1.txt} | 0 .../table/incomplete/canal-data-2.txt} | 0 .../table/initialemptytopic/canal-data-1.txt} | 0 .../table/nononddldata/canal-data-1.txt} | 0 .../table/nonpk/canal-data-1.txt} | 0 .../table/optionschange/canal-data-1.txt} | 0 .../table/optionschange/canal-data-2.txt} | 0 .../table/schemaevolution/canal-data-1.txt} | 0 .../table/schemaevolution/canal-data-2.txt} | 0 .../table/schemaevolution/canal-data-3.txt} | 0 .../table/schemaevolution/canal-data-4.txt} | 0 .../table/schemaevolution/canal-data-5.txt} | 0 .../canal-data-1.txt} | 0 .../canal-data-2.txt} | 0 .../canal-data-3.txt} | 0 .../canal-data-4.txt} | 0 .../canal-data-5.txt} | 0 .../schemaevolutionmultiple/canal-data-1.txt} | 0 .../schemaevolutionmultiple/canal-data-2.txt} | 0 .../table/startupmode/canal-data-1.txt} | 0 .../table/startupmode/canal-data-2.txt} | 0 .../table/tostring/canal-data-1.txt} | 0 .../table/watermark/canal-data-1.txt} | 0 .../include/topic0/debezium-data-1.txt} | 0 .../prefixsuffix/topic0/debezium-data-1.txt} | 0 .../prefixsuffix/topic0/debezium-data-2.txt} | 0 .../prefixsuffix/topic1/debezium-data-1.txt} | 0 .../prefixsuffix/topic1/debezium-data-2.txt} | 0 .../topic0/debezium-data-1.txt} | 0 .../topic0/debezium-data-2.txt} | 0 .../topic1/debezium-data-1.txt} | 0 .../topic1/debezium-data-2.txt} | 0 .../table/computedcolumn/debezium-data-1.txt} | 0 .../table/nestedtype/debezium-data-1.txt} | 0 .../table/nullvalue/debezium-data-1.txt} | 0 .../table/nullvalue/debezium-data-2.txt} | 0 .../table/schema/alltype/debezium-data-1.txt} | 0 .../computedcolumn/debezium-data-1.txt} | 0 .../table/schema/include/debezium-data-1.txt} | 0 .../schemaevolution/debezium-data-1.txt} | 0 .../schemaevolution/debezium-data-2.txt} | 0 .../schemaevolution/debezium-data-3.txt} | 0 .../schemaevolution/debezium-data-4.txt} | 0 .../schemaevolution/debezium-data-1.txt} | 0 .../schemaevolution/debezium-data-2.txt} | 0 .../schemaevolution/debezium-data-3.txt} | 0 .../table/startupmode/debezium-data-1.txt} | 0 .../table/startupmode/debezium-data-2.txt} | 0 .../table/watermark/debezium-data-1.txt} | 0 .../include/topic0/maxwell-data-1.txt} | 0 .../prefixsuffix/topic0/maxwell-data-1.txt} | 0 .../prefixsuffix/topic0/maxwell-data-2.txt} | 0 .../prefixsuffix/topic1/maxwell-data-1.txt} | 0 .../prefixsuffix/topic1/maxwell-data-2.txt} | 0 .../topic0/maxwell-data-1.txt} | 0 .../topic0/maxwell-data-2.txt} | 0 .../topic1/maxwell-data-1.txt} | 0 .../topic1/maxwell-data-2.txt} | 0 .../table/computedcolumn/maxwell-data-1.txt} | 0 .../table/schemaevolution/maxwell-data-1.txt} | 0 .../table/schemaevolution/maxwell-data-2.txt} | 0 .../table/schemaevolution/maxwell-data-3.txt} | 0 .../table/schemaevolution/maxwell-data-4.txt} | 0 .../table/startupmode/maxwell-data-1.txt} | 0 .../table/startupmode/maxwell-data-2.txt} | 0 .../table/watermark/maxwell-data-1.txt} | 0 .../database/case-insensitive/ogg-data-1.txt} | 0 .../database/include/topic0/ogg-data-1.txt} | 0 .../prefixsuffix/topic0/ogg-data-1.txt} | 0 .../prefixsuffix/topic0/ogg-data-2.txt} | 0 .../prefixsuffix/topic1/ogg-data-1.txt} | 0 .../prefixsuffix/topic1/ogg-data-2.txt} | 0 .../schemaevolution/topic0/ogg-data-1.txt} | 0 .../schemaevolution/topic0/ogg-data-2.txt} | 0 .../schemaevolution/topic1/ogg-data-1.txt} | 0 .../schemaevolution/topic1/ogg-data-2.txt} | 0 .../table/computedcolumn/ogg-data-1.txt} | 0 .../table/event/event-delete.txt | 0 .../table/event/event-insert.txt | 0 .../table/event/event-update.txt | 0 .../table/schemaevolution/ogg-data-1.txt} | 0 .../table/schemaevolution/ogg-data-2.txt} | 0 .../table/schemaevolution/ogg-data-3.txt} | 0 .../table/startupmode/ogg-data-1.txt} | 0 .../table/startupmode/ogg-data-2.txt} | 0 .../table/watermark/ogg-data-1.txt} | 0 130 files changed, 253 insertions(+), 283 deletions(-) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/case-insensitive/canal-json-data-1.txt => canal/database/case-insensitive/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/include/topic0/canal-json-data-1.txt => canal/database/include/topic0/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/incomplete/canal-json-data-1.txt => canal/database/incomplete/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt => canal/database/prefixsuffix/topic0/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt => canal/database/prefixsuffix/topic0/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt => canal/database/prefixsuffix/topic0/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt => canal/database/prefixsuffix/topic1/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt => canal/database/prefixsuffix/topic1/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt => canal/database/prefixsuffix/topic1/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic0/canal-json-data-1.txt => canal/database/schemaevolution/topic0/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic0/canal-json-data-2.txt => canal/database/schemaevolution/topic0/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic0/canal-json-data-3.txt => canal/database/schemaevolution/topic0/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic1/canal-json-data-1.txt => canal/database/schemaevolution/topic1/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic1/canal-json-data-2.txt => canal/database/schemaevolution/topic1/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic1/canal-json-data-3.txt => canal/database/schemaevolution/topic1/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic2/canal-json-data-1.txt => canal/database/schemaevolution/topic2/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic2/canal-json-data-2.txt => canal/database/schemaevolution/topic2/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/schemaevolution/topic2/canal-json-data-3.txt => canal/database/schemaevolution/topic2/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/database/tostring/canal-json-data-1.txt => canal/database/tostring/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/alltype/canal-json-data.txt => canal/table/alltype/canal-data.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/computedcolumn/canal-json-data-1.txt => canal/table/computedcolumn/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/computedcolumn/canal-json-data-2.txt => canal/table/computedcolumn/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json => canal}/table/event/event-delete.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json => canal}/table/event/event-insert.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json => canal}/table/event/event-row.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json => canal}/table/event/event-update.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/incomplete/canal-json-data-1.txt => canal/table/incomplete/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/incomplete/canal-json-data-2.txt => canal/table/incomplete/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/initialemptytopic/canal-json-data-1.txt => canal/table/initialemptytopic/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/nononddldata/canal-json-data-1.txt => canal/table/nononddldata/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/nonpk/canal-json-data-1.txt => canal/table/nonpk/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/optionschange/canal-json-data-1.txt => canal/table/optionschange/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/optionschange/canal-json-data-2.txt => canal/table/optionschange/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolution/canal-json-data-1.txt => canal/table/schemaevolution/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolution/canal-json-data-2.txt => canal/table/schemaevolution/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolution/canal-json-data-3.txt => canal/table/schemaevolution/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolution/canal-json-data-4.txt => canal/table/schemaevolution/canal-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolution/canal-json-data-5.txt => canal/table/schemaevolution/canal-data-5.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt => canal/table/schemaevolutionmissingddl/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt => canal/table/schemaevolutionmissingddl/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt => canal/table/schemaevolutionmissingddl/canal-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt => canal/table/schemaevolutionmissingddl/canal-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt => canal/table/schemaevolutionmissingddl/canal-data-5.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt => canal/table/schemaevolutionmultiple/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt => canal/table/schemaevolutionmultiple/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/startupmode/canal-json-data-1.txt => canal/table/startupmode/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/startupmode/canal-json-data-2.txt => canal/table/startupmode/canal-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/tostring/canal-json-data-1.txt => canal/table/tostring/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{canal-json/table/watermark/canal-json-data-1.txt => canal/table/watermark/canal-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/include/topic0/debezium-json-data-1.txt => debezium/database/include/topic0/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt => debezium/database/prefixsuffix/topic0/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt => debezium/database/prefixsuffix/topic0/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt => debezium/database/prefixsuffix/topic1/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt => debezium/database/prefixsuffix/topic1/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt => debezium/database/schemaevolution/topic0/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt => debezium/database/schemaevolution/topic0/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt => debezium/database/schemaevolution/topic1/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt => debezium/database/schemaevolution/topic1/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/computedcolumn/debezium-json-data-1.txt => debezium/table/computedcolumn/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/nestedtype/debezium-json-data-1.txt => debezium/table/nestedtype/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/nullvalue/debezium-json-data-1.txt => debezium/table/nullvalue/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/nullvalue/debezium-json-data-2.txt => debezium/table/nullvalue/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/alltype/debezium-json-data-1.txt => debezium/table/schema/alltype/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt => debezium/table/schema/computedcolumn/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/include/debezium-json-data-1.txt => debezium/table/schema/include/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt => debezium/table/schema/schemaevolution/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt => debezium/table/schema/schemaevolution/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt => debezium/table/schema/schemaevolution/debezium-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt => debezium/table/schema/schemaevolution/debezium-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schemaevolution/debezium-json-data-1.txt => debezium/table/schemaevolution/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schemaevolution/debezium-json-data-2.txt => debezium/table/schemaevolution/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/schemaevolution/debezium-json-data-3.txt => debezium/table/schemaevolution/debezium-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/startupmode/debezium-json-data-1.txt => debezium/table/startupmode/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/startupmode/debezium-json-data-2.txt => debezium/table/startupmode/debezium-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{debezium-json/table/watermark/debezium-json-data-1.txt => debezium/table/watermark/debezium-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/include/topic0/maxwell-json-data-1.txt => maxwell/database/include/topic0/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt => maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt => maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt => maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt => maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt => maxwell/database/schemaevolution/topic0/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt => maxwell/database/schemaevolution/topic0/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt => maxwell/database/schemaevolution/topic1/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt => maxwell/database/schemaevolution/topic1/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/computedcolumn/maxwell-json-data-1.txt => maxwell/table/computedcolumn/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/schemaevolution/maxwell-json-data-1.txt => maxwell/table/schemaevolution/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/schemaevolution/maxwell-json-data-2.txt => maxwell/table/schemaevolution/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/schemaevolution/maxwell-json-data-3.txt => maxwell/table/schemaevolution/maxwell-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/schemaevolution/maxwell-json-data-4.txt => maxwell/table/schemaevolution/maxwell-data-4.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/startupmode/maxwell-json-data-1.txt => maxwell/table/startupmode/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/startupmode/maxwell-json-data-2.txt => maxwell/table/startupmode/maxwell-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{maxwell-json/table/watermark/maxwell-json-data-1.txt => maxwell/table/watermark/maxwell-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/case-insensitive/ogg-json-data-1.txt => ogg/database/case-insensitive/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/include/topic0/ogg-json-data-1.txt => ogg/database/include/topic0/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt => ogg/database/prefixsuffix/topic0/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt => ogg/database/prefixsuffix/topic0/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt => ogg/database/prefixsuffix/topic1/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt => ogg/database/prefixsuffix/topic1/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt => ogg/database/schemaevolution/topic0/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt => ogg/database/schemaevolution/topic0/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt => ogg/database/schemaevolution/topic1/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt => ogg/database/schemaevolution/topic1/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/computedcolumn/ogg-json-data-1.txt => ogg/table/computedcolumn/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json => ogg}/table/event/event-delete.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json => ogg}/table/event/event-insert.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json => ogg}/table/event/event-update.txt (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/schemaevolution/ogg-json-data-1.txt => ogg/table/schemaevolution/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/schemaevolution/ogg-json-data-2.txt => ogg/table/schemaevolution/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/schemaevolution/ogg-json-data-3.txt => ogg/table/schemaevolution/ogg-data-3.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/startupmode/ogg-json-data-1.txt => ogg/table/startupmode/ogg-data-1.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/startupmode/ogg-json-data-2.txt => ogg/table/startupmode/ogg-data-2.txt} (100%) rename paimon-flink/paimon-flink-cdc/src/test/resources/kafka/{ogg-json/table/watermark/ogg-json-data-1.txt => ogg/table/watermark/ogg-data-1.txt} (100%) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java index 4b180cf27984..9ed85c71d7de 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.catalog.FileSystemCatalogOptions; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -47,7 +46,6 @@ /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaCanalSyncDatabaseActionITCase extends KafkaActionITCaseBase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) @@ -63,14 +61,12 @@ public void testSchemaEvolutionMultiTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( topics.get(i), - "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/schemaevolution/topic%s/canal-data-1.txt", + i); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); if (ThreadLocalRandom.current().nextBoolean()) { kafkaConfig.put(TOPIC.key(), String.join(";", topics)); } else { @@ -98,14 +94,12 @@ public void testSchemaEvolutionOneTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( topics.get(0), - "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/schemaevolution/topic%s/canal-data-1.txt", + i); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = @@ -147,10 +141,8 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/%s/database/schemaevolution/topic%s/%s-data-2.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/schemaevolution/topic%s/canal-data-2.txt", + i); } rowType1 = @@ -188,10 +180,8 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/%s/database/schemaevolution/topic%s/%s-data-3.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/schemaevolution/topic%s/canal-data-3.txt", + i); } rowType1 = @@ -232,7 +222,7 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int @Test public void testTopicIsEmpty() { Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); @@ -266,16 +256,12 @@ public void testTableAffixMultiTopic() throws Exception { for (int i = 0; i < topics.size(); i++) { writeRecordsToKafka( - topics.get(i), - "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", - FORMAT, - i, - FORMAT); + topics.get(i), "kafka/canal/database/prefixsuffix/topic%s/canal-data-1.txt", i); } // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -312,16 +298,12 @@ public void testTableAffixOneTopic() throws Exception { for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( - topics.get(0), - "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", - FORMAT, - i, - FORMAT); + topics.get(0), "kafka/canal/database/prefixsuffix/topic%s/canal-data-1.txt", i); } // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -362,10 +344,8 @@ private void testTableAffixImpl(List topics, boolean writeOne, int fileC for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/%s/database/prefixsuffix/topic%s/%s-data-2.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/prefixsuffix/topic%s/canal-data-2.txt", + i); } rowType1 = RowType.of( @@ -398,10 +378,8 @@ private void testTableAffixImpl(List topics, boolean writeOne, int fileC for (int i = 0; i < fileCount; i++) { writeRecordsToKafka( writeOne ? topics.get(0) : topics.get(i), - "kafka/%s/database/prefixsuffix/topic%s/%s-data-3.txt", - FORMAT, - i, - FORMAT); + "kafka/canal/database/prefixsuffix/topic%s/canal-data-3.txt", + i); } rowType1 = @@ -474,12 +452,11 @@ private void includingAndExcludingTablesImpl( final String topic1 = "include_exclude" + UUID.randomUUID(); List topics = Collections.singletonList(topic1); topics.forEach(topic -> createTestTopic(topic, 1, 1)); - writeRecordsToKafka( - topics.get(0), "kafka/%s/database/include/topic0/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topics.get(0), "kafka/canal/database/include/topic0/canal-data-1.txt"); // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -499,10 +476,10 @@ private void includingAndExcludingTablesImpl( public void testTypeMappingToString() throws Exception { final String topic = "map-to-string"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/database/tostring/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/database/tostring/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = @@ -545,11 +522,10 @@ public void testCatalogAndTableConfig() { public void testCaseInsensitive() throws Exception { final String topic = "case-insensitive"; createTestTopic(topic, 1, 1); - writeRecordsToKafka( - topic, "kafka/%s/database/case-insensitive/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/database/case-insensitive/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = @@ -581,10 +557,10 @@ public void testCaseInsensitive() throws Exception { public void testCannotSynchronizeIncompleteJson() throws Exception { final String topic = "incomplete"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/database/incomplete/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/database/incomplete/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index 9594a7faedef..8a4dc2f3035b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -20,7 +20,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.FileSystemCatalogOptions; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -58,7 +57,7 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaCanalSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); + private static final String CANAL = "canal"; @Test @Timeout(60) @@ -75,10 +74,10 @@ public void testSchemaEvolutionWithMissingDdl() throws Exception { private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { final String topic = "schema_evolution"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-1.txt", FORMAT, sourceDir, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-1.txt", sourceDir); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -106,7 +105,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce List expected = Arrays.asList("+I[1, 1, one]", "+I[1, 2, two]", "+I[2, 4, four]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-2.txt", FORMAT, sourceDir, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-2.txt", sourceDir); rowType = RowType.of( @@ -127,7 +126,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[1, 6, six, 60]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-3.txt", FORMAT, sourceDir, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-3.txt", sourceDir); rowType = RowType.of( @@ -149,7 +148,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[2, 8, eight, 80000000000]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-4.txt", FORMAT, sourceDir, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-4.txt", sourceDir); rowType = RowType.of( @@ -175,7 +174,7 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110], 9.9]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-5.txt", FORMAT, sourceDir, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/%s/canal-data-5.txt", sourceDir); rowType = RowType.of( @@ -207,11 +206,10 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce public void testMultipleSchemaEvolutions() throws Exception { final String topic = "schema_evolution_multiple"; createTestTopic(topic, 1, 1); - writeRecordsToKafka( - topic, "kafka/%s/table/schemaevolutionmultiple/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); if (ThreadLocalRandom.current().nextBoolean()) { kafkaConfig.put(TOPIC.key(), topic); @@ -241,8 +239,7 @@ private void testSchemaEvolutionMultipleImpl(String topic) throws Exception { List expected = Collections.singletonList("+I[1, one, 10, string_1]"); waitForResult(expected, table, rowType, primaryKeys); - writeRecordsToKafka( - topic, "kafka/%s/table/schemaevolutionmultiple/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt"); rowType = RowType.of( @@ -278,10 +275,10 @@ public void testAllTypes() throws Exception { private void testAllTypesOnce() throws Exception { final String topic = "all_type" + UUID.randomUUID(); createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/alltype/%s-data.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/alltype/canal-data.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -538,7 +535,7 @@ private void testAllTypesImpl() throws Exception { public void testNotSupportFormat() throws Exception { final String topic = "not_support"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); kafkaConfig.put(VALUE_FORMAT.key(), "togg-json"); @@ -561,10 +558,10 @@ public void testNotSupportFormat() throws Exception { public void testKafkaNoNonDdlData() throws Exception { final String topic = "no_non_ddl_data"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/nononddldata/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/nononddldata/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -584,10 +581,10 @@ public void testKafkaNoNonDdlData() throws Exception { public void testAssertSchemaCompatible() throws Exception { final String topic = "assert_schema_compatible"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); // create an incompatible table @@ -621,10 +618,10 @@ public void testAssertSchemaCompatible() throws Exception { public void testStarUpOptionSpecific() throws Exception { final String topic = "start_up_specific"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); @@ -657,11 +654,10 @@ public void testStarUpOptionSpecific() throws Exception { public void testStarUpOptionLatest() throws Exception { final String topic = "start_up_latest"; createTestTopic(topic, 1, 1); - writeRecordsToKafka( - topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, true, "kafka/canal/table/startupmode/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -675,7 +671,7 @@ public void testStarUpOptionLatest() throws Exception { // wait task running to commit LATEST_OFFSET Thread.sleep(5_000); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); FileStoreTable table = getFileStoreTable(tableName); @@ -698,11 +694,10 @@ public void testStarUpOptionLatest() throws Exception { public void testStarUpOptionTimestamp() throws Exception { final String topic = "start_up_timestamp"; createTestTopic(topic, 1, 1); - writeRecordsToKafka( - topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, true, "kafka/canal/table/startupmode/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); kafkaConfig.put( @@ -715,7 +710,7 @@ public void testStarUpOptionTimestamp() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); FileStoreTable table = getFileStoreTable(tableName); @@ -738,10 +733,10 @@ public void testStarUpOptionTimestamp() throws Exception { public void testStarUpOptionEarliest() throws Exception { final String topic = "start_up_earliest"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -752,7 +747,7 @@ public void testStarUpOptionEarliest() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); FileStoreTable table = getFileStoreTable(tableName); @@ -777,10 +772,10 @@ public void testStarUpOptionEarliest() throws Exception { public void testStarUpOptionGroup() throws Exception { final String topic = "start_up_group"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); KafkaSyncTableAction action = @@ -791,7 +786,7 @@ public void testStarUpOptionGroup() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/startupmode/canal-data-2.txt"); FileStoreTable table = getFileStoreTable(tableName); @@ -816,10 +811,10 @@ public void testStarUpOptionGroup() throws Exception { public void testComputedColumn() throws Exception { String topic = "computed_column"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/computedcolumn/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -848,10 +843,10 @@ public void testComputedColumn() throws Exception { public void testTypeMappingToString() throws Exception { final String topic = "map-to-string"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/tostring/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/tostring/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -893,10 +888,10 @@ public void testCatalogAndTableConfig() { public void testCDCOperations(boolean ignoreDelete) throws Exception { final String topic = "event-insert" + UUID.randomUUID(); createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/event/event-row.txt", FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/event/event-row.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = getBasicTableConfig(); @@ -927,7 +922,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110], 9.9]"); waitForResult(expectedRow, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/event/event-insert.txt", FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/event/event-insert.txt"); // For the INSERT operation List expectedInsert = @@ -938,7 +933,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[2, 4, four, NULL, NULL, NULL, NULL]"); waitForResult(expectedInsert, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/event/event-update.txt", FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/event/event-update.txt"); // For the UPDATE operation List expectedUpdate = @@ -949,7 +944,7 @@ public void testCDCOperations(boolean ignoreDelete) throws Exception { "+I[2, 4, four, NULL, NULL, NULL, NULL]"); waitForResult(expectedUpdate, table, rowType, primaryKeys); - writeRecordsToKafka(topic, "kafka/%s/table/event/event-delete.txt", FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/event/event-delete.txt"); // For the DELETE operation List expectedDelete = @@ -988,8 +983,7 @@ public void testSyncWithInitialEmptyTopic() throws Exception { .build(); runActionWithDefaultEnv(action); - writeRecordsToKafka( - topic, "kafka/%s/table/initialemptytopic/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/initialemptytopic/canal-data-1.txt"); RowType rowType = RowType.of( @@ -1012,10 +1006,10 @@ public void testSyncWithInitialEmptyTopic() throws Exception { public void testSynchronizeIncompleteJson() throws Exception { String topic = "incomplete"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/incomplete/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/incomplete/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -1043,10 +1037,10 @@ public void testSynchronizeIncompleteJson() throws Exception { public void testSynchronizeNonPkTable() throws Exception { String topic = "non_pk"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/nonpk/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/nonpk/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = getBasicTableConfig(); tableConfig.remove("bucket"); @@ -1073,10 +1067,10 @@ public void testSynchronizeNonPkTable() throws Exception { public void testMissingDecimalPrecision() throws Exception { String topic = "missing-decimal-precision"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/incomplete/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/incomplete/canal-data-2.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = @@ -1116,12 +1110,11 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval Collections.emptyList(), Collections.emptyMap()); } else { - writeRecordsToKafka( - topic, "kafka/%s/table/computedcolumn/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-2.txt"); } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -1134,8 +1127,7 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval runActionWithDefaultEnv(action); if (triggerSchemaRetrievalException) { - writeRecordsToKafka( - topic, "kafka/%s/table/computedcolumn/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/computedcolumn/canal-data-2.txt"); } RowType rowType = @@ -1154,6 +1146,6 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(FORMAT); + testWaterMarkSyncTable(CANAL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java index 3a8bd5dc6c6a..3520f861da34 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java @@ -18,67 +18,65 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; - import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaDebeziumSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = DataFormat.DEBEZIUM_JSON.asConfigString(); + private static final String DEBEZIUM = "debezium"; @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(FORMAT); + testSchemaEvolutionMultiTopic(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(FORMAT); + testSchemaEvolutionOneTopic(DEBEZIUM); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(FORMAT); + testTopicIsEmpty(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(FORMAT); + testTableAffixMultiTopic(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(FORMAT); + testTableAffixOneTopic(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(FORMAT); + testIncludingTables(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(FORMAT); + testExcludingTables(DEBEZIUM); } @Disabled // TODO How to append table in combined mode? @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(FORMAT); + testIncludingAndExcludingTables(DEBEZIUM); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index ccea411591ff..04dfb3769a61 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -37,84 +36,84 @@ /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaDebeziumSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = DataFormat.DEBEZIUM_JSON.asConfigString(); + private static final String DEBEZIUM = "debezium"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", FORMAT); + runSingleTableSchemaEvolution("schemaevolution", DEBEZIUM); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(FORMAT); + testNotSupportFormat(DEBEZIUM); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(FORMAT); + testAssertSchemaCompatible(DEBEZIUM); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(FORMAT); + testStarUpOptionSpecific(DEBEZIUM); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(FORMAT); + testStarUpOptionLatest(DEBEZIUM); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(FORMAT); + testStarUpOptionTimestamp(DEBEZIUM); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(FORMAT); + testStarUpOptionEarliest(DEBEZIUM); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(FORMAT); + testStarUpOptionGroup(DEBEZIUM); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(FORMAT); + testComputedColumn(DEBEZIUM); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(FORMAT); + testWaterMarkSyncTable(DEBEZIUM); } @Test @Timeout(60) public void testKafkaBuildSchemaWithDelete() throws Exception { - testKafkaBuildSchemaWithDelete(FORMAT); + testKafkaBuildSchemaWithDelete(DEBEZIUM); } @Test @Timeout(60) public void testSchemaIncludeRecord1() throws Exception { - testSchemaIncludeRecord(FORMAT); + testSchemaIncludeRecord(DEBEZIUM); } @Test @Timeout(60) public void testAllTypesWithSchema() throws Exception { - testAllTypesWithSchemaImpl(FORMAT); + testAllTypesWithSchemaImpl(DEBEZIUM); } @Test @@ -123,13 +122,13 @@ public void testMessageWithNullValue() throws Exception { final String topic = "test_null_value"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/nullvalue/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/debezium/table/nullvalue/debezium-data-1.txt"); // write null value kafkaProducer.send(new ProducerRecord<>(topic, null)); - writeRecordsToKafka(topic, "kafka/%s/table/nullvalue/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/debezium/table/nullvalue/debezium-data-2.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java index c1d514eaa59d..189863f44c0d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java @@ -18,60 +18,58 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaMaxwellSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = DataFormat.MAXWELL_JSON.asConfigString(); + private static final String MAXWELL = "maxwell"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(FORMAT); + testSchemaEvolutionMultiTopic(MAXWELL); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(FORMAT); + testSchemaEvolutionOneTopic(MAXWELL); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(FORMAT); + testTopicIsEmpty(MAXWELL); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(FORMAT); + testTableAffixMultiTopic(MAXWELL); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(FORMAT); + testTableAffixOneTopic(MAXWELL); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(FORMAT); + testIncludingTables(MAXWELL); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(FORMAT); + testExcludingTables(MAXWELL); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(FORMAT); + testIncludingAndExcludingTables(MAXWELL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java index d41fba70df60..f15c06c5eb87 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java @@ -18,79 +18,77 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaMaxwellSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = DataFormat.MAXWELL_JSON.asConfigString(); + private static final String MAXWELL = "maxwell"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", FORMAT); + runSingleTableSchemaEvolution("schemaevolution", MAXWELL); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(FORMAT); + testNotSupportFormat(MAXWELL); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(FORMAT); + testAssertSchemaCompatible(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(FORMAT); + testStarUpOptionSpecific(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(FORMAT); + testStarUpOptionLatest(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(FORMAT); + testStarUpOptionTimestamp(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(FORMAT); + testStarUpOptionEarliest(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(FORMAT); + testStarUpOptionGroup(MAXWELL); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(FORMAT); + testComputedColumn(MAXWELL); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(FORMAT); + testWaterMarkSyncTable(MAXWELL); } @Test @Timeout(60) public void testFieldValNullSyncTable() throws Exception { - testTableFiledValNull(FORMAT); + testTableFiledValNull(MAXWELL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java index ab132c366d64..9eacbfea2874 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java @@ -18,65 +18,63 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncDatabaseAction}. */ public class KafkaOggSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { - private static final String FORMAT = DataFormat.OGG_JSON.asConfigString(); + private static final String OGG = "ogg"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - testSchemaEvolutionMultiTopic(FORMAT); + testSchemaEvolutionMultiTopic(OGG); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - testSchemaEvolutionOneTopic(FORMAT); + testSchemaEvolutionOneTopic(OGG); } @Test public void testTopicIsEmpty() { - testTopicIsEmpty(FORMAT); + testTopicIsEmpty(OGG); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - testTableAffixMultiTopic(FORMAT); + testTableAffixMultiTopic(OGG); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - testTableAffixOneTopic(FORMAT); + testTableAffixOneTopic(OGG); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - testIncludingTables(FORMAT); + testIncludingTables(OGG); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - testExcludingTables(FORMAT); + testExcludingTables(OGG); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - testIncludingAndExcludingTables(FORMAT); + testIncludingAndExcludingTables(OGG); } @Test @Timeout(60) public void testCaseInsensitive() throws Exception { - testCaseInsensitive(FORMAT); + testCaseInsensitive(OGG); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java index 5f368236e6a9..6394b5b71978 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java @@ -18,79 +18,77 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.flink.action.cdc.format.DataFormat; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaOggSyncTableActionITCase extends KafkaSyncTableActionITCase { - private static final String FORMAT = DataFormat.OGG_JSON.asConfigString(); + private static final String OGG = "ogg"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution", FORMAT); + runSingleTableSchemaEvolution("schemaevolution", OGG); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - testNotSupportFormat(FORMAT); + testNotSupportFormat(OGG); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - testAssertSchemaCompatible(FORMAT); + testAssertSchemaCompatible(OGG); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - testStarUpOptionSpecific(FORMAT); + testStarUpOptionSpecific(OGG); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - testStarUpOptionLatest(FORMAT); + testStarUpOptionLatest(OGG); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - testStarUpOptionTimestamp(FORMAT); + testStarUpOptionTimestamp(OGG); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - testStarUpOptionEarliest(FORMAT); + testStarUpOptionEarliest(OGG); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - testStarUpOptionGroup(FORMAT); + testStarUpOptionGroup(OGG); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - testComputedColumn(FORMAT); + testComputedColumn(OGG); } @Test @Timeout(60) public void testCDCOperations() throws Exception { - testCDCOperations(FORMAT); + testCDCOperations(OGG); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - testWaterMarkSyncTable(FORMAT); + testWaterMarkSyncTable(OGG); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java index 49d28774f045..84bb802bc195 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java @@ -20,7 +20,6 @@ import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataField; @@ -44,17 +43,15 @@ /** Tests for building schema from Kafka. */ public class KafkaSchemaITCase extends KafkaActionITCaseBase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); - @Test @Timeout(60) public void testKafkaSchema() throws Exception { final String topic = "test_kafka_schema"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/schemaevolution/canal-data-1.txt"); Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); - kafkaConfig.setString(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.setString(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.setString(TOPIC.key(), topic); Schema kafkaSchema = @@ -75,10 +72,10 @@ kafkaConfig, new KafkaDebeziumJsonDeserializationSchema()), public void testTableOptionsChange() throws Exception { final String topic = "test_table_options_change"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/optionschange/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/optionschange/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = new HashMap<>(); tableConfig.put("bucket", "1"); @@ -91,7 +88,7 @@ public void testTableOptionsChange() throws Exception { waitingTables(tableName); jobClient.cancel(); - writeRecordsToKafka(topic, "kafka/%s/table/optionschange/%s-data-2.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/table/optionschange/canal-data-2.txt"); tableConfig.put("sink.savepoint.auto-tag", "true"); tableConfig.put("tag.num-retained-max", "5"); @@ -118,11 +115,10 @@ public void testTableOptionsChange() throws Exception { public void testNewlyAddedTablesOptionsChange() throws Exception { final String topic = "test_database_options_change"; createTestTopic(topic, 1, 1); - writeRecordsToKafka( - topic, "kafka/%s/database/schemaevolution/topic0/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/database/schemaevolution/topic0/canal-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), FORMAT); + kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); kafkaConfig.put(TOPIC.key(), topic); Map tableConfig = new HashMap<>(); tableConfig.put("bucket", "1"); @@ -144,8 +140,7 @@ public void testNewlyAddedTablesOptionsChange() throws Exception { tableConfig.put("snapshot.num-retained.max", "10"); tableConfig.put("changelog-producer", "input"); - writeRecordsToKafka( - topic, "kafka/%s/database/schemaevolution/topic1/%s-data-1.txt", FORMAT, FORMAT); + writeRecordsToKafka(topic, "kafka/canal/database/schemaevolution/topic1/canal-data-1.txt"); KafkaSyncDatabaseAction action2 = syncDatabaseActionBuilder(kafkaConfig).withTableConfig(tableConfig).build(); runActionWithDefaultEnv(action2); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index 3201eab8e6d0..de189bc20536 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.catalog.FileSystemCatalogOptions; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -60,7 +59,7 @@ protected void testSchemaEvolutionMultiTopic(String format) throws Exception { } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -88,7 +87,7 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception { } Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -176,7 +175,7 @@ private void testSchemaEvolutionImpl( }, new String[] {"id", "name", "description", "weight", "address"}); - if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { + if (format.equals("debezium")) { expected = Arrays.asList( "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, NULL]", @@ -195,7 +194,7 @@ private void testSchemaEvolutionImpl( protected void testTopicIsEmpty(String format) { Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); @@ -241,7 +240,7 @@ protected void testTableAffixMultiTopic(String format) throws Exception { // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -289,7 +288,7 @@ protected void testTableAffixOneTopic(String format) throws Exception { // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -361,7 +360,7 @@ private void testTableAffixImpl( DataTypes.STRING() }, new String[] {"id", "name", "description", "weight", "address"}); - if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { + if (format.equals("debezium")) { expected = Arrays.asList( "+I[101, scooter, Small 2-wheel scooter, 3.14, Beijing]", @@ -386,7 +385,7 @@ private void testTableAffixImpl( DataTypes.STRING() }, new String[] {"id", "name", "description", "weight", "age"}); - if (format.equals(DataFormat.DEBEZIUM_JSON.asConfigString())) { + if (format.equals("debezium")) { expected = Arrays.asList( "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", @@ -445,7 +444,7 @@ private void includingAndExcludingTablesImpl( // try synchronization Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) @@ -468,7 +467,7 @@ protected void testCaseInsensitive(String format) throws Exception { topic, "kafka/%s/database/case-insensitive/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncDatabaseAction action = @@ -500,19 +499,17 @@ protected void testCaseInsensitive(String format) throws Exception { } private DataType getDataType(String format) { - return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) - ? DataTypes.STRING() - : DataTypes.STRING().notNull(); + return format.equals("debezium") ? DataTypes.STRING() : DataTypes.STRING().notNull(); } private List getPrimaryKey(String format) { - return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) + return format.equals("debezium") ? Collections.emptyList() : Collections.singletonList("id"); } private List getBucketKey(String format) { - return format.equals(DataFormat.DEBEZIUM_JSON.asConfigString()) + return format.equals("debezium") ? Collections.singletonList("id") : Collections.emptyList(); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 1389a969ed55..117994835600 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -21,7 +21,6 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataField; @@ -62,7 +61,7 @@ protected void runSingleTableSchemaEvolution(String sourceDir, String format) th writeRecordsToKafka(topic, "kafka/%s/table/%s/%s-data-1.txt", format, sourceDir, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -164,7 +163,7 @@ protected void testAssertSchemaCompatible(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); // create an incompatible table @@ -198,7 +197,7 @@ protected void testStarUpOptionSpecific(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); @@ -234,7 +233,7 @@ protected void testStarUpOptionLatest(String format) throws Exception { topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -274,7 +273,7 @@ public void testStarUpOptionTimestamp(String format) throws Exception { topic, true, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); kafkaConfig.put( @@ -314,7 +313,7 @@ public void testStarUpOptionEarliest(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = @@ -354,7 +353,7 @@ public void testStarUpOptionGroup(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/startupmode/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); KafkaSyncTableAction action = @@ -394,7 +393,7 @@ public void testComputedColumn(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/computedcolumn/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -426,7 +425,7 @@ protected void testCDCOperations(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/event/event-insert.txt", format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -482,15 +481,13 @@ public void testKafkaBuildSchemaWithDelete(String format) throws Exception { topic, "kafka/%s/table/schema/schemaevolution/%s-data-4.txt", format, format); Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); - kafkaConfig.setString(VALUE_FORMAT.key(), format); + kafkaConfig.setString(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.setString(TOPIC.key(), topic); - DataFormat dataFormat = DataFormat.fromConfigString(format); - Schema kafkaSchema = MessageQueueSchemaUtils.getSchema( getKafkaEarliestConsumer( - kafkaConfig, dataFormat.createKafkaDeserializer(kafkaConfig)), + kafkaConfig, new KafkaDebeziumJsonDeserializationSchema()), getDataFormat(kafkaConfig), TypeMapping.defaultMapping()); List fields = new ArrayList<>(); @@ -508,11 +505,11 @@ public void testWaterMarkSyncTable(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/watermark/%s-data-1.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); Map config = getBasicTableConfig(); - if (DataFormat.DEBEZIUM_JSON.asConfigString().equals(format)) { + if ("debezium".equals(format)) { // debezium has no key // append mode never stop with compaction config.remove("bucket"); @@ -543,10 +540,10 @@ public void testWaterMarkSyncTable(String format) throws Exception { public void testSchemaIncludeRecord(String format) throws Exception { String topic = "schema_include"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/schema/include/%s-data-1.txt", format, format); + writeRecordsToKafka(topic, "kafka/debezium/table/schema/include/debezium-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -577,10 +574,10 @@ public void testSchemaIncludeRecord(String format) throws Exception { public void testAllTypesWithSchemaImpl(String format) throws Exception { String topic = "schema_include_all_type"; createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, "kafka/%s/table/schema/alltype/%s-data-1.txt", format, format); + writeRecordsToKafka(topic, "kafka/debezium/table/schema/alltype/debezium-data-1.txt"); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) @@ -822,7 +819,7 @@ protected void testTableFiledValNull(String format) throws Exception { writeRecordsToKafka(topic, "kafka/%s/table/schemaevolution/%s-data-4.txt", format, format); Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), format); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); KafkaSyncTableAction action = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index fb78d16eb4b2..db8e1744b35b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -297,11 +297,11 @@ private void testSchemaEvolutionMultipleImpl(Statement statement) throws Excepti + "MODIFY COLUMN v1 VARCHAR(20), " // I'd love to change COMMENT to DEFAULT // however debezium parser seems to have a bug here - + "ADD COLUMN (v5 DOUBLE, v6 DECIMAL(5, 3), `$% ^,& *(` VARCHAR(10) COMMENT 'Hi, v700 DOUBLE \\', v701 INT a test'), " + + "ADD COLUMN (v5 DOUBLE, v6 DECIMAL(5, 3), `$% ^,& *(` VARCHAR(10), v7 INTEGER COMMENT 'Hi, v700 DOUBLE \\', v701 INT a test'), " + "MODIFY v2 BIGINT"); statement.executeUpdate( "INSERT INTO schema_evolution_multiple VALUES " - + "(2, 'long_string_two', 2000000000000, 'string_2', 20, 20.5, 20.002, 'test_2')"); + + "(2, 'long_string_two', 2000000000000, 'string_2', 20, 20.5, 20.002, 'test_2', 200)"); rowType = RowType.of( new DataType[] { @@ -312,13 +312,16 @@ private void testSchemaEvolutionMultipleImpl(Statement statement) throws Excepti DataTypes.INT(), DataTypes.DOUBLE(), DataTypes.DECIMAL(5, 3), - DataTypes.VARCHAR(10) + DataTypes.VARCHAR(10), + DataTypes.INT(), }, - new String[] {"_id", "v1", "v2", "v3", "v4", "v5", "v6", "$% ^,& *("}); + new String[] { + "_id", "v1", "v2", "v3", "v4", "v5", "v6", "$% ^,& *(", "v7" + }); expected = Arrays.asList( - "+I[1, one, 10, string_1, NULL, NULL, NULL, NULL]", - "+I[2, long_string_two, 2000000000000, string_2, 20, 20.5, 20.002, test_2]"); + "+I[1, one, 10, string_1, NULL, NULL, NULL, NULL, NULL]", + "+I[2, long_string_two, 2000000000000, string_2, 20, 20.5, 20.002, test_2, 200]"); waitForResult(expected, table, rowType, primaryKeys); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java index d139ef2aa6b7..969a9416f6c1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarActionITCaseBase.java @@ -203,12 +203,8 @@ private void deleteTopics() throws Exception { } } - protected List getMessages(String resourceDirFormat, Object... args) - throws IOException { - URL url = - PulsarActionITCaseBase.class - .getClassLoader() - .getResource(String.format(resourceDirFormat, args)); + protected List getMessages(String resource) throws IOException { + URL url = PulsarActionITCaseBase.class.getClassLoader().getResource(resource); assertThat(url).isNotNull(); java.nio.file.Path path = new File(url.getFile()).toPath(); List lines = Files.readAllLines(path); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java index fe8ea0a818b4..255d43bfbc81 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSchemaITCase.java @@ -20,7 +20,6 @@ import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.flink.action.cdc.serialization.CdcJsonDeserializationSchema; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; @@ -41,7 +40,6 @@ /** Tests for building schema from Pulsar. */ public class PulsarSchemaITCase extends PulsarActionITCaseBase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) @@ -50,13 +48,12 @@ public void testPulsarSchema() throws Exception { createTopic(topic); // ---------- Write the Canal json into pulsar ------------------- - List messages = - getMessages("kafka/%s/table/schemaevolution/%s-data-1.txt", FORMAT, FORMAT); + List messages = getMessages("kafka/canal/table/schemaevolution/canal-data-1.txt"); sendMessages(topic, messages); Configuration pulsarConfig = Configuration.fromMap(getBasicPulsarConfig()); pulsarConfig.setString(TOPIC.key(), topic); - pulsarConfig.set(VALUE_FORMAT, FORMAT); + pulsarConfig.set(VALUE_FORMAT, "canal-json"); Schema pulsarSchema = MessageQueueSchemaUtils.getSchema( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java index 7b569a261b25..37e6b66bf554 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionITCase.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action.cdc.pulsar; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -40,7 +39,6 @@ /** IT cases for {@link PulsarSyncDatabaseAction}. */ public class PulsarSyncDatabaseActionITCase extends PulsarActionITCaseBase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(60) @@ -60,8 +58,9 @@ public void testSchemaEvolutionMultiTopic() throws Exception { sendMessages( topics.get(i), getMessages( - "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", - FORMAT, i, FORMAT)); + "kafka/canal/database/schemaevolution/topic" + + i + + "/canal-data-1.txt")); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -69,7 +68,7 @@ public void testSchemaEvolutionMultiTopic() throws Exception { Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); - pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); + pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); if (ThreadLocalRandom.current().nextBoolean()) { pulsarConfig.put(TOPIC.key(), String.join(";", topics)); } else { @@ -101,8 +100,9 @@ public void testSchemaEvolutionOneTopic() throws Exception { sendMessages( topics.get(0), getMessages( - "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", - FORMAT, i, FORMAT)); + "kafka/canal/database/schemaevolution/topic" + + i + + "/canal-data-1.txt")); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -110,7 +110,7 @@ public void testSchemaEvolutionOneTopic() throws Exception { Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); - pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); + pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); pulsarConfig.put(TOPIC.key(), String.join(";", topics)); PulsarSyncDatabaseAction action = @@ -154,8 +154,9 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int sendMessages( writeOne ? topics.get(0) : topics.get(i), getMessages( - "kafka/%s/database/schemaevolution/topic%s/%s-data-2.txt", - FORMAT, i, FORMAT)); + "kafka/canal/database/schemaevolution/topic" + + i + + "/canal-data-2.txt")); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } @@ -198,8 +199,9 @@ private void testSchemaEvolutionImpl(List topics, boolean writeOne, int sendMessages( writeOne ? topics.get(0) : topics.get(i), getMessages( - "kafka/%s/database/schemaevolution/topic%s/%s-data-3.txt", - FORMAT, i, FORMAT)); + "kafka/canal/database/schemaevolution/topic" + + i + + "/canal-data-3.txt")); } catch (Exception e) { throw new Exception("Failed to write canal data to Pulsar.", e); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java index 79a4b6c64efd..a4fb332d5033 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.pulsar; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.flink.action.cdc.format.DataFormat; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -41,7 +40,6 @@ /** IT cases for {@link PulsarSyncTableAction}. */ public class PulsarSyncTableActionITCase extends PulsarActionITCaseBase { - private static final String FORMAT = DataFormat.CANAL_JSON.asConfigString(); @Test @Timeout(120) @@ -55,7 +53,8 @@ private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { createTopic(topic, 1); // ---------- Write the Canal json into Pulsar ------------------- sendMessages( - topic, getMessages("kafka/%s/table/%s/%s-data-1.txt", FORMAT, sourceDir, FORMAT)); + topic, + getMessages(String.format("kafka/canal/table/%s/canal-data-1.txt", sourceDir))); Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); @@ -64,7 +63,7 @@ private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { } else { pulsarConfig.put(TOPIC_PATTERN.key(), "schema_.*"); } - pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); + pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); PulsarSyncTableAction action = syncTableActionBuilder(pulsarConfig) @@ -93,7 +92,8 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, getMessages("kafka/%s/table/%s/%s-data-2.txt", FORMAT, sourceDir, FORMAT)); + topic, + getMessages(String.format("kafka/canal/table/%s/canal-data-2.txt", sourceDir))); rowType = RowType.of( @@ -115,7 +115,8 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, getMessages("kafka/%s/table/%s/%s-data-3.txt", FORMAT, sourceDir, FORMAT)); + topic, + getMessages(String.format("kafka/canal/table/%s/canal-data-3.txt", sourceDir))); rowType = RowType.of( @@ -138,7 +139,8 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, getMessages("kafka/%s/table/%s/%s-data-4.txt", FORMAT, sourceDir, FORMAT)); + topic, + getMessages(String.format("kafka/canal/table/%s/canal-data-4.txt", sourceDir))); rowType = RowType.of( @@ -165,7 +167,8 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exce waitForResult(expected, table, rowType, primaryKeys); sendMessages( - topic, getMessages("kafka/%s/table/%s/%s-data-5.txt", FORMAT, sourceDir, FORMAT)); + topic, + getMessages(String.format("kafka/canal/table/%s/canal-data-5.txt", sourceDir))); rowType = RowType.of( @@ -198,12 +201,12 @@ public void testWaterMarkSyncTable() throws Exception { String topic = "watermark"; topics = Collections.singletonList(topic); createTopic(topic, 1); - sendMessages(topic, getMessages("kafka/%s/table/watermark/%s-data-1.txt", FORMAT, FORMAT)); + sendMessages(topic, getMessages("kafka/canal/table/watermark/canal-data-1.txt")); Map pulsarConfig = getBasicPulsarConfig(); pulsarConfig.put(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1"); pulsarConfig.put(TOPIC.key(), topic); - pulsarConfig.put(VALUE_FORMAT.key(), FORMAT); + pulsarConfig.put(VALUE_FORMAT.key(), "canal-json"); Map config = getBasicTableConfig(); config.put("tag.automatic-creation", "watermark"); config.put("tag.creation-period", "hourly"); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java index 0b127491ac20..a3aa12fcc706 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java @@ -43,13 +43,17 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; import static org.apache.paimon.CoreOptions.SCAN_MODE; import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; +import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; import static org.assertj.core.api.Assertions.assertThat; @@ -58,7 +62,25 @@ public class KafkaLogStoreFactoryTest { @ParameterizedTest @EnumSource(CoreOptions.StartupMode.class) - public void testCreateKafkaLogStoreFactory(CoreOptions.StartupMode startupMode) { + public void testCreateKafkaLogStoreFactoryTimestamp(CoreOptions.StartupMode startupMode) { + String now = String.valueOf(System.currentTimeMillis()); + Consumer> setter = + (options) -> options.put(SCAN_TIMESTAMP_MILLIS.key(), now); + testCreateKafkaLogStoreFactory(startupMode, setter); + } + + @ParameterizedTest + @EnumSource(CoreOptions.StartupMode.class) + public void testCreateKafkaLogStoreFactoryTimestampStr(CoreOptions.StartupMode startupMode) { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + String timestampString = LocalDateTime.now().format(formatter); + Consumer> setter = + (options) -> options.put(SCAN_TIMESTAMP.key(), timestampString); + testCreateKafkaLogStoreFactory(startupMode, setter); + } + + private static void testCreateKafkaLogStoreFactory( + CoreOptions.StartupMode startupMode, Consumer> optionsSetter) { Map dynamicOptions = new HashMap<>(); dynamicOptions.put(FlinkConnectorOptions.LOG_SYSTEM.key(), "kafka"); dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); @@ -66,8 +88,7 @@ public void testCreateKafkaLogStoreFactory(CoreOptions.StartupMode startupMode) || startupMode == CoreOptions.StartupMode.FROM_SNAPSHOT_FULL) { dynamicOptions.put(SCAN_SNAPSHOT_ID.key(), "1"); } else if (startupMode == CoreOptions.StartupMode.FROM_TIMESTAMP) { - dynamicOptions.put( - SCAN_TIMESTAMP_MILLIS.key(), String.valueOf(System.currentTimeMillis())); + optionsSetter.accept(dynamicOptions); } dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); DynamicTableFactory.Context context = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index ab4ac26bc30d..fd23e500d5e5 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import static org.assertj.core.api.Assertions.assertThat; @@ -59,7 +60,8 @@ public void cancel() {} () -> FlinkCatalogFactory.createPaimonCatalog(new Options()), FlinkConnectorOptions.SINK_COMMITTER_CPU.defaultValue(), null, - true); + true, + UUID.randomUUID().toString()); DataStreamSink dataStreamSink = sink.sinkFrom(input); // check the transformation graph diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/case-insensitive/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/case-insensitive/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/case-insensitive/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/case-insensitive/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/include/topic0/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/include/topic0/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/include/topic0/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/include/topic0/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/incomplete/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/incomplete/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/incomplete/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/incomplete/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic0/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic0/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/prefixsuffix/topic1/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/prefixsuffix/topic1/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic0/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic0/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic1/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic1/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/schemaevolution/topic2/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/schemaevolution/topic2/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/tostring/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/tostring/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/database/tostring/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/database/tostring/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/alltype/canal-json-data.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/alltype/canal-data.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/alltype/canal-json-data.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/alltype/canal-data.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/computedcolumn/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/computedcolumn/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-delete.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-delete.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-delete.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-insert.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-insert.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-insert.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-row.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-row.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-row.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-row.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-update.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-update.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/event/event-update.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/event/event-update.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/incomplete/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/incomplete/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/initialemptytopic/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/initialemptytopic/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/initialemptytopic/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/initialemptytopic/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nononddldata/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nononddldata/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nononddldata/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nononddldata/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nonpk/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nonpk/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/nonpk/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/nonpk/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/optionschange/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/optionschange/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-5.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-5.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolution/canal-json-data-5.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolution/canal-data-5.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-5.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmissingddl/canal-json-data-5.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmissingddl/canal-data-5.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/schemaevolutionmultiple/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/schemaevolutionmultiple/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/startupmode/canal-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/startupmode/canal-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/tostring/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/tostring/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/tostring/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/tostring/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/watermark/canal-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/watermark/canal-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal-json/table/watermark/canal-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/canal/table/watermark/canal-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/include/topic0/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/include/topic0/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic0/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/prefixsuffix/topic1/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic0/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/database/schemaevolution/topic1/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/computedcolumn/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/computedcolumn/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nestedtype/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nestedtype/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nestedtype/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nestedtype/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/nullvalue/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/alltype/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/alltype/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/alltype/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/alltype/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/computedcolumn/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/computedcolumn/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/computedcolumn/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/include/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/include/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/include/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/include/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schema/schemaevolution/debezium-json-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/schemaevolution/debezium-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schemaevolution/debezium-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/startupmode/debezium-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/watermark/debezium-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/watermark/debezium-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-json/table/watermark/debezium-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/watermark/debezium-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/include/topic0/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/include/topic0/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/include/topic0/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/include/topic0/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic0/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic0/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/prefixsuffix/topic1/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/prefixsuffix/topic1/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic0/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic0/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/database/schemaevolution/topic1/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/database/schemaevolution/topic1/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/computedcolumn/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/computedcolumn/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/computedcolumn/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/computedcolumn/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-4.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/schemaevolution/maxwell-json-data-4.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/schemaevolution/maxwell-data-4.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/startupmode/maxwell-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/startupmode/maxwell-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/watermark/maxwell-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/watermark/maxwell-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell-json/table/watermark/maxwell-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/maxwell/table/watermark/maxwell-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/case-insensitive/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/case-insensitive/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/include/topic0/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/include/topic0/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic0/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/prefixsuffix/topic1/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic0/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/database/schemaevolution/topic1/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/computedcolumn/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/computedcolumn/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/computedcolumn/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/computedcolumn/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-delete.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-insert.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-update.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/event/event-update.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/schemaevolution/ogg-json-data-3.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/startupmode/ogg-json-data-2.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/watermark/ogg-json-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/watermark/ogg-data-1.txt similarity index 100% rename from paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg-json/table/watermark/ogg-json-data-1.txt rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/watermark/ogg-data-1.txt From b5bf0d9650805847176cb6838cecfb10d024bbf3 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Tue, 18 Jun 2024 14:11:26 +0800 Subject: [PATCH 08/12] add KafkaDebeziumAvroSyncTableActionITCase --- .../cdc/kafka/KafkaActionITCaseBase.java | 51 +++ ...afkaDebeziumAvroSyncTableActionITCase.java | 309 ++++++++++++++++++ .../schema/alltype/debezium-avro-data-1.txt | 22 ++ 3 files changed, 382 insertions(+) create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java index 7d8c78c2eddf..26565e8e23cc 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java @@ -43,12 +43,14 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Order; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -78,6 +80,7 @@ public abstract class KafkaActionITCaseBase extends CdcActionITCaseBase { private static final Logger LOG = LoggerFactory.getLogger(KafkaActionITCaseBase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; + private static final String INTER_CONTAINER_SCHEMA_REGISTRY_ALIAS = "schemaregistry"; private static final Network NETWORK = Network.newNetwork(); private static final int ZK_TIMEOUT_MILLIS = 30000; @@ -89,6 +92,7 @@ public abstract class KafkaActionITCaseBase extends CdcActionITCaseBase { private final Timer loggingTimer = new Timer("Debug Logging Timer"); @RegisterExtension + @Order(1) public static final KafkaContainerExtension KAFKA_CONTAINER = (KafkaContainerExtension) new KafkaContainerExtension(DockerImageName.parse(DockerImageVersions.KAFKA)) { @@ -109,6 +113,21 @@ protected void doStart() { // test run .withEnv("KAFKA_LOG_RETENTION_MS", "-1"); + @RegisterExtension + @Order(2) + public static final SchemaRegistryContainerExtension SCHEMA_REGISTRY_CONTAINER = + new SchemaRegistryContainerExtension( + DockerImageName.parse(DockerImageVersions.SCHEMA_REGISTRY)) + .dependsOn(KAFKA_CONTAINER) + .withNetwork(NETWORK) + .withEnv("SCHEMA_REGISTRY_HOST_NAME", "schema_registry") + .withEnv( + "SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", + "PLAINTEXT://" + KAFKA_CONTAINER.getNetworkAliases().get(0) + ":9092") + .withNetworkAliases(INTER_CONTAINER_SCHEMA_REGISTRY_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(LOG)) + .withStartupTimeout(Duration.ofSeconds(60)); + @BeforeAll public static void beforeAll() { // create KafkaProducer @@ -257,6 +276,10 @@ protected Map getBasicKafkaConfig() { return config; } + protected String getSchemaRegistryUrl() { + return SCHEMA_REGISTRY_CONTAINER.getSchemaRegistryUrl(); + } + protected KafkaSyncTableActionBuilder syncTableActionBuilder(Map kafkaConfig) { return new KafkaSyncTableActionBuilder(kafkaConfig); } @@ -356,4 +379,32 @@ public void afterAll(ExtensionContext extensionContext) { this.close(); } } + + /** Schema registry container extension for junit5. */ + private static class SchemaRegistryContainerExtension + extends GenericContainer + implements BeforeAllCallback, AfterAllCallback { + + private static final Integer SCHEMA_REGISTRY_EXPOSED_PORT = 8081; + + private SchemaRegistryContainerExtension(DockerImageName dockerImageName) { + super(dockerImageName); + addExposedPorts(SCHEMA_REGISTRY_EXPOSED_PORT); + } + + @Override + public void beforeAll(ExtensionContext extensionContext) { + this.doStart(); + } + + @Override + public void afterAll(ExtensionContext extensionContext) { + this.close(); + } + + public String getSchemaRegistryUrl() { + return String.format( + "http://%s:%s", getHost(), getMappedPort(SCHEMA_REGISTRY_EXPOSED_PORT)); + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java new file mode 100644 index 000000000000..8419c9ad31e1 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java @@ -0,0 +1,309 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.flink.core.execution.JobClient; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaDebeziumAvroSyncTableActionITCase extends KafkaActionITCaseBase { + + @Test + @Timeout(60) + public void testAllTypes() throws Exception { + // the first round checks for table creation + // the second round checks for running the action on an existing table + for (int i = 0; i < 2; i++) { + testAllTypesOnce(); + Thread.sleep(3000); + } + } + + private void testAllTypesOnce() throws Exception { + final String topic = "all_type" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + writeRecordsToKafka( + topic, "kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt"); + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put("value.format", "debezium-avro"); + kafkaConfig.put("topic", topic); + kafkaConfig.put("schema.registry.url", getSchemaRegistryUrl()); + + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPartitionKeys("pt") + .withPrimaryKeys("pt", "_id") + .build(); + JobClient client = runActionWithDefaultEnv(action); + + testAllTypesImpl(); + client.cancel().get(); + } + + /** For all types test case. */ + protected void testAllTypesImpl() throws Exception { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), // _id + DataTypes.DECIMAL(2, 1).notNull(), // pt + DataTypes.BOOLEAN(), // _tinyint1 + DataTypes.BOOLEAN(), // _boolean + DataTypes.BOOLEAN(), // _bool + DataTypes.TINYINT(), // _tinyint + DataTypes.SMALLINT(), // _tinyint_unsigned + DataTypes.SMALLINT(), // _tinyint_unsigned_zerofill + DataTypes.SMALLINT(), // _smallint + DataTypes.INT(), // _smallint_unsigned + DataTypes.INT(), // _smallint_unsigned_zerofill + DataTypes.INT(), // _mediumint + DataTypes.BIGINT(), // _mediumint_unsigned + DataTypes.BIGINT(), // _mediumint_unsigned_zerofill + DataTypes.INT(), // _int + DataTypes.BIGINT(), // _int_unsigned + DataTypes.BIGINT(), // _int_unsigned_zerofill + DataTypes.BIGINT(), // _bigint + DataTypes.DECIMAL(20, 0), // _bigint_unsigned + DataTypes.DECIMAL(20, 0), // _bigint_unsigned_zerofill + DataTypes.DECIMAL(20, 0), // _serial + DataTypes.FLOAT(), // _float + DataTypes.FLOAT(), // _float_unsigned + DataTypes.FLOAT(), // _float_unsigned_zerofill + DataTypes.DOUBLE(), // _real + DataTypes.DOUBLE(), // _real_unsigned + DataTypes.DOUBLE(), // _real_unsigned_zerofill + DataTypes.DOUBLE(), // _double + DataTypes.DOUBLE(), // _double_unsigned + DataTypes.DOUBLE(), // _double_unsigned_zerofill + DataTypes.DOUBLE(), // _double_precision + DataTypes.DOUBLE(), // _double_precision_unsigned + DataTypes.DOUBLE(), // _double_precision_unsigned_zerofill + DataTypes.DECIMAL(8, 3), // _numeric + DataTypes.DECIMAL(8, 3), // _numeric_unsigned + DataTypes.DECIMAL(8, 3), // _numeric_unsigned_zerofill + DataTypes.STRING(), // _fixed + DataTypes.STRING(), // _fixed_unsigned + DataTypes.STRING(), // _fixed_unsigned_zerofill + DataTypes.DECIMAL(8, 0), // _decimal + DataTypes.DECIMAL(8, 0), // _decimal_unsigned + DataTypes.DECIMAL(8, 0), // _decimal_unsigned_zerofill + DataTypes.INT(), // _date + DataTypes.BIGINT(), // _datetime + DataTypes.BIGINT(), // _datetime3 + DataTypes.BIGINT(), // _datetime6 + DataTypes.BIGINT(), // _datetime_p + DataTypes.BIGINT(), // _datetime_p2 + DataTypes.STRING(), // _timestamp + DataTypes.STRING(), // _timestamp0 + DataTypes.CHAR(10), // _char + DataTypes.VARCHAR(20), // _varchar + DataTypes.STRING(), // _tinytext + DataTypes.STRING(), // _text + DataTypes.STRING(), // _mediumtext + DataTypes.STRING(), // _longtext + DataTypes.VARBINARY(10), // _bin + DataTypes.VARBINARY(20), // _varbin + DataTypes.BYTES(), // _tinyblob + DataTypes.BYTES(), // _blob + DataTypes.BYTES(), // _mediumblob + DataTypes.BYTES(), // _longblob + DataTypes.STRING(), // _json + DataTypes.STRING(), // _enum + DataTypes.INT(), // _year + DataTypes.BIGINT(), // _time + DataTypes.STRING(), // _point + DataTypes.STRING(), // _geometry + DataTypes.STRING(), // _linestring + DataTypes.STRING(), // _polygon + DataTypes.STRING(), // _multipoint + DataTypes.STRING(), // _multiline + DataTypes.STRING(), // _multipolygon + DataTypes.STRING(), // _geometrycollection + DataTypes.ARRAY(DataTypes.STRING()) // _set + }, + new String[] { + "_id", + "pt", + "_tinyint1", + "_boolean", + "_bool", + "_tinyint", + "_tinyint_unsigned", + "_tinyint_unsigned_zerofill", + "_smallint", + "_smallint_unsigned", + "_smallint_unsigned_zerofill", + "_mediumint", + "_mediumint_unsigned", + "_mediumint_unsigned_zerofill", + "_int", + "_int_unsigned", + "_int_unsigned_zerofill", + "_bigint", + "_bigint_unsigned", + "_bigint_unsigned_zerofill", + "_serial", + "_float", + "_float_unsigned", + "_float_unsigned_zerofill", + "_real", + "_real_unsigned", + "_real_unsigned_zerofill", + "_double", + "_double_unsigned", + "_double_unsigned_zerofill", + "_double_precision", + "_double_precision_unsigned", + "_double_precision_unsigned_zerofill", + "_numeric", + "_numeric_unsigned", + "_numeric_unsigned_zerofill", + "_fixed", + "_fixed_unsigned", + "_fixed_unsigned_zerofill", + "_decimal", + "_decimal_unsigned", + "_decimal_unsigned_zerofill", + "_date", + "_datetime", + "_datetime3", + "_datetime6", + "_datetime_p", + "_datetime_p2", + "_timestamp", + "_timestamp0", + "_char", + "_varchar", + "_tinytext", + "_text", + "_mediumtext", + "_longtext", + "_bin", + "_varbin", + "_tinyblob", + "_blob", + "_mediumblob", + "_longblob", + "_json", + "_enum", + "_year", + "_time", + "_point", + "_geometry", + "_linestring", + "_polygon", + "_multipoint", + "_multiline", + "_multipolygon", + "_geometrycollection", + "_set", + }); + FileStoreTable table = getFileStoreTable(tableName); + List expected = + Arrays.asList( + "+I[" + + "1, 1.1, " + + "true, true, false, 1, 2, 3, " + + "1000, 2000, 3000, " + + "100000, 200000, 300000, " + + "1000000, 2000000, 3000000, " + + "10000000000, 20000000000, 30000000000, 40000000000, " + + "1.5, 2.5, 3.5, " + + "1.000001, 2.000002, 3.000003, " + + "1.000011, 2.000022, 3.000033, " + + "1.000111, 2.000222, 3.000333, " + + "12345.110, 12345.220, 12345.330, " + + "123456789876543212345678987654321.110, 123456789876543212345678987654321.220, 123456789876543212345678987654321.330, " + + "11111, 22222, 33333, " + + "19439, " + // display value of datetime is not affected by timezone + + "1679581805000, 1679581805123, 1679581805123456, " + + "1679668200000, 1679668205120, " + + "2023-03-23T07:00:10.123456Z, 2023-03-22T16:10:00Z, " + + "Paimon, Apache Paimon, Apache Paimon MySQL TINYTEXT Test Data, Apache Paimon MySQL Test Data, Apache Paimon MySQL MEDIUMTEXT Test Data, Apache Paimon MySQL Long Test Data, " + + "[98, 121, 116, 101, 115, 0, 0, 0, 0, 0], " + + "[109, 111, 114, 101, 32, 98, 121, 116, 101, 115], " + + "[84, 73, 78, 89, 66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + + "[66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + + "[77, 69, 68, 73, 85, 77, 66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + + "[76, 79, 78, 71, 66, 76, 79, 66, 32, 32, 98, 121, 116, 101, 115, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + + "{\"a\": \"b\"}, " + + "value1, " + + "2023, " + + "36803000000, " + + "{\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}, " + + "{\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, " + + "{\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0}, " + + "{\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, " + + "{\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0}, " + + "{\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0}, " + + "{\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0}, " + + "{\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0}, " + + "[a, b]" + + "]", + "+I[" + + "2, 2.2, " + + "NULL, NULL, NULL, NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, 50000000000, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, NULL, " + + "NULL, " + + "NULL, NULL, NULL, " + + "NULL, NULL, " + + "NULL, NULL, " + + "NULL, NULL, NULL, NULL, NULL, NULL, " + + "NULL, NULL, NULL, NULL, NULL, NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL, " + + "NULL" + + "]"); + + waitForResult(expected, table, rowType, Arrays.asList("pt", "_id")); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt new file mode 100644 index 000000000000..6c9b0ad75979 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt @@ -0,0 +1,22 @@ +/* + * 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. + */ + +{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":1}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":1,"pt":"1.1","_tinyint1":1,"_boolean":true,"_bool":false,"_tinyint":1,"_tinyint_unsigned":2,"_tinyint_unsigned_zerofill":3,"_smallint":1000,"_smallint_unsigned":2000,"_smallint_unsigned_zerofill":3000,"_mediumint":100000,"_mediumint_unsigned":200000,"_mediumint_unsigned_zerofill":300000,"_int":1000000,"_int_unsigned":2000000,"_int_unsigned_zerofill":3000000,"_bigint":10000000000,"_bigint_unsigned":20000000000,"_bigint_unsigned_zerofill":30000000000,"_serial":40000000000,"_float":1.5,"_float_unsigned":2.5,"_float_unsigned_zerofill":3.5,"_real":1.000001,"_real_unsigned":2.000002,"_real_unsigned_zerofill":3.000003,"_double":1.000011,"_double_unsigned":2.000022,"_double_unsigned_zerofill":3.000033,"_double_precision":1.000111,"_double_precision_unsigned":2.000222,"_double_precision_unsigned_zerofill":3.000333,"_numeric":"12345.110","_numeric_unsigned":"12345.220","_numeric_unsigned_zerofill":"12345.330","_fixed":"123456789876543212345678987654321.110","_fixed_unsigned":"123456789876543212345678987654321.220","_fixed_unsigned_zerofill":"123456789876543212345678987654321.330","_decimal":"11111","_decimal_unsigned":"22222","_decimal_unsigned_zerofill":"33333","_date":19439,"_datetime":1679581805000,"_datetime3":1679581805123,"_datetime6":1679581805123456,"_datetime_p":1679668200000,"_datetime_p2":1679668205120,"_timestamp":"2023-03-23T07:00:10.123456Z","_timestamp0":"2023-03-22T16:10:00Z","_char":"Paimon","_varchar":"Apache Paimon","_tinytext":"Apache Paimon MySQL TINYTEXT Test Data","_text":"Apache Paimon MySQL Test Data","_mediumtext":"Apache Paimon MySQL MEDIUMTEXT Test Data","_longtext":"Apache Paimon MySQL Long Test Data","_bin":"Ynl0ZXMAAAAAAA==","_varbin":"bW9yZSBieXRlcw==","_tinyblob":"VElOWUJMT0IgdHlwZSB0ZXN0IGRhdGE=","_blob":"QkxPQiB0eXBlIHRlc3QgZGF0YQ==","_mediumblob":"TUVESVVNQkxPQiB0eXBlIHRlc3QgZGF0YQ==","_longblob":"TE9OR0JMT0IgIGJ5dGVzIHRlc3QgZGF0YQ==","_json":"{\"a\": \"b\"}","_enum":"value1","_year":2023,"_time":36803000000,"_point":{"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},"_geometry":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_linestring":{"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},"_polygon":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_multipoint":{"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},"_multiline":{"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},"_multipolygon":{"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},"_geometrycollection":{"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null},"_set":"a,b"},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695606891000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210576","file":"mysql-bin.000074","pos":13593,"row":0,"thread":42,"query":null},"op":"c","ts_ms":1695606891883,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":2}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":2,"pt":"2.2","_tinyint1":null,"_boolean":null,"_bool":null,"_tinyint":null,"_tinyint_unsigned":null,"_tinyint_unsigned_zerofill":null,"_smallint":null,"_smallint_unsigned":null,"_smallint_unsigned_zerofill":null,"_mediumint":null,"_mediumint_unsigned":null,"_mediumint_unsigned_zerofill":null,"_int":null,"_int_unsigned":null,"_int_unsigned_zerofill":null,"_bigint":null,"_bigint_unsigned":null,"_bigint_unsigned_zerofill":null,"_serial":50000000000,"_float":null,"_float_unsigned":null,"_float_unsigned_zerofill":null,"_real":null,"_real_unsigned":null,"_real_unsigned_zerofill":null,"_double":null,"_double_unsigned":null,"_double_unsigned_zerofill":null,"_double_precision":null,"_double_precision_unsigned":null,"_double_precision_unsigned_zerofill":null,"_numeric":null,"_numeric_unsigned":null,"_numeric_unsigned_zerofill":null,"_fixed":null,"_fixed_unsigned":null,"_fixed_unsigned_zerofill":null,"_decimal":null,"_decimal_unsigned":null,"_decimal_unsigned_zerofill":null,"_date":null,"_datetime":null,"_datetime3":null,"_datetime6":null,"_datetime_p":null,"_datetime_p2":null,"_timestamp":null,"_timestamp0":null,"_char":null,"_varchar":null,"_tinytext":null,"_text":null,"_mediumtext":null,"_longtext":null,"_bin":null,"_varbin":null,"_tinyblob":null,"_blob":null,"_mediumblob":null,"_longblob":null,"_json":null,"_enum":null,"_year":null,"_time":null,"_point":null,"_geometry":null,"_linestring":null,"_polygon":null,"_multipoint":null,"_multiline":null,"_multipolygon":null,"_geometrycollection":null,"_set":null},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695126006000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210568","file":"mysql-bin.000073","pos":337176435,"row":0,"thread":279,"query":null},"op":"c","ts_ms":1695126006877,"transaction":null}} \ No newline at end of file From cc10e49248142851f24a3bf0dd71529e84f6e66d Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Fri, 21 Jun 2024 11:59:52 +0800 Subject: [PATCH 09/12] add test --- .../KafkaDebeziumAvroDeserializationSchema.java | 12 +++--------- .../table/schema/alltype/debezium-avro-data-1.txt | 3 --- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index 0cbca3ffd7ff..16f422a1dbcc 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -23,7 +23,6 @@ import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; @@ -43,19 +42,14 @@ public class KafkaDebeziumAvroDeserializationSchema private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; private final String topic; - private final String schemaRegistryUrl; /** The deserializer to deserialize Debezium Avro data. */ - private ConfluentAvroDeserializationSchema avroDeserializer; + private final ConfluentAvroDeserializationSchema avroDeserializer; public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); - } - - @Override - public void open(DeserializationSchema.InitializationContext context) throws Exception { - avroDeserializer = + String schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.avroDeserializer = new ConfluentAvroDeserializationSchema( new CachedSchemaRegistryClient( schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt index 6c9b0ad75979..1b58a6e2c905 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt @@ -16,7 +16,4 @@ * limitations under the License. */ -{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":1}} {"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":1,"pt":"1.1","_tinyint1":1,"_boolean":true,"_bool":false,"_tinyint":1,"_tinyint_unsigned":2,"_tinyint_unsigned_zerofill":3,"_smallint":1000,"_smallint_unsigned":2000,"_smallint_unsigned_zerofill":3000,"_mediumint":100000,"_mediumint_unsigned":200000,"_mediumint_unsigned_zerofill":300000,"_int":1000000,"_int_unsigned":2000000,"_int_unsigned_zerofill":3000000,"_bigint":10000000000,"_bigint_unsigned":20000000000,"_bigint_unsigned_zerofill":30000000000,"_serial":40000000000,"_float":1.5,"_float_unsigned":2.5,"_float_unsigned_zerofill":3.5,"_real":1.000001,"_real_unsigned":2.000002,"_real_unsigned_zerofill":3.000003,"_double":1.000011,"_double_unsigned":2.000022,"_double_unsigned_zerofill":3.000033,"_double_precision":1.000111,"_double_precision_unsigned":2.000222,"_double_precision_unsigned_zerofill":3.000333,"_numeric":"12345.110","_numeric_unsigned":"12345.220","_numeric_unsigned_zerofill":"12345.330","_fixed":"123456789876543212345678987654321.110","_fixed_unsigned":"123456789876543212345678987654321.220","_fixed_unsigned_zerofill":"123456789876543212345678987654321.330","_decimal":"11111","_decimal_unsigned":"22222","_decimal_unsigned_zerofill":"33333","_date":19439,"_datetime":1679581805000,"_datetime3":1679581805123,"_datetime6":1679581805123456,"_datetime_p":1679668200000,"_datetime_p2":1679668205120,"_timestamp":"2023-03-23T07:00:10.123456Z","_timestamp0":"2023-03-22T16:10:00Z","_char":"Paimon","_varchar":"Apache Paimon","_tinytext":"Apache Paimon MySQL TINYTEXT Test Data","_text":"Apache Paimon MySQL Test Data","_mediumtext":"Apache Paimon MySQL MEDIUMTEXT Test Data","_longtext":"Apache Paimon MySQL Long Test Data","_bin":"Ynl0ZXMAAAAAAA==","_varbin":"bW9yZSBieXRlcw==","_tinyblob":"VElOWUJMT0IgdHlwZSB0ZXN0IGRhdGE=","_blob":"QkxPQiB0eXBlIHRlc3QgZGF0YQ==","_mediumblob":"TUVESVVNQkxPQiB0eXBlIHRlc3QgZGF0YQ==","_longblob":"TE9OR0JMT0IgIGJ5dGVzIHRlc3QgZGF0YQ==","_json":"{\"a\": \"b\"}","_enum":"value1","_year":2023,"_time":36803000000,"_point":{"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},"_geometry":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_linestring":{"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},"_polygon":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_multipoint":{"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},"_multiline":{"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},"_multipolygon":{"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},"_geometrycollection":{"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null},"_set":"a,b"},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695606891000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210576","file":"mysql-bin.000074","pos":13593,"row":0,"thread":42,"query":null},"op":"c","ts_ms":1695606891883,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":2}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":2,"pt":"2.2","_tinyint1":null,"_boolean":null,"_bool":null,"_tinyint":null,"_tinyint_unsigned":null,"_tinyint_unsigned_zerofill":null,"_smallint":null,"_smallint_unsigned":null,"_smallint_unsigned_zerofill":null,"_mediumint":null,"_mediumint_unsigned":null,"_mediumint_unsigned_zerofill":null,"_int":null,"_int_unsigned":null,"_int_unsigned_zerofill":null,"_bigint":null,"_bigint_unsigned":null,"_bigint_unsigned_zerofill":null,"_serial":50000000000,"_float":null,"_float_unsigned":null,"_float_unsigned_zerofill":null,"_real":null,"_real_unsigned":null,"_real_unsigned_zerofill":null,"_double":null,"_double_unsigned":null,"_double_unsigned_zerofill":null,"_double_precision":null,"_double_precision_unsigned":null,"_double_precision_unsigned_zerofill":null,"_numeric":null,"_numeric_unsigned":null,"_numeric_unsigned_zerofill":null,"_fixed":null,"_fixed_unsigned":null,"_fixed_unsigned_zerofill":null,"_decimal":null,"_decimal_unsigned":null,"_decimal_unsigned_zerofill":null,"_date":null,"_datetime":null,"_datetime3":null,"_datetime6":null,"_datetime_p":null,"_datetime_p2":null,"_timestamp":null,"_timestamp0":null,"_char":null,"_varchar":null,"_tinytext":null,"_text":null,"_mediumtext":null,"_longtext":null,"_bin":null,"_varbin":null,"_tinyblob":null,"_blob":null,"_mediumblob":null,"_longblob":null,"_json":null,"_enum":null,"_year":null,"_time":null,"_point":null,"_geometry":null,"_linestring":null,"_polygon":null,"_multipoint":null,"_multiline":null,"_multipolygon":null,"_geometrycollection":null,"_set":null},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695126006000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210568","file":"mysql-bin.000073","pos":337176435,"row":0,"thread":279,"query":null},"op":"c","ts_ms":1695126006877,"transaction":null}} \ No newline at end of file From ecf4a2074824120d71bacee9d494cd3b0e08100a Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Wed, 26 Jun 2024 17:27:36 +0800 Subject: [PATCH 10/12] add avro cdc test --- .../debezium/DebeziumAvroRecordParser.java | 6 +- ...afkaDebeziumAvroDeserializationSchema.java | 14 +- .../flink/action/cdc/CdcActionITCaseBase.java | 2 + .../cdc/kafka/KafkaActionITCaseBase.java | 6 +- ...afkaDebeziumAvroSyncTableActionITCase.java | 520 +++++++++++++++++- .../schema/alltype/debezium-avro-data-1.txt | 3 + .../src/test/resources/log4j2-test.properties | 2 +- 7 files changed, 528 insertions(+), 25 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java index ddaa253d0936..6d0eddef2774 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java @@ -137,7 +137,9 @@ private Map extractRowData( String transformed = DebeziumSchemaUtils.transformRawValue( rawValue, - schema.getLogicalType().getName(), + schema.getLogicalType() != null + ? schema.getLogicalType().getName() + : null, schema.getFullName(), typeMapping, () -> @@ -189,7 +191,7 @@ private String getFromSourceField(String key) { if (Objects.isNull(source)) { return null; } - return (String) source.get(key); + return source.get(key).toString(); } protected Object getAndCheck(String key) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index 16f422a1dbcc..305e55bfbcf0 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.action.cdc.serialization.ConfluentAvroDeserializationSchema; import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.serializers.GenericContainerWithVersion; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; @@ -62,10 +63,15 @@ public CdcSourceRecord deserialize(ConsumerRecord message) throw return null; } - GenericRecord key = - (GenericRecord) this.avroDeserializer.deserialize(topic, true, message.key()); - GenericRecord value = - (GenericRecord) this.avroDeserializer.deserialize(topic, false, message.value()); + GenericContainerWithVersion keyContainerWithVersion = + this.avroDeserializer.deserialize(topic, true, message.key()); + GenericContainerWithVersion valueContainerWithVersion = + this.avroDeserializer.deserialize(topic, false, message.value()); + GenericRecord key = null; + if (keyContainerWithVersion != null) { + key = (GenericRecord) keyContainerWithVersion.container(); + } + GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); return new CdcSourceRecord(topic, key, value); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 468a3074d61b..5e1d240dd5b3 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -125,6 +125,8 @@ protected void waitForResult( boolean sameType = field.type().equals(rowType.getFieldTypes().get(i)); if (sameName && sameType) { cnt++; + } else { + System.out.println("ss"); } } if (cnt == rowType.getFieldCount()) { diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java index 26565e8e23cc..22ed53d13cab 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java @@ -75,7 +75,7 @@ /** Base test class for Kafka synchronization. */ public abstract class KafkaActionITCaseBase extends CdcActionITCaseBase { - private final ObjectMapper objectMapper = new ObjectMapper(); + protected final ObjectMapper objectMapper = new ObjectMapper(); private static final Logger LOG = LoggerFactory.getLogger(KafkaActionITCaseBase.class); @@ -334,7 +334,7 @@ protected void writeRecordsToKafka(String topic, String resourceDirFormat, Objec protected void writeRecordsToKafka( String topic, boolean wait, String resourceDirFormat, Object... args) throws Exception { URL url = - KafkaCanalSyncTableActionITCase.class + KafkaActionITCaseBase.class .getClassLoader() .getResource(String.format(resourceDirFormat, args)); Files.readAllLines(Paths.get(url.toURI())).stream() @@ -342,7 +342,7 @@ protected void writeRecordsToKafka( .forEach(r -> send(topic, r, wait)); } - private boolean isRecordLine(String line) { + protected boolean isRecordLine(String line) { try { objectMapper.readTree(line); return !StringUtils.isEmpty(line); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java index 8419c9ad31e1..2dff8e92f55a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java @@ -23,20 +23,74 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.Base64Variants; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; + +import io.confluent.kafka.serializers.KafkaAvroSerializer; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.flink.core.execution.JobClient; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Paths; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.UUID; +import java.util.stream.Collectors; + +import static io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.AUTO_REGISTER_SCHEMAS; +import static io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG; /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaDebeziumAvroSyncTableActionITCase extends KafkaActionITCaseBase { + // Serializer for debezium avro format + protected static final Schema NULL_AVRO_SCHEMA = Schema.create(Schema.Type.NULL); + protected KafkaAvroSerializer kafkaKeyAvroSerializer; + protected KafkaAvroSerializer kafkaValueAvroSerializer; + + private static final String ALL_TYPES_TABLE_KEY_SCHEMA = + "{\"type\":\"record\",\"name\":\"Key\",\"namespace\":\"test_avro.workdb.all_types_table\",\"fields\":[{\"name\":\"_id\",\"type\":\"int\"}],\"connect.name\":\"test_avro.workdb.all_types_table.Key\"}"; + private static final String ALL_TYPES_TABLE_VALUE_SCHEMA = + "{\"type\":\"record\",\"name\":\"Envelope\",\"namespace\":\"test_avro.workdb.all_types_table\",\"fields\":[{\"name\":\"before\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Value\",\"fields\":[{\"name\":\"_id\",\"type\":{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"INT\",\"__debezium.source.column.name\":\"_id\"}}},{\"name\":\"pt\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DECIMAL\",\"__debezium.source.column.length\":\"2\",\"__debezium.source.column.scale\":\"1\",\"__debezium.source.column.name\":\"pt\"}}],\"default\":null},{\"name\":\"_tinyint1\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYINT\",\"__debezium.source.column.length\":\"1\",\"__debezium.source.column.name\":\"_tinyint1\"},\"connect.type\":\"int16\"}],\"default\":null},{\"name\":\"_boolean\",\"type\":[\"null\",{\"type\":\"boolean\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BOOLEAN\",\"__debezium.source.column.name\":\"_boolean\"}}],\"default\":null},{\"name\":\"_bool\",\"type\":[\"null\",{\"type\":\"boolean\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BOOL\",\"__debezium.source.column.name\":\"_bool\"}}],\"default\":null},{\"name\":\"_tinyint\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYINT\",\"__debezium.source.column.name\":\"_tinyint\"},\"connect.type\":\"int16\"}],\"default\":null},{\"name\":\"_tinyint_unsigned\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYINT UNSIGNED\",\"__debezium.source.column.length\":\"2\",\"__debezium.source.column.name\":\"_tinyint_unsigned\"},\"connect.type\":\"int16\"}],\"default\":null},{\"name\":\"_tinyint_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYINT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"2\",\"__debezium.source.column.name\":\"_tinyint_unsigned_zerofill\"},\"connect.type\":\"int16\"}],\"default\":null},{\"name\":\"_smallint\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"SMALLINT\",\"__debezium.source.column.name\":\"_smallint\"},\"connect.type\":\"int16\"}],\"default\":null},{\"name\":\"_smallint_unsigned\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"SMALLINT UNSIGNED\",\"__debezium.source.column.name\":\"_smallint_unsigned\"}}],\"default\":null},{\"name\":\"_smallint_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"SMALLINT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"4\",\"__debezium.source.column.name\":\"_smallint_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_mediumint\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"MEDIUMINT\",\"__debezium.source.column.name\":\"_mediumint\"}}],\"default\":null},{\"name\":\"_mediumint_unsigned\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"MEDIUMINT UNSIGNED\",\"__debezium.source.column.name\":\"_mediumint_unsigned\"}}],\"default\":null},{\"name\":\"_mediumint_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"MEDIUMINT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.name\":\"_mediumint_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_int\",\"type\":[\"null\",{\"type\":\"int\",\"connect.parameters\":{\"__debezium.source.column.type\":\"INT\",\"__debezium.source.column.name\":\"_int\"}}],\"default\":null},{\"name\":\"_int_unsigned\",\"type\":[\"null\",{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"INT UNSIGNED\",\"__debezium.source.column.name\":\"_int_unsigned\"}}],\"default\":null},{\"name\":\"_int_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"INT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.name\":\"_int_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_bigint\",\"type\":[\"null\",{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BIGINT\",\"__debezium.source.column.name\":\"_bigint\"}}],\"default\":null},{\"name\":\"_bigint_unsigned\",\"type\":[\"null\",{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BIGINT UNSIGNED\",\"__debezium.source.column.name\":\"_bigint_unsigned\"}}],\"default\":null},{\"name\":\"_bigint_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BIGINT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"16\",\"__debezium.source.column.name\":\"_bigint_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_serial\",\"type\":{\"type\":\"long\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BIGINT UNSIGNED\",\"__debezium.source.column.name\":\"_serial\"}}},{\"name\":\"_float\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FLOAT\",\"__debezium.source.column.name\":\"_float\"}}],\"default\":null},{\"name\":\"_float_unsigned\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FLOAT UNSIGNED\",\"__debezium.source.column.name\":\"_float_unsigned\"}}],\"default\":null},{\"name\":\"_float_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FLOAT UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"4\",\"__debezium.source.column.name\":\"_float_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_real\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"REAL\",\"__debezium.source.column.name\":\"_real\"}}],\"default\":null},{\"name\":\"_real_unsigned\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"REAL UNSIGNED\",\"__debezium.source.column.name\":\"_real_unsigned\"}}],\"default\":null},{\"name\":\"_real_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"float\",\"connect.parameters\":{\"__debezium.source.column.type\":\"REAL UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"10\",\"__debezium.source.column.scale\":\"7\",\"__debezium.source.column.name\":\"_real_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_double\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE\",\"__debezium.source.column.name\":\"_double\"}}],\"default\":null},{\"name\":\"_double_unsigned\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE UNSIGNED\",\"__debezium.source.column.name\":\"_double_unsigned\"}}],\"default\":null},{\"name\":\"_double_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"10\",\"__debezium.source.column.scale\":\"7\",\"__debezium.source.column.name\":\"_double_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_double_precision\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE PRECISION\",\"__debezium.source.column.name\":\"_double_precision\"}}],\"default\":null},{\"name\":\"_double_precision_unsigned\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE PRECISION UNSIGNED\",\"__debezium.source.column.name\":\"_double_precision_unsigned\"}}],\"default\":null},{\"name\":\"_double_precision_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"double\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DOUBLE PRECISION UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"10\",\"__debezium.source.column.scale\":\"7\",\"__debezium.source.column.name\":\"_double_precision_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_numeric\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"NUMERIC\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_numeric\"}}],\"default\":null},{\"name\":\"_numeric_unsigned\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"NUMERIC UNSIGNED\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_numeric_unsigned\"}}],\"default\":null},{\"name\":\"_numeric_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"NUMERIC UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_numeric_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_fixed\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FIXED\",\"__debezium.source.column.length\":\"40\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_fixed\"}}],\"default\":null},{\"name\":\"_fixed_unsigned\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FIXED UNSIGNED\",\"__debezium.source.column.length\":\"40\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_fixed_unsigned\"}}],\"default\":null},{\"name\":\"_fixed_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"FIXED UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"40\",\"__debezium.source.column.scale\":\"3\",\"__debezium.source.column.name\":\"_fixed_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_decimal\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DECIMAL\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"0\",\"__debezium.source.column.name\":\"_decimal\"}}],\"default\":null},{\"name\":\"_decimal_unsigned\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DECIMAL UNSIGNED\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"0\",\"__debezium.source.column.name\":\"_decimal_unsigned\"}}],\"default\":null},{\"name\":\"_decimal_unsigned_zerofill\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"DECIMAL UNSIGNED ZEROFILL\",\"__debezium.source.column.length\":\"8\",\"__debezium.source.column.scale\":\"0\",\"__debezium.source.column.name\":\"_decimal_unsigned_zerofill\"}}],\"default\":null},{\"name\":\"_date\",\"type\":[\"null\",{\"type\":\"int\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATE\",\"__debezium.source.column.name\":\"_date\"},\"connect.name\":\"io.debezium.time.Date\"}],\"default\":null},{\"name\":\"_datetime\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATETIME\",\"__debezium.source.column.name\":\"_datetime\"},\"connect.name\":\"io.debezium.time.Timestamp\"}],\"default\":null},{\"name\":\"_datetime3\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATETIME\",\"__debezium.source.column.length\":\"3\",\"__debezium.source.column.name\":\"_datetime3\"},\"connect.name\":\"io.debezium.time.Timestamp\"}],\"default\":null},{\"name\":\"_datetime6\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATETIME\",\"__debezium.source.column.length\":\"6\",\"__debezium.source.column.name\":\"_datetime6\"},\"connect.name\":\"io.debezium.time.MicroTimestamp\"}],\"default\":null},{\"name\":\"_datetime_p\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATETIME\",\"__debezium.source.column.name\":\"_datetime_p\"},\"connect.name\":\"io.debezium.time.Timestamp\"}],\"default\":null},{\"name\":\"_datetime_p2\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"DATETIME\",\"__debezium.source.column.length\":\"2\",\"__debezium.source.column.name\":\"_datetime_p2\"},\"connect.name\":\"io.debezium.time.Timestamp\"}],\"default\":null},{\"name\":\"_timestamp\",\"type\":[\"null\",{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"TIMESTAMP\",\"__debezium.source.column.length\":\"6\",\"__debezium.source.column.name\":\"_timestamp\"},\"connect.name\":\"io.debezium.time.ZonedTimestamp\"}],\"default\":null},{\"name\":\"_timestamp0\",\"type\":[\"null\",{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"TIMESTAMP\",\"__debezium.source.column.name\":\"_timestamp0\"},\"connect.name\":\"io.debezium.time.ZonedTimestamp\"}],\"default\":null},{\"name\":\"_char\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"CHAR\",\"__debezium.source.column.length\":\"10\",\"__debezium.source.column.name\":\"_char\"}}],\"default\":null},{\"name\":\"_varchar\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"VARCHAR\",\"__debezium.source.column.length\":\"20\",\"__debezium.source.column.name\":\"_varchar\"}}],\"default\":null},{\"name\":\"_tinytext\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYTEXT\",\"__debezium.source.column.name\":\"_tinytext\"}}],\"default\":null},{\"name\":\"_text\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TEXT\",\"__debezium.source.column.name\":\"_text\"}}],\"default\":null},{\"name\":\"_mediumtext\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"MEDIUMTEXT\",\"__debezium.source.column.name\":\"_mediumtext\"}}],\"default\":null},{\"name\":\"_longtext\",\"type\":[\"null\",{\"type\":\"string\",\"connect.parameters\":{\"__debezium.source.column.type\":\"LONGTEXT\",\"__debezium.source.column.name\":\"_longtext\"}}],\"default\":null},{\"name\":\"_bin\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BINARY\",\"__debezium.source.column.length\":\"10\",\"__debezium.source.column.name\":\"_bin\"}}],\"default\":null},{\"name\":\"_varbin\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"VARBINARY\",\"__debezium.source.column.length\":\"20\",\"__debezium.source.column.name\":\"_varbin\"}}],\"default\":null},{\"name\":\"_tinyblob\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"TINYBLOB\",\"__debezium.source.column.name\":\"_tinyblob\"}}],\"default\":null},{\"name\":\"_blob\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"BLOB\",\"__debezium.source.column.name\":\"_blob\"}}],\"default\":null},{\"name\":\"_mediumblob\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"MEDIUMBLOB\",\"__debezium.source.column.name\":\"_mediumblob\"}}],\"default\":null},{\"name\":\"_longblob\",\"type\":[\"null\",{\"type\":\"bytes\",\"connect.parameters\":{\"__debezium.source.column.type\":\"LONGBLOB\",\"__debezium.source.column.name\":\"_longblob\"}}],\"default\":null},{\"name\":\"_json\",\"type\":[\"null\",{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"JSON\",\"__debezium.source.column.name\":\"_json\"},\"connect.name\":\"io.debezium.data.Json\"}],\"default\":null},{\"name\":\"_enum\",\"type\":[\"null\",{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"value1,value2,value3\",\"__debezium.source.column.type\":\"ENUM\",\"__debezium.source.column.length\":\"1\",\"__debezium.source.column.name\":\"_enum\"},\"connect.name\":\"io.debezium.data.Enum\"}],\"default\":null},{\"name\":\"_year\",\"type\":[\"null\",{\"type\":\"int\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"YEAR\",\"__debezium.source.column.name\":\"_year\"},\"connect.name\":\"io.debezium.time.Year\"}],\"default\":null},{\"name\":\"_time\",\"type\":[\"null\",{\"type\":\"long\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"TIME\",\"__debezium.source.column.name\":\"_time\"},\"connect.name\":\"io.debezium.time.MicroTime\"}],\"default\":null},{\"name\":\"_point\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Point\",\"namespace\":\"io.debezium.data.geometry\",\"fields\":[{\"name\":\"x\",\"type\":\"double\"},{\"name\":\"y\",\"type\":\"double\"},{\"name\":\"wkb\",\"type\":[\"null\",\"bytes\"],\"default\":null},{\"name\":\"srid\",\"type\":[\"null\",\"int\"],\"default\":null}],\"connect.doc\":\"Geometry (POINT)\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"POINT\",\"__debezium.source.column.name\":\"_point\"},\"connect.name\":\"io.debezium.data.geometry.Point\"}],\"default\":null},{\"name\":\"_geometry\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Geometry\",\"namespace\":\"io.debezium.data.geometry\",\"fields\":[{\"name\":\"wkb\",\"type\":\"bytes\"},{\"name\":\"srid\",\"type\":[\"null\",\"int\"],\"default\":null}],\"connect.doc\":\"Geometry\",\"connect.version\":1,\"connect.parameters\":{\"__debezium.source.column.type\":\"GEOMETRY\",\"__debezium.source.column.name\":\"_geometry\"},\"connect.name\":\"io.debezium.data.geometry.Geometry\"}],\"default\":null},{\"name\":\"_linestring\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_polygon\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_multipoint\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_multiline\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_multipolygon\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_geometrycollection\",\"type\":[\"null\",\"io.debezium.data.geometry.Geometry\"],\"default\":null},{\"name\":\"_set\",\"type\":[\"null\",{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"a,b,c,d\",\"__debezium.source.column.type\":\"SET\",\"__debezium.source.column.length\":\"7\",\"__debezium.source.column.name\":\"_set\"},\"connect.name\":\"io.debezium.data.EnumSet\"}],\"default\":null}],\"connect.name\":\"test_avro.workdb.all_types_table.Value\"}],\"default\":null},{\"name\":\"after\",\"type\":[\"null\",\"Value\"],\"default\":null},{\"name\":\"source\",\"type\":{\"type\":\"record\",\"name\":\"Source\",\"namespace\":\"io.debezium.connector.mysql\",\"fields\":[{\"name\":\"version\",\"type\":\"string\"},{\"name\":\"connector\",\"type\":\"string\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":\"long\"},{\"name\":\"snapshot\",\"type\":[{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"true,last,false,incremental\"},\"connect.default\":\"false\",\"connect.name\":\"io.debezium.data.Enum\"},\"null\"],\"default\":\"false\"},{\"name\":\"db\",\"type\":\"string\"},{\"name\":\"sequence\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"table\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"server_id\",\"type\":\"long\"},{\"name\":\"gtid\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file\",\"type\":\"string\"},{\"name\":\"pos\",\"type\":\"long\"},{\"name\":\"row\",\"type\":\"int\"},{\"name\":\"thread\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"query\",\"type\":[\"null\",\"string\"],\"default\":null}],\"connect.name\":\"io.debezium.connector.mysql.Source\"}},{\"name\":\"op\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"transaction\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"block\",\"namespace\":\"event\",\"fields\":[{\"name\":\"id\",\"type\":\"string\"},{\"name\":\"total_order\",\"type\":\"long\"},{\"name\":\"data_collection_order\",\"type\":\"long\"}],\"connect.version\":1,\"connect.name\":\"event.block\"}],\"default\":null}],\"connect.version\":1,\"connect.name\":\"test_avro.workdb.all_types_table.Envelope\"}"; + private Schema allTypesTableKeySchema; + private Schema allTypesTableValueSchema; + private Schema debeziumSourceSchema; + + @BeforeEach + public void setup() { + super.setup(); + // Init avro serializer for kafka key/value + Map props = new HashMap<>(); + props.put(SCHEMA_REGISTRY_URL_CONFIG, getSchemaRegistryUrl()); + props.put(AUTO_REGISTER_SCHEMAS, true); + kafkaKeyAvroSerializer = new KafkaAvroSerializer(); + kafkaKeyAvroSerializer.configure(props, true); + kafkaValueAvroSerializer = new KafkaAvroSerializer(); + kafkaValueAvroSerializer.configure(props, false); + + // Init kafka key/value schema + Schema.Parser parser = new Schema.Parser(); + allTypesTableKeySchema = parser.parse(ALL_TYPES_TABLE_KEY_SCHEMA); + allTypesTableValueSchema = parser.parse(ALL_TYPES_TABLE_VALUE_SCHEMA); + debeziumSourceSchema = allTypesTableValueSchema.getField("source").schema(); + } + @Test - @Timeout(60) + @Timeout(160) public void testAllTypes() throws Exception { // the first round checks for table creation // the second round checks for running the action on an existing table @@ -68,6 +122,404 @@ private void testAllTypesOnce() throws Exception { client.cancel().get(); } + protected void writeRecordsToKafka(String topic, String resourceDirFormat, Object... args) + throws Exception { + Properties producerProperties = getStandardProps(); + producerProperties.setProperty("retries", "0"); + producerProperties.put("key.serializer", ByteArraySerializer.class.getName()); + producerProperties.put("value.serializer", ByteArraySerializer.class.getName()); + + URL url = + KafkaActionITCaseBase.class + .getClassLoader() + .getResource(String.format(resourceDirFormat, args)); + List lines = + Files.readAllLines(Paths.get(url.toURI())).stream() + .filter(this::isRecordLine) + .collect(Collectors.toList()); + try (KafkaProducer kafkaProducer = + new KafkaProducer<>(producerProperties)) { + for (int i = 0; i < lines.size(); i += 2) { + JsonNode key = objectMapper.readTree(lines.get(i)); + JsonNode value = objectMapper.readTree(lines.get(i + 1)); + JsonNode keyPayload = key.get("payload"); + JsonNode valuePayload = value.get("payload"); + JsonNode source = valuePayload.get("source"); + JsonNode after = valuePayload.get("after"); + + GenericRecord avroKey = new GenericData.Record(allTypesTableKeySchema); + avroKey.put("_id", keyPayload.get("_id").asInt()); + + GenericRecord avroValue = new GenericData.Record(allTypesTableValueSchema); + Schema beforeSchema = allTypesTableValueSchema.getField("before").schema(); + GenericRecord afterAvroValue = + new GenericData.Record(sanitizedSchema(beforeSchema)); + afterAvroValue.put("_id", after.get("_id").asInt()); + afterAvroValue.put("pt", after.get("pt").asText()); + if (nonNullNode(after.get("_tinyint1"))) { + afterAvroValue.put("_tinyint1", after.get("_tinyint1").asInt()); + } + if (nonNullNode(after.get("_boolean"))) { + afterAvroValue.put("_boolean", after.get("_boolean").asBoolean()); + } + if (nonNullNode(after.get("_bool"))) { + afterAvroValue.put("_bool", after.get("_bool").asBoolean()); + } + if (nonNullNode(after.get("_tinyint"))) { + afterAvroValue.put("_tinyint", after.get("_tinyint").asInt()); + } + if (nonNullNode(after.get("_tinyint_unsigned"))) { + afterAvroValue.put("_tinyint_unsigned", after.get("_tinyint_unsigned").asInt()); + } + if (nonNullNode(after.get("_tinyint_unsigned_zerofill"))) { + afterAvroValue.put( + "_tinyint_unsigned_zerofill", + after.get("_tinyint_unsigned_zerofill").asInt()); + } + if (nonNullNode(after.get("_smallint"))) { + afterAvroValue.put("_smallint", after.get("_smallint").asInt()); + } + if (nonNullNode(after.get("_smallint_unsigned"))) { + afterAvroValue.put( + "_smallint_unsigned", after.get("_smallint_unsigned").asInt()); + } + if (nonNullNode(after.get("_smallint_unsigned_zerofill"))) { + afterAvroValue.put( + "_smallint_unsigned_zerofill", + after.get("_smallint_unsigned_zerofill").asInt()); + } + if (nonNullNode(after.get("_mediumint"))) { + afterAvroValue.put("_mediumint", after.get("_mediumint").asInt()); + } + if (nonNullNode(after.get("_mediumint_unsigned"))) { + afterAvroValue.put( + "_mediumint_unsigned", after.get("_mediumint_unsigned").asInt()); + } + if (nonNullNode(after.get("_mediumint_unsigned_zerofill"))) { + afterAvroValue.put( + "_mediumint_unsigned_zerofill", + after.get("_mediumint_unsigned_zerofill").asInt()); + } + if (nonNullNode(after.get("_int"))) { + afterAvroValue.put("_int", after.get("_int").asInt()); + } + if (nonNullNode(after.get("_int_unsigned"))) { + afterAvroValue.put("_int_unsigned", after.get("_int_unsigned").asLong()); + } + if (nonNullNode(after.get("_int_unsigned_zerofill"))) { + afterAvroValue.put( + "_int_unsigned_zerofill", after.get("_int_unsigned_zerofill").asLong()); + } + if (nonNullNode(after.get("_bigint"))) { + afterAvroValue.put("_bigint", after.get("_bigint").asLong()); + } + if (nonNullNode(after.get("_bigint_unsigned"))) { + afterAvroValue.put("_bigint_unsigned", after.get("_bigint_unsigned").asLong()); + } + if (nonNullNode(after.get("_bigint_unsigned_zerofill"))) { + afterAvroValue.put( + "_bigint_unsigned_zerofill", + after.get("_bigint_unsigned_zerofill").asLong()); + } + afterAvroValue.put("_serial", after.get("_serial").asLong()); + if (nonNullNode(after.get("_float"))) { + afterAvroValue.put("_float", after.get("_float").floatValue()); + } + if (nonNullNode(after.get("_float_unsigned"))) { + afterAvroValue.put( + "_float_unsigned", after.get("_float_unsigned").floatValue()); + } + if (nonNullNode(after.get("_float_unsigned_zerofill"))) { + afterAvroValue.put( + "_float_unsigned_zerofill", + after.get("_float_unsigned_zerofill").floatValue()); + } + if (nonNullNode(after.get("_real"))) { + afterAvroValue.put("_real", after.get("_real").floatValue()); + } + if (nonNullNode(after.get("_real_unsigned"))) { + afterAvroValue.put("_real_unsigned", after.get("_real_unsigned").floatValue()); + } + if (nonNullNode(after.get("_real_unsigned_zerofill"))) { + afterAvroValue.put( + "_real_unsigned_zerofill", + after.get("_real_unsigned_zerofill").floatValue()); + } + if (nonNullNode(after.get("_double"))) { + afterAvroValue.put("_double", after.get("_double").asDouble()); + } + if (nonNullNode(after.get("_double_unsigned"))) { + afterAvroValue.put( + "_double_unsigned", after.get("_double_unsigned").asDouble()); + } + if (nonNullNode(after.get("_double_unsigned_zerofill"))) { + afterAvroValue.put( + "_double_unsigned_zerofill", + after.get("_double_unsigned_zerofill").asDouble()); + } + if (nonNullNode(after.get("_double_precision"))) { + afterAvroValue.put( + "_double_precision", after.get("_double_precision").asDouble()); + } + if (nonNullNode(after.get("_double_precision_unsigned"))) { + afterAvroValue.put( + "_double_precision_unsigned", + after.get("_double_precision_unsigned").asDouble()); + } + if (nonNullNode(after.get("_double_precision_unsigned_zerofill"))) { + afterAvroValue.put( + "_double_precision_unsigned_zerofill", + after.get("_double_precision_unsigned_zerofill").asDouble()); + } + // Decimal types + if (nonNullNode(after.get("_numeric"))) { + afterAvroValue.put("_numeric", after.get("_numeric").asText()); + } + if (nonNullNode(after.get("_numeric_unsigned"))) { + afterAvroValue.put( + "_numeric_unsigned", after.get("_numeric_unsigned").asText()); + } + if (nonNullNode(after.get("_numeric_unsigned_zerofill"))) { + afterAvroValue.put( + "_numeric_unsigned_zerofill", + after.get("_numeric_unsigned_zerofill").asText()); + } + if (nonNullNode(after.get("_fixed"))) { + afterAvroValue.put("_fixed", after.get("_fixed").asText()); + } + if (nonNullNode(after.get("_fixed_unsigned"))) { + afterAvroValue.put("_fixed_unsigned", after.get("_fixed_unsigned").asText()); + } + if (nonNullNode(after.get("_fixed_unsigned_zerofill"))) { + afterAvroValue.put( + "_fixed_unsigned_zerofill", + after.get("_fixed_unsigned_zerofill").asText()); + } + if (nonNullNode(after.get("_decimal"))) { + afterAvroValue.put("_decimal", after.get("_decimal").asText()); + } + if (nonNullNode(after.get("_decimal_unsigned"))) { + afterAvroValue.put( + "_decimal_unsigned", after.get("_decimal_unsigned").asText()); + } + if (nonNullNode(after.get("_decimal_unsigned_zerofill"))) { + afterAvroValue.put( + "_decimal_unsigned_zerofill", + after.get("_decimal_unsigned_zerofill").asText()); + } + // Date types + if (nonNullNode(after.get("_date"))) { + afterAvroValue.put("_date", after.get("_date").asInt()); + } + if (nonNullNode(after.get("_datetime"))) { + afterAvroValue.put("_datetime", after.get("_datetime").asLong()); + } + if (nonNullNode(after.get("_datetime3"))) { + afterAvroValue.put("_datetime3", after.get("_datetime3").asLong()); + } + if (nonNullNode(after.get("_datetime6"))) { + afterAvroValue.put("_datetime6", after.get("_datetime6").asLong()); + } + if (nonNullNode(after.get("_datetime_p"))) { + afterAvroValue.put("_datetime_p", after.get("_datetime_p").asLong()); + } + if (nonNullNode(after.get("_datetime_p2"))) { + afterAvroValue.put("_datetime_p2", after.get("_datetime_p2").asLong()); + } + if (nonNullNode(after.get("_timestamp"))) { + afterAvroValue.put("_timestamp", after.get("_timestamp").asText()); + } + if (nonNullNode(after.get("_timestamp0"))) { + afterAvroValue.put("_timestamp0", after.get("_timestamp0").asText()); + } + // String types + if (nonNullNode(after.get("_char"))) { + afterAvroValue.put("_char", after.get("_char").asText()); + } + if (nonNullNode(after.get("_varchar"))) { + afterAvroValue.put("_varchar", after.get("_varchar").asText()); + } + if (nonNullNode(after.get("_tinytext"))) { + afterAvroValue.put("_tinytext", after.get("_tinytext").asText()); + } + if (nonNullNode(after.get("_text"))) { + afterAvroValue.put("_text", after.get("_text").asText()); + } + if (nonNullNode(after.get("_mediumtext"))) { + afterAvroValue.put("_mediumtext", after.get("_mediumtext").asText()); + } + if (nonNullNode(after.get("_longtext"))) { + afterAvroValue.put("_longtext", after.get("_longtext").asText()); + } + // Bytes + if (nonNullNode(after.get("_bin"))) { + afterAvroValue.put( + "_bin", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_bin").asText()))); + } + if (nonNullNode(after.get("_varbin"))) { + afterAvroValue.put( + "_varbin", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_varbin").asText()))); + } + if (nonNullNode(after.get("_tinyblob"))) { + afterAvroValue.put( + "_tinyblob", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_tinyblob").asText()))); + } + if (nonNullNode(after.get("_blob"))) { + afterAvroValue.put( + "_blob", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_blob").asText()))); + } + if (nonNullNode(after.get("_mediumblob"))) { + afterAvroValue.put( + "_mediumblob", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_mediumblob").asText()))); + } + if (nonNullNode(after.get("_longblob"))) { + afterAvroValue.put( + "_longblob", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(after.get("_longblob").asText()))); + } + // Json + if (nonNullNode(after.get("_json"))) { + afterAvroValue.put("_json", after.get("_json").asText()); + } + // Enum + if (nonNullNode(after.get("_enum"))) { + afterAvroValue.put("_enum", after.get("_enum").asText()); + } + + if (nonNullNode(after.get("_year"))) { + afterAvroValue.put("_year", after.get("_year").asInt()); + } + if (nonNullNode(after.get("_time"))) { + afterAvroValue.put("_time", after.get("_time").asLong()); + } + // Point + JsonNode pointJsonValue = after.get("_point"); + if (nonNullNode(pointJsonValue)) { + Schema pointSchema = + sanitizedSchema( + sanitizedSchema(beforeSchema).getField("_point").schema()); + afterAvroValue.put("_point", buildPointRecord(pointJsonValue, pointSchema)); + } + // Geometry + JsonNode geometryJsonValue = after.get("_geometry"); + Schema geometrySchema = + sanitizedSchema( + sanitizedSchema(beforeSchema).getField("_geometry").schema()); + if (nonNullNode(geometryJsonValue)) { + afterAvroValue.put( + "_geometry", buildGeometryRecord(geometryJsonValue, geometrySchema)); + } + + JsonNode linestringJsonNode = after.get("_linestring"); + if (nonNullNode(linestringJsonNode)) { + afterAvroValue.put( + "_linestring", buildGeometryRecord(linestringJsonNode, geometrySchema)); + } + + JsonNode polygonJsonNode = after.get("_polygon"); + if (nonNullNode(polygonJsonNode)) { + afterAvroValue.put( + "_polygon", buildGeometryRecord(polygonJsonNode, geometrySchema)); + } + + JsonNode multipointJsonNode = after.get("_multipoint"); + if (nonNullNode(multipointJsonNode)) { + afterAvroValue.put( + "_multipoint", buildGeometryRecord(multipointJsonNode, geometrySchema)); + } + + JsonNode multilineJsonNode = after.get("_multiline"); + if (nonNullNode(multilineJsonNode)) { + afterAvroValue.put( + "_multiline", buildGeometryRecord(multilineJsonNode, geometrySchema)); + } + + JsonNode multipolygonJsonNode = after.get("_multipolygon"); + if (nonNullNode(multipolygonJsonNode)) { + afterAvroValue.put( + "_multipolygon", + buildGeometryRecord(multipolygonJsonNode, geometrySchema)); + } + + JsonNode geometrycollectionJsonNode = after.get("_geometrycollection"); + if (nonNullNode(geometrycollectionJsonNode)) { + afterAvroValue.put( + "_geometrycollection", + buildGeometryRecord(geometrycollectionJsonNode, geometrySchema)); + } + // Set + if (nonNullNode(after.get("_set"))) { + afterAvroValue.put("_set", after.get("_set").asText()); + } + + avroValue.put("after", afterAvroValue); + // Common properties + avroValue.put("source", buildDebeziumSourceProperty(debeziumSourceSchema, source)); + avroValue.put("op", valuePayload.get("op").asText()); + avroValue.put("ts_ms", valuePayload.get("ts_ms").asLong()); + + // Write to kafka + kafkaProducer.send( + new ProducerRecord<>( + topic, + kafkaKeyAvroSerializer.serialize(topic, avroKey), + kafkaValueAvroSerializer.serialize(topic, avroValue))); + } + } + } + + private boolean nonNullNode(JsonNode jsonNode) { + return jsonNode != null && jsonNode.getNodeType() != JsonNodeType.NULL; + } + + private GenericRecord buildPointRecord(JsonNode pointJsonValue, Schema debeziumPointSchema) { + GenericRecord pointAvroValue = new GenericData.Record(debeziumPointSchema); + pointAvroValue.put("x", pointJsonValue.get("x").asDouble()); + pointAvroValue.put("y", pointJsonValue.get("y").asDouble()); + pointAvroValue.put( + "wkb", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(pointJsonValue.get("wkb").asText()))); + pointAvroValue.put( + "srid", + pointJsonValue.get("srid") != null ? pointJsonValue.get("srid").asInt() : null); + return pointAvroValue; + } + + private GenericRecord buildGeometryRecord( + JsonNode geometryJsonValue, Schema debeziumGeometrySchema) { + GenericRecord geometryAvroValue = new GenericData.Record(debeziumGeometrySchema); + geometryAvroValue.put( + "wkb", + ByteBuffer.wrap( + Base64Variants.getDefaultVariant() + .decode(geometryJsonValue.get("wkb").asText()))); + geometryAvroValue.put( + "srid", + geometryJsonValue.get("srid") != null + ? geometryJsonValue.get("srid").asInt() + : null); + return geometryAvroValue; + } + /** For all types test case. */ protected void testAllTypesImpl() throws Exception { RowType rowType = @@ -115,14 +567,14 @@ protected void testAllTypesImpl() throws Exception { DataTypes.DECIMAL(8, 0), // _decimal DataTypes.DECIMAL(8, 0), // _decimal_unsigned DataTypes.DECIMAL(8, 0), // _decimal_unsigned_zerofill - DataTypes.INT(), // _date - DataTypes.BIGINT(), // _datetime - DataTypes.BIGINT(), // _datetime3 - DataTypes.BIGINT(), // _datetime6 - DataTypes.BIGINT(), // _datetime_p - DataTypes.BIGINT(), // _datetime_p2 - DataTypes.STRING(), // _timestamp - DataTypes.STRING(), // _timestamp0 + DataTypes.DATE(), // _date + DataTypes.TIMESTAMP(0), // _datetime + DataTypes.TIMESTAMP(3), // _datetime3 + DataTypes.TIMESTAMP(6), // _datetime6 + DataTypes.TIMESTAMP(0), // _datetime_p + DataTypes.TIMESTAMP(2), // _datetime_p2 + DataTypes.TIMESTAMP(6), // _timestamp + DataTypes.TIMESTAMP(0), // _timestamp0 DataTypes.CHAR(10), // _char DataTypes.VARCHAR(20), // _varchar DataTypes.STRING(), // _tinytext @@ -138,7 +590,7 @@ protected void testAllTypesImpl() throws Exception { DataTypes.STRING(), // _json DataTypes.STRING(), // _enum DataTypes.INT(), // _year - DataTypes.BIGINT(), // _time + DataTypes.TIME(), // _time DataTypes.STRING(), // _point DataTypes.STRING(), // _geometry DataTypes.STRING(), // _linestring @@ -245,11 +697,11 @@ protected void testAllTypesImpl() throws Exception { + "11111, 22222, 33333, " + "19439, " // display value of datetime is not affected by timezone - + "1679581805000, 1679581805123, 1679581805123456, " - + "1679668200000, 1679668205120, " - + "2023-03-23T07:00:10.123456Z, 2023-03-22T16:10:00Z, " + + "2023-03-23T14:30:05, 2023-03-23T14:30:05.123, 2023-03-23T14:30:05.123456, " + + "2023-03-24T14:30, 2023-03-24T14:30:05.120, " + + "2023-03-23T15:00:10.123456, 2023-03-23T00:10, " + "Paimon, Apache Paimon, Apache Paimon MySQL TINYTEXT Test Data, Apache Paimon MySQL Test Data, Apache Paimon MySQL MEDIUMTEXT Test Data, Apache Paimon MySQL Long Test Data, " - + "[98, 121, 116, 101, 115, 0, 0, 0, 0, 0], " + + "[98, 121, 116, 101, 115], " + "[109, 111, 114, 101, 32, 98, 121, 116, 101, 115], " + "[84, 73, 78, 89, 66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + "[66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " @@ -258,7 +710,7 @@ protected void testAllTypesImpl() throws Exception { + "{\"a\": \"b\"}, " + "value1, " + "2023, " - + "36803000000, " + + "36803000, " + "{\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}, " + "{\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, " + "{\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0}, " @@ -306,4 +758,42 @@ protected void testAllTypesImpl() throws Exception { waitForResult(expected, table, rowType, Arrays.asList("pt", "_id")); } + + private Schema sanitizedSchema(Schema schema) { + if (schema.getType() == Schema.Type.UNION + && schema.getTypes().size() == 2 + && schema.getTypes().contains(NULL_AVRO_SCHEMA)) { + for (Schema memberSchema : schema.getTypes()) { + if (!memberSchema.equals(NULL_AVRO_SCHEMA)) { + return memberSchema; + } + } + } + return schema; + } + + private GenericRecord buildDebeziumSourceProperty(Schema sourceSchema, JsonNode sourceValue) { + GenericRecord source = new GenericData.Record(sourceSchema); + source.put("version", sourceValue.get("version").asText()); + source.put("connector", sourceValue.get("connector").asText()); + source.put("name", sourceValue.get("name").asText()); + source.put("ts_ms", sourceValue.get("ts_ms").asLong()); + source.put("snapshot", sourceValue.get("snapshot").asText()); + source.put("db", sourceValue.get("db").asText()); + source.put( + "sequence", + sourceValue.get("sequence") == null ? null : sourceValue.get("sequence").asText()); + source.put("table", sourceValue.get("table").asText()); + source.put("server_id", sourceValue.get("server_id").asLong()); + source.put( + "gtid", sourceValue.get("gtid") == null ? null : sourceValue.get("gtid").asText()); + source.put("file", sourceValue.get("file").asText()); + source.put("pos", sourceValue.get("pos").asLong()); + source.put("row", sourceValue.get("row").asInt()); + source.put("thread", sourceValue.get("thread").asLong()); + source.put( + "query", + sourceValue.get("query") == null ? null : sourceValue.get("query").asText()); + return source; + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt index 1b58a6e2c905..5fe51e5e9a67 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium-avro/table/schema/alltype/debezium-avro-data-1.txt @@ -16,4 +16,7 @@ * limitations under the License. */ +{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":1}} {"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":1,"pt":"1.1","_tinyint1":1,"_boolean":true,"_bool":false,"_tinyint":1,"_tinyint_unsigned":2,"_tinyint_unsigned_zerofill":3,"_smallint":1000,"_smallint_unsigned":2000,"_smallint_unsigned_zerofill":3000,"_mediumint":100000,"_mediumint_unsigned":200000,"_mediumint_unsigned_zerofill":300000,"_int":1000000,"_int_unsigned":2000000,"_int_unsigned_zerofill":3000000,"_bigint":10000000000,"_bigint_unsigned":20000000000,"_bigint_unsigned_zerofill":30000000000,"_serial":40000000000,"_float":1.5,"_float_unsigned":2.5,"_float_unsigned_zerofill":3.5,"_real":1.000001,"_real_unsigned":2.000002,"_real_unsigned_zerofill":3.000003,"_double":1.000011,"_double_unsigned":2.000022,"_double_unsigned_zerofill":3.000033,"_double_precision":1.000111,"_double_precision_unsigned":2.000222,"_double_precision_unsigned_zerofill":3.000333,"_numeric":"12345.110","_numeric_unsigned":"12345.220","_numeric_unsigned_zerofill":"12345.330","_fixed":"123456789876543212345678987654321.110","_fixed_unsigned":"123456789876543212345678987654321.220","_fixed_unsigned_zerofill":"123456789876543212345678987654321.330","_decimal":"11111","_decimal_unsigned":"22222","_decimal_unsigned_zerofill":"33333","_date":19439,"_datetime":1679581805000,"_datetime3":1679581805123,"_datetime6":1679581805123456,"_datetime_p":1679668200000,"_datetime_p2":1679668205120,"_timestamp":"2023-03-23T07:00:10.123456Z","_timestamp0":"2023-03-22T16:10:00Z","_char":"Paimon","_varchar":"Apache Paimon","_tinytext":"Apache Paimon MySQL TINYTEXT Test Data","_text":"Apache Paimon MySQL Test Data","_mediumtext":"Apache Paimon MySQL MEDIUMTEXT Test Data","_longtext":"Apache Paimon MySQL Long Test Data","_bin":"Ynl0ZXMAAAAAAA==","_varbin":"bW9yZSBieXRlcw==","_tinyblob":"VElOWUJMT0IgdHlwZSB0ZXN0IGRhdGE=","_blob":"QkxPQiB0eXBlIHRlc3QgZGF0YQ==","_mediumblob":"TUVESVVNQkxPQiB0eXBlIHRlc3QgZGF0YQ==","_longblob":"TE9OR0JMT0IgIGJ5dGVzIHRlc3QgZGF0YQ==","_json":"{\"a\": \"b\"}","_enum":"value1","_year":2023,"_time":36803000000,"_point":{"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},"_geometry":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_linestring":{"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},"_polygon":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},"_multipoint":{"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},"_multiline":{"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},"_multipolygon":{"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},"_geometrycollection":{"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null},"_set":"a,b"},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695606891000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210576","file":"mysql-bin.000074","pos":13593,"row":0,"thread":42,"query":null},"op":"c","ts_ms":1695606891883,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"_id"}],"optional":false,"name":"test_json.workdb.all_types_table.Key"},"payload":{"_id":2}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_id"},"field":"_id"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"2","__debezium.source.column.scale":"1","__debezium.source.column.name":"pt"},"field":"pt"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.length":"1","__debezium.source.column.name":"_tinyint1"},"field":"_tinyint1"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOLEAN","__debezium.source.column.name":"_boolean"},"field":"_boolean"},{"type":"boolean","optional":true,"parameters":{"__debezium.source.column.type":"BOOL","__debezium.source.column.name":"_bool"},"field":"_bool"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT","__debezium.source.column.name":"_tinyint"},"field":"_tinyint"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned"},"field":"_tinyint_unsigned"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"TINYINT UNSIGNED ZEROFILL","__debezium.source.column.length":"2","__debezium.source.column.name":"_tinyint_unsigned_zerofill"},"field":"_tinyint_unsigned_zerofill"},{"type":"int16","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT","__debezium.source.column.name":"_smallint"},"field":"_smallint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED","__debezium.source.column.name":"_smallint_unsigned"},"field":"_smallint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"SMALLINT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_smallint_unsigned_zerofill"},"field":"_smallint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT","__debezium.source.column.name":"_mediumint"},"field":"_mediumint"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED","__debezium.source.column.name":"_mediumint_unsigned"},"field":"_mediumint_unsigned"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMINT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_mediumint_unsigned_zerofill"},"field":"_mediumint_unsigned_zerofill"},{"type":"int32","optional":true,"parameters":{"__debezium.source.column.type":"INT","__debezium.source.column.name":"_int"},"field":"_int"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED","__debezium.source.column.name":"_int_unsigned"},"field":"_int_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"INT UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.name":"_int_unsigned_zerofill"},"field":"_int_unsigned_zerofill"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT","__debezium.source.column.name":"_bigint"},"field":"_bigint"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_bigint_unsigned"},"field":"_bigint_unsigned"},{"type":"int64","optional":true,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED ZEROFILL","__debezium.source.column.length":"16","__debezium.source.column.name":"_bigint_unsigned_zerofill"},"field":"_bigint_unsigned_zerofill"},{"type":"int64","optional":false,"parameters":{"__debezium.source.column.type":"BIGINT UNSIGNED","__debezium.source.column.name":"_serial"},"field":"_serial"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT","__debezium.source.column.name":"_float"},"field":"_float"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED","__debezium.source.column.name":"_float_unsigned"},"field":"_float_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"FLOAT UNSIGNED ZEROFILL","__debezium.source.column.length":"4","__debezium.source.column.name":"_float_unsigned_zerofill"},"field":"_float_unsigned_zerofill"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL","__debezium.source.column.name":"_real"},"field":"_real"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED","__debezium.source.column.name":"_real_unsigned"},"field":"_real_unsigned"},{"type":"float","optional":true,"parameters":{"__debezium.source.column.type":"REAL UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_real_unsigned_zerofill"},"field":"_real_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE","__debezium.source.column.name":"_double"},"field":"_double"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED","__debezium.source.column.name":"_double_unsigned"},"field":"_double_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_unsigned_zerofill"},"field":"_double_unsigned_zerofill"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION","__debezium.source.column.name":"_double_precision"},"field":"_double_precision"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED","__debezium.source.column.name":"_double_precision_unsigned"},"field":"_double_precision_unsigned"},{"type":"double","optional":true,"parameters":{"__debezium.source.column.type":"DOUBLE PRECISION UNSIGNED ZEROFILL","__debezium.source.column.length":"10","__debezium.source.column.scale":"7","__debezium.source.column.name":"_double_precision_unsigned_zerofill"},"field":"_double_precision_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric"},"field":"_numeric"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned"},"field":"_numeric_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"NUMERIC UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"3","__debezium.source.column.name":"_numeric_unsigned_zerofill"},"field":"_numeric_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed"},"field":"_fixed"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned"},"field":"_fixed_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"FIXED UNSIGNED ZEROFILL","__debezium.source.column.length":"40","__debezium.source.column.scale":"3","__debezium.source.column.name":"_fixed_unsigned_zerofill"},"field":"_fixed_unsigned_zerofill"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal"},"field":"_decimal"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned"},"field":"_decimal_unsigned"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"DECIMAL UNSIGNED ZEROFILL","__debezium.source.column.length":"8","__debezium.source.column.scale":"0","__debezium.source.column.name":"_decimal_unsigned_zerofill"},"field":"_decimal_unsigned_zerofill"},{"type":"int32","optional":true,"name":"io.debezium.time.Date","version":1,"parameters":{"__debezium.source.column.type":"DATE","__debezium.source.column.name":"_date"},"field":"_date"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime"},"field":"_datetime"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"3","__debezium.source.column.name":"_datetime3"},"field":"_datetime3"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTimestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"6","__debezium.source.column.name":"_datetime6"},"field":"_datetime6"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.name":"_datetime_p"},"field":"_datetime_p"},{"type":"int64","optional":true,"name":"io.debezium.time.Timestamp","version":1,"parameters":{"__debezium.source.column.type":"DATETIME","__debezium.source.column.length":"2","__debezium.source.column.name":"_datetime_p2"},"field":"_datetime_p2"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.length":"6","__debezium.source.column.name":"_timestamp"},"field":"_timestamp"},{"type":"string","optional":true,"name":"io.debezium.time.ZonedTimestamp","version":1,"parameters":{"__debezium.source.column.type":"TIMESTAMP","__debezium.source.column.name":"_timestamp0"},"field":"_timestamp0"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"CHAR","__debezium.source.column.length":"10","__debezium.source.column.name":"_char"},"field":"_char"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"VARCHAR","__debezium.source.column.length":"20","__debezium.source.column.name":"_varchar"},"field":"_varchar"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TINYTEXT","__debezium.source.column.name":"_tinytext"},"field":"_tinytext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"TEXT","__debezium.source.column.name":"_text"},"field":"_text"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMTEXT","__debezium.source.column.name":"_mediumtext"},"field":"_mediumtext"},{"type":"string","optional":true,"parameters":{"__debezium.source.column.type":"LONGTEXT","__debezium.source.column.name":"_longtext"},"field":"_longtext"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BINARY","__debezium.source.column.length":"10","__debezium.source.column.name":"_bin"},"field":"_bin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"VARBINARY","__debezium.source.column.length":"20","__debezium.source.column.name":"_varbin"},"field":"_varbin"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"TINYBLOB","__debezium.source.column.name":"_tinyblob"},"field":"_tinyblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"BLOB","__debezium.source.column.name":"_blob"},"field":"_blob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"MEDIUMBLOB","__debezium.source.column.name":"_mediumblob"},"field":"_mediumblob"},{"type":"bytes","optional":true,"parameters":{"__debezium.source.column.type":"LONGBLOB","__debezium.source.column.name":"_longblob"},"field":"_longblob"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"parameters":{"__debezium.source.column.type":"JSON","__debezium.source.column.name":"_json"},"field":"_json"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"value1,value2,value3","__debezium.source.column.type":"ENUM","__debezium.source.column.length":"1","__debezium.source.column.name":"_enum"},"field":"_enum"},{"type":"int32","optional":true,"name":"io.debezium.time.Year","version":1,"parameters":{"__debezium.source.column.type":"YEAR","__debezium.source.column.name":"_year"},"field":"_year"},{"type":"int64","optional":true,"name":"io.debezium.time.MicroTime","version":1,"parameters":{"__debezium.source.column.type":"TIME","__debezium.source.column.name":"_time"},"field":"_time"},{"type":"struct","fields":[{"type":"double","optional":false,"field":"x"},{"type":"double","optional":false,"field":"y"},{"type":"bytes","optional":true,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Point","version":1,"doc":"Geometry (POINT)","parameters":{"__debezium.source.column.type":"POINT","__debezium.source.column.name":"_point"},"field":"_point"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRY","__debezium.source.column.name":"_geometry"},"field":"_geometry"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"LINESTRING","__debezium.source.column.name":"_linestring"},"field":"_linestring"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"POLYGON","__debezium.source.column.name":"_polygon"},"field":"_polygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOINT","__debezium.source.column.name":"_multipoint"},"field":"_multipoint"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTILINESTRING","__debezium.source.column.name":"_multiline"},"field":"_multiline"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"MULTIPOLYGON","__debezium.source.column.name":"_multipolygon"},"field":"_multipolygon"},{"type":"struct","fields":[{"type":"bytes","optional":false,"field":"wkb"},{"type":"int32","optional":true,"field":"srid"}],"optional":true,"name":"io.debezium.data.geometry.Geometry","version":1,"doc":"Geometry","parameters":{"__debezium.source.column.type":"GEOMETRYCOLLECTION","__debezium.source.column.name":"_geometrycollection"},"field":"_geometrycollection"},{"type":"string","optional":true,"name":"io.debezium.data.EnumSet","version":1,"parameters":{"allowed":"a,b,c,d","__debezium.source.column.type":"SET","__debezium.source.column.length":"7","__debezium.source.column.name":"_set"},"field":"_set"}],"optional":true,"name":"test_json.workdb.all_types_table.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"name":"event.block","version":1,"field":"transaction"}],"optional":false,"name":"test_json.workdb.all_types_table.Envelope","version":1},"payload":{"before":null,"after":{"_id":2,"pt":"2.2","_tinyint1":null,"_boolean":null,"_bool":null,"_tinyint":null,"_tinyint_unsigned":null,"_tinyint_unsigned_zerofill":null,"_smallint":null,"_smallint_unsigned":null,"_smallint_unsigned_zerofill":null,"_mediumint":null,"_mediumint_unsigned":null,"_mediumint_unsigned_zerofill":null,"_int":null,"_int_unsigned":null,"_int_unsigned_zerofill":null,"_bigint":null,"_bigint_unsigned":null,"_bigint_unsigned_zerofill":null,"_serial":50000000000,"_float":null,"_float_unsigned":null,"_float_unsigned_zerofill":null,"_real":null,"_real_unsigned":null,"_real_unsigned_zerofill":null,"_double":null,"_double_unsigned":null,"_double_unsigned_zerofill":null,"_double_precision":null,"_double_precision_unsigned":null,"_double_precision_unsigned_zerofill":null,"_numeric":null,"_numeric_unsigned":null,"_numeric_unsigned_zerofill":null,"_fixed":null,"_fixed_unsigned":null,"_fixed_unsigned_zerofill":null,"_decimal":null,"_decimal_unsigned":null,"_decimal_unsigned_zerofill":null,"_date":null,"_datetime":null,"_datetime3":null,"_datetime6":null,"_datetime_p":null,"_datetime_p2":null,"_timestamp":null,"_timestamp0":null,"_char":null,"_varchar":null,"_tinytext":null,"_text":null,"_mediumtext":null,"_longtext":null,"_bin":null,"_varbin":null,"_tinyblob":null,"_blob":null,"_mediumblob":null,"_longblob":null,"_json":null,"_enum":null,"_year":null,"_time":null,"_point":null,"_geometry":null,"_linestring":null,"_polygon":null,"_multipoint":null,"_multiline":null,"_multipolygon":null,"_geometrycollection":null,"_set":null},"source":{"version":"2.4.0-SNAPSHOT","connector":"mysql","name":"test_json","ts_ms":1695126006000,"snapshot":"false","db":"workdb","sequence":null,"table":"all_types_table","server_id":999999,"gtid":"b8642e5a-af4d-11ed-b484-98f6094952c0:4210568","file":"mysql-bin.000073","pos":337176435,"row":0,"thread":279,"query":null},"op":"c","ts_ms":1695126006877,"transaction":null}} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/log4j2-test.properties b/paimon-flink/paimon-flink-cdc/src/test/resources/log4j2-test.properties index 1b3980d15104..e27922dad60b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/log4j2-test.properties +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/log4j2-test.properties @@ -18,7 +18,7 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -rootLogger.level = OFF +rootLogger.level = INFO rootLogger.appenderRef.test.ref = TestLogger appender.testlogger.name = TestLogger From 579293ebc718b5d6817d8e44fdccd20f3f566ac5 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Thu, 27 Jun 2024 19:41:26 +0800 Subject: [PATCH 11/12] fix test --- .../debezium/DebeziumAvroRecordParser.java | 29 ++++++---- .../format/debezium/DebeziumSchemaUtils.java | 56 ++++++++++++------- ...afkaDebeziumAvroDeserializationSchema.java | 26 +++++++-- ...lsarDebeziumAvroDeserializationSchema.java | 22 ++++++-- .../flink/action/cdc/CdcActionITCaseBase.java | 2 - ...afkaDebeziumAvroSyncTableActionITCase.java | 7 ++- 6 files changed, 93 insertions(+), 49 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java index 6d0eddef2774..f89183d6d34d 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java @@ -26,7 +26,6 @@ import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; -import io.debezium.data.geometry.Geometry; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; @@ -34,7 +33,6 @@ import javax.annotation.Nullable; -import java.nio.ByteBuffer; import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Collections; @@ -42,8 +40,10 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.CONNECT_NAME_PROP; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_AFTER; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_BEFORE; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_DB; @@ -55,7 +55,9 @@ import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_READE; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_TRUNCATE; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.OP_UPDATE; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.SCHEMA_PARAMETER_COLUMN_NAME; import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.avroToPaimonDataType; +import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.getAvroConnectParameters; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** @@ -132,20 +134,25 @@ private Map extractRowData( LinkedHashMap resultMap = new LinkedHashMap<>(); for (Schema.Field field : payloadSchema.getFields()) { Schema schema = sanitizedSchema(field.schema()); - String fieldName = field.name(); + Map connectParameters = getAvroConnectParameters(schema); + + String fieldName = + Optional.of(schema) + .filter(s -> s.getType() == Schema.Type.RECORD) + .map(s -> field.name()) + .orElseGet( + () -> + connectParameters.getOrDefault( + SCHEMA_PARAMETER_COLUMN_NAME, field.name())); String rawValue = Objects.toString(record.get(fieldName), null); + String className = schema.getProp(CONNECT_NAME_PROP); String transformed = - DebeziumSchemaUtils.transformRawValue( + DebeziumSchemaUtils.transformAvroRawValue( rawValue, - schema.getLogicalType() != null - ? schema.getLogicalType().getName() - : null, schema.getFullName(), + className, typeMapping, - () -> - (ByteBuffer) - ((GenericRecord) record.get(fieldName)) - .get(Geometry.WKB_FIELD), + record.get(fieldName), ZoneOffset.UTC); resultMap.put(fieldName, transformed); rowTypeBuilder.field(fieldName, avroToPaimonDataType(schema)); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java index 97a0ba43462d..59838ebf2ccb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java @@ -37,6 +37,7 @@ import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.kafka.connect.json.JsonConverterConfig; import javax.annotation.Nullable; @@ -49,8 +50,8 @@ import java.time.ZoneId; import java.time.ZoneOffset; import java.util.Base64; +import java.util.HashMap; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.function.Supplier; @@ -103,6 +104,27 @@ public static String transformRawValue( serverTimeZone); } + public static String transformAvroRawValue( + @Nullable String rawValue, + String debeziumType, + @Nullable String className, + TypeMapping typeMapping, + Object origin, + ZoneId serverTimeZone) { + + if (rawValue != null && "bytes".equals(debeziumType) && className == null) { + return new String(((ByteBuffer) origin).array()); + } + + return transformRawValue( + rawValue, + debeziumType, + className, + typeMapping, + () -> (ByteBuffer) ((GenericRecord) origin).get(Geometry.WKB_FIELD), + serverTimeZone); + } + /** Transform raw string value according to schema. */ public static String transformRawValue( @Nullable String rawValue, @@ -301,33 +323,25 @@ public static String decimalLogicalName() { protected static final String CONNECT_PARAMETERS_PROP = "connect.parameters"; protected static final String CONNECT_NAME_PROP = "connect.name"; - private static final String POINT_LOGICAL_NAME = "io.debezium.data.geometry.Point"; - private static final String GEOMETRY_LOGICAL_NAME = "io.debezium.data.geometry.Geometry"; - private static final String ENUM_SET_LOGICAL_NAME = "io.debezium.data.EnumSet"; - private static final String DATE_SCHEMA_NAME = "io.debezium.time.Date"; - private static final String TIMESTAMP_SCHEMA_NAME = "io.debezium.time.Timestamp"; - private static final String MICRO_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.MicroTimestamp"; - private static final String NANO_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.NanoTimestamp"; - private static final String TIME_SCHEMA_NAME = "io.debezium.time.Time"; - private static final String MICRO_TIME_SCHEMA_NAME = "io.debezium.time.MicroTime"; - private static final String NANO_TIME_SCHEMA_NAME = "io.debezium.time.NanoTime"; - private static final String ZONED_TIME_SCHEMA_NAME = "io.debezium.time.ZonedTime"; - private static final String ZONED_TIMESTAMP_SCHEMA_NAME = "io.debezium.time.ZonedTimestamp"; - private static final String DECIMAL_PRECISE_SCHEMA_NAME = - "org.apache.kafka.connect.data.Decimal"; - private static final String SCHEMA_PARAMETER_COLUMN_TYPE = "__debezium.source.column.type"; + static final String SCHEMA_PARAMETER_COLUMN_TYPE = "__debezium.source.column.type"; private static final String SCHEMA_PARAMETER_COLUMN_SIZE = "__debezium.source.column.length"; private static final String SCHEMA_PARAMETER_COLUMN_PRECISION = "__debezium.source.column.scale"; - private static final String SCHEMA_PARAMETER_COLUMN_NAME = "__debezium.source.column.name"; + static final String SCHEMA_PARAMETER_COLUMN_NAME = "__debezium.source.column.name"; - public static DataType avroToPaimonDataType(Schema schema) { + public static Map getAvroConnectParameters(Schema schema) { // Mapping by mysql types // Parse actual source column type from connect.parameters if enable debezium property // "column.propagate.source.type", otherwise will infer avro schema type mapping to paimon - Map connectParameters = - (Map) schema.getObjectProp(CONNECT_PARAMETERS_PROP); - if (Objects.nonNull(connectParameters)) { + if (schema.getObjectProp(CONNECT_PARAMETERS_PROP) != null) { + return (Map) schema.getObjectProp(CONNECT_PARAMETERS_PROP); + } + return new HashMap<>(); + } + + public static DataType avroToPaimonDataType(Schema schema) { + Map connectParameters = getAvroConnectParameters(schema); + if (!connectParameters.isEmpty()) { String typeName = connectParameters.getOrDefault( SCHEMA_PARAMETER_COLUMN_TYPE, schema.getType().name()); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index 305e55bfbcf0..78c298b99b71 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -24,6 +24,7 @@ import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; import io.confluent.kafka.serializers.GenericContainerWithVersion; import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; @@ -43,17 +44,19 @@ public class KafkaDebeziumAvroDeserializationSchema private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; private final String topic; + private final String schemaRegistryUrl; /** The deserializer to deserialize Debezium Avro data. */ - private final ConfluentAvroDeserializationSchema avroDeserializer; + private ConfluentAvroDeserializationSchema avroDeserializer; public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); - String schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); - this.avroDeserializer = - new ConfluentAvroDeserializationSchema( - new CachedSchemaRegistryClient( - schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + } + + @Override + public void open(DeserializationSchema.InitializationContext context) throws Exception { + initAvroDeserializer(); } @Override @@ -63,6 +66,10 @@ public CdcSourceRecord deserialize(ConsumerRecord message) throw return null; } + if (this.avroDeserializer == null) { + initAvroDeserializer(); + } + GenericContainerWithVersion keyContainerWithVersion = this.avroDeserializer.deserialize(topic, true, message.key()); GenericContainerWithVersion valueContainerWithVersion = @@ -84,4 +91,11 @@ public boolean isEndOfStream(CdcSourceRecord nextElement) { public TypeInformation getProducedType() { return getForClass(CdcSourceRecord.class); } + + private void initAvroDeserializer() { + this.avroDeserializer = + new ConfluentAvroDeserializationSchema( + new CachedSchemaRegistryClient( + schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java index 042c7574d87a..7adf7f38da00 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.action.cdc.serialization.ConfluentAvroDeserializationSchema; import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.serializers.GenericContainerWithVersion; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -53,10 +54,7 @@ public PulsarDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { @Override public void open(InitializationContext context) throws Exception { - avroDeserializer = - new ConfluentAvroDeserializationSchema( - new CachedSchemaRegistryClient( - schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + initAvroDeserializer(); } @Override @@ -65,8 +63,13 @@ public CdcSourceRecord deserialize(byte[] message) throws IOException { return null; } - GenericRecord value = - (GenericRecord) this.avroDeserializer.deserialize(topic, false, message); + if (this.avroDeserializer == null) { + initAvroDeserializer(); + } + + GenericContainerWithVersion valueContainerWithVersion = + this.avroDeserializer.deserialize(topic, false, message); + GenericRecord value = (GenericRecord) valueContainerWithVersion.container(); return new CdcSourceRecord(topic, null, value); } @@ -79,4 +82,11 @@ public boolean isEndOfStream(CdcSourceRecord nextElement) { public TypeInformation getProducedType() { return getForClass(CdcSourceRecord.class); } + + private void initAvroDeserializer() { + this.avroDeserializer = + new ConfluentAvroDeserializationSchema( + new CachedSchemaRegistryClient( + schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 5e1d240dd5b3..468a3074d61b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -125,8 +125,6 @@ protected void waitForResult( boolean sameType = field.type().equals(rowType.getFieldTypes().get(i)); if (sameName && sameType) { cnt++; - } else { - System.out.println("ss"); } } if (cnt == rowType.getFieldCount()) { diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java index 2dff8e92f55a..d4e9b6bac0d1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java @@ -90,7 +90,7 @@ public void setup() { } @Test - @Timeout(160) + @Timeout(60) public void testAllTypes() throws Exception { // the first round checks for table creation // the second round checks for running the action on an existing table @@ -699,9 +699,10 @@ protected void testAllTypesImpl() throws Exception { // display value of datetime is not affected by timezone + "2023-03-23T14:30:05, 2023-03-23T14:30:05.123, 2023-03-23T14:30:05.123456, " + "2023-03-24T14:30, 2023-03-24T14:30:05.120, " - + "2023-03-23T15:00:10.123456, 2023-03-23T00:10, " + // TODO haven't handle zone + + "2023-03-23T07:00:10.123456, 2023-03-22T16:10, " + "Paimon, Apache Paimon, Apache Paimon MySQL TINYTEXT Test Data, Apache Paimon MySQL Test Data, Apache Paimon MySQL MEDIUMTEXT Test Data, Apache Paimon MySQL Long Test Data, " - + "[98, 121, 116, 101, 115], " + + "[98, 121, 116, 101, 115, 0, 0, 0, 0, 0], " + "[109, 111, 114, 101, 32, 98, 121, 116, 101, 115], " + "[84, 73, 78, 89, 66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " + "[66, 76, 79, 66, 32, 116, 121, 112, 101, 32, 116, 101, 115, 116, 32, 100, 97, 116, 97], " From 971a233beae8bb3e5d6d3dbd1f0b9b29595de772 Mon Sep 17 00:00:00 2001 From: zhuangchong Date: Thu, 27 Jun 2024 20:03:44 +0800 Subject: [PATCH 12/12] add doc --- docs/content/flink/cdc-ingestion/kafka-cdc.md | 25 ++++++++++++++++++- .../content/flink/cdc-ingestion/pulsar-cdc.md | 9 +++++-- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/content/flink/cdc-ingestion/kafka-cdc.md b/docs/content/flink/cdc-ingestion/kafka-cdc.md index 5b16a1d6f8e4..93655daf2455 100644 --- a/docs/content/flink/cdc-ingestion/kafka-cdc.md +++ b/docs/content/flink/cdc-ingestion/kafka-cdc.md @@ -33,7 +33,7 @@ flink-sql-connector-kafka-*.jar ``` ## Supported Formats -Flink provides several Kafka CDC formats: Canal, Debezium, Ogg, Maxwell and Normal JSON. +Flink provides several Kafka CDC formats: Canal Json, Debezium Json, Debezium Avro, Ogg Json, Maxwell Json and Normal Json. If a message in a Kafka topic is a change event captured from another database using the Change Data Capture (CDC) tool, then you can use the Paimon Kafka CDC. Write the INSERT, UPDATE, DELETE messages parsed into the paimon table. @@ -252,3 +252,26 @@ Synchronization from multiple Kafka topics to Paimon database. --table_conf changelog-producer=input \ --table_conf sink.parallelism=4 ``` + +## Additional kafka_config + +There are some useful options to build Flink Kafka Source, but they are not provided by flink-kafka-connector document. They are: + +
+ + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
schema.registry.url(none)StringWhen configuring "value.format=debezium-avro" which requires using the Confluence schema registry model for Apache Avro serialization, you need to provide the schema registry URL.
diff --git a/docs/content/flink/cdc-ingestion/pulsar-cdc.md b/docs/content/flink/cdc-ingestion/pulsar-cdc.md index c068f2fd2a96..aa29cf3c3ef7 100644 --- a/docs/content/flink/cdc-ingestion/pulsar-cdc.md +++ b/docs/content/flink/cdc-ingestion/pulsar-cdc.md @@ -33,7 +33,7 @@ flink-connector-pulsar-*.jar ``` ## Supported Formats -Flink provides several Pulsar CDC formats: Canal, Debezium, Ogg, Maxwell and Normal JSON. +Flink provides several Pulsar CDC formats: Canal Json, Debezium Json, Debezium Avro, Ogg Json, Maxwell Json and Normal Json. If a message in a pulsar topic is a change event captured from another database using the Change Data Capture (CDC) tool, then you can use the Paimon Pulsar CDC. Write the INSERT, UPDATE, DELETE messages parsed into the paimon table. @@ -352,6 +352,11 @@ There are some useful options to build Flink Pulsar Source, but they are not pro + + + + + +
Boolean To specify the boundedness of a stream.
schema.registry.url(none)StringWhen configuring "value.format=debezium-avro" which requires using the Confluence schema registry model for Apache Avro serialization, you need to provide the schema registry URL.
-