paimonFields) {
- return new RichCdcMultiplexRecord(
- getDatabaseName(),
- getTableName(),
- paimonFields,
- extractPrimaryKeys(),
- 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()
@@ -219,21 +137,19 @@ protected JsonNode mergeOldRecord(JsonNode data, JsonNode oldNode) {
}
@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();
}
- 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(
@@ -262,6 +178,4 @@ protected JsonNode getAndCheck(String key, String conditionKey, String condition
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/AbstractRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/AbstractRecordParser.java
new file mode 100644
index 000000000000..8b8946a99a6e
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/AbstractRecordParser.java
@@ -0,0 +1,149 @@
+/*
+ * 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.CdcRecord;
+import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * 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 AbstractRecordParser
+ implements FlatMapFunction {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AbstractRecordParser.class);
+
+ protected static final String FIELD_TABLE = "table";
+ protected static final String FIELD_DATABASE = "database";
+ protected final TypeMapping typeMapping;
+ protected final List computedColumns;
+
+ public AbstractRecordParser(TypeMapping typeMapping, List computedColumns) {
+ this.typeMapping = typeMapping;
+ this.computedColumns = computedColumns;
+ }
+
+ @Nullable
+ public Schema buildSchema(CdcSourceRecord record) {
+ try {
+ setRoot(record);
+ if (isDDL()) {
+ return null;
+ }
+
+ Optional recordOpt = extractRecords().stream().findFirst();
+ if (!recordOpt.isPresent()) {
+ return null;
+ }
+
+ Schema.Builder builder = Schema.newBuilder();
+ recordOpt
+ .get()
+ .fields()
+ .forEach(
+ field ->
+ builder.column(
+ field.name(), field.type(), field.description()));
+ builder.primaryKey(extractPrimaryKeys());
+ return builder.build();
+ } catch (Exception e) {
+ logInvalidSourceRecord(record);
+ throw e;
+ }
+ }
+
+ @Override
+ public void flatMap(CdcSourceRecord value, Collector out) {
+ try {
+ setRoot(value);
+ extractRecords().forEach(out::collect);
+ } catch (Exception e) {
+ logInvalidSourceRecord(value);
+ throw e;
+ }
+ }
+
+ protected abstract void setRoot(CdcSourceRecord record);
+
+ protected abstract List extractRecords();
+
+ protected boolean isDDL() {
+ return false;
+ }
+
+ protected abstract List extractPrimaryKeys();
+
+ /** generate values for computed columns. */
+ protected void evalComputedColumns(
+ Map rowData, RowType.Builder rowTypeBuilder) {
+ computedColumns.forEach(
+ computedColumn -> {
+ rowData.put(
+ computedColumn.columnName(),
+ computedColumn.eval(rowData.get(computedColumn.fieldReference())));
+ rowTypeBuilder.field(computedColumn.columnName(), computedColumn.columnType());
+ });
+ }
+
+ /** Handle case sensitivity here. */
+ protected RichCdcMultiplexRecord createRecord(
+ RowKind rowKind, Map data, List paimonFields) {
+ return new RichCdcMultiplexRecord(
+ getDatabaseName(),
+ getTableName(),
+ paimonFields,
+ extractPrimaryKeys(),
+ new CdcRecord(rowKind, data));
+ }
+
+ @Nullable
+ protected abstract String getTableName();
+
+ @Nullable
+ protected abstract String getDatabaseName();
+
+ private void logInvalidSourceRecord(CdcSourceRecord record) {
+ LOG.error("Invalid source record:\n{}", record.toString());
+ }
+
+ protected abstract String format();
+}
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 2af44c9022cc..4f4c504a61cd 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,49 +18,106 @@
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;
-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.json.JsonRecordParser;
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
- * associated {@link RecordParser}.
+ * associated {@link AbstractRecordParser}.
*
* Each data format is associated with a specific implementation of {@link RecordParserFactory},
- * which can be used to create instances of {@link RecordParser} for that format.
+ * which can be used to create instances of {@link AbstractRecordParser} for that format.
*/
public enum DataFormat {
- CANAL_JSON(CanalRecordParser::new),
- OGG_JSON(OggRecordParser::new),
- MAXWELL_JSON(MaxwellRecordParser::new),
- DEBEZIUM_JSON(DebeziumRecordParser::new),
- JSON(JsonRecordParser::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),
+ JSON(
+ JsonRecordParser::new,
+ KafkaDebeziumJsonDeserializationSchema::new,
+ CdcJsonDeserializationSchema::new);
+
// 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;
- DataFormat(RecordParserFactory parser) {
+ DataFormat(
+ RecordParserFactory parser,
+ Function> kafkaDeserializer,
+ Function> pulsarDeserializer) {
this.parser = parser;
+ this.kafkaDeserializer = kafkaDeserializer;
+ this.pulsarDeserializer = pulsarDeserializer;
}
/**
- * Creates a new instance of {@link RecordParser} for this data format with the specified
- * configurations.
+ * Creates a new instance of {@link AbstractRecordParser} for this data format with the
+ * specified configurations.
*
* @param computedColumns List of computed columns to be considered by the parser.
- * @return A new instance of {@link RecordParser}.
+ * @return A new instance of {@link AbstractRecordParser}.
*/
- public RecordParser createParser(
+ public AbstractRecordParser createParser(
TypeMapping typeMapping, List computedColumns) {
return parser.createParser(typeMapping, computedColumns);
}
+ public KafkaDeserializationSchema createKafkaDeserializer(
+ Configuration cdcSourceConfig) {
+ return kafkaDeserializer.apply(cdcSourceConfig);
+ }
+
+ public DeserializationSchema createPulsarDeserializer(
+ Configuration cdcSourceConfig) {
+ 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/main/java/org/apache/paimon/flink/action/cdc/format/RecordParserFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParserFactory.java
index 612dfefd687f..52b7e87fe494 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParserFactory.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/RecordParserFactory.java
@@ -24,22 +24,23 @@
import java.util.List;
/**
- * Represents a factory for creating instances of {@link RecordParser}.
+ * Represents a factory for creating instances of {@link AbstractRecordParser}.
*
* This interface provides a method to create a new RecordParser with specific configurations
* such as case sensitivity, table name conversion, and computed columns.
*
- * @see RecordParser
+ * @see AbstractRecordParser
*/
@FunctionalInterface
public interface RecordParserFactory {
/**
- * Creates a new instance of {@link RecordParser} with the specified configurations.
+ * Creates a new instance of {@link AbstractRecordParser} with the specified configurations.
*
* @param typeMapping Data type mapping options.
* @param computedColumns List of computed columns to be considered by the parser.
- * @return A new instance of {@link RecordParser}.
+ * @return A new instance of {@link AbstractRecordParser}.
*/
- RecordParser createParser(TypeMapping typeMapping, List computedColumns);
+ AbstractRecordParser createParser(
+ TypeMapping typeMapping, List computedColumns);
}
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 aae4aab82a8d..170cea4d7fb2 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.AbstractJsonRecordParser;
import org.apache.paimon.flink.action.cdc.mysql.MySqlTypeUtils;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.types.DataType;
@@ -65,7 +65,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 AbstractJsonRecordParser {
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
new file mode 100644
index 000000000000..f89183d6d34d
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumAvroRecordParser.java
@@ -0,0 +1,209 @@
+/*
+ * 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.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.AbstractRecordParser;
+import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+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 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;
+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.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;
+
+/**
+ * Implementation of {@link AbstractRecordParser} 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 AbstractRecordParser {
+ private static final Logger LOG = LoggerFactory.getLogger(DebeziumAvroRecordParser.class);
+
+ private static final Schema NULL_AVRO_SCHEMA = Schema.create(Schema.Type.NULL);
+
+ private GenericRecord keyRecord;
+ private GenericRecord valueRecord;
+
+ public DebeziumAvroRecordParser(TypeMapping typeMapping, List computedColumns) {
+ super(typeMapping, computedColumns);
+ }
+
+ @Override
+ protected void setRoot(CdcSourceRecord record) {
+ keyRecord = (GenericRecord) record.getKey();
+ valueRecord = (GenericRecord) record.getValue();
+ }
+
+ @Override
+ protected List extractRecords() {
+ String operation = getAndCheck(FIELD_TYPE).toString();
+ List records = new ArrayList<>();
+ switch (operation) {
+ case OP_READE:
+ case OP_INSERT:
+ processRecord((GenericRecord) getAndCheck(FIELD_AFTER), RowKind.INSERT, records);
+ break;
+ case OP_UPDATE:
+ processRecord((GenericRecord) getAndCheck(FIELD_BEFORE), RowKind.DELETE, records);
+ processRecord((GenericRecord) getAndCheck(FIELD_AFTER), RowKind.INSERT, records);
+ break;
+ case OP_DELETE:
+ processRecord((GenericRecord) getAndCheck(FIELD_BEFORE), RowKind.DELETE, records);
+ break;
+ case OP_TRUNCATE:
+ case OP_MESSAGE:
+ LOG.info("Skip record operation: {}", operation);
+ break;
+ default:
+ throw new UnsupportedOperationException("Unknown record operation: " + operation);
+ }
+
+ return records;
+ }
+
+ private void processRecord(
+ GenericRecord record, RowKind rowKind, List records) {
+ RowType.Builder rowTypeBuilder = RowType.builder();
+ Map rowData = this.extractRowData(record, rowTypeBuilder);
+ records.add(createRecord(rowKind, rowData, rowTypeBuilder.build().getFields()));
+ }
+
+ @Override
+ 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, RowType.Builder rowTypeBuilder) {
+ Schema payloadSchema = sanitizedSchema(record.getSchema());
+
+ LinkedHashMap resultMap = new LinkedHashMap<>();
+ for (Schema.Field field : payloadSchema.getFields()) {
+ Schema schema = sanitizedSchema(field.schema());
+ 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.transformAvroRawValue(
+ rawValue,
+ schema.getFullName(),
+ className,
+ typeMapping,
+ record.get(fieldName),
+ ZoneOffset.UTC);
+ resultMap.put(fieldName, transformed);
+ rowTypeBuilder.field(fieldName, avroToPaimonDataType(schema));
+ }
+
+ evalComputedColumns(resultMap, rowTypeBuilder);
+ return resultMap;
+ }
+
+ @Override
+ protected String format() {
+ return "debezium-avro";
+ }
+
+ 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;
+ }
+
+ @Nullable
+ @Override
+ protected String getTableName() {
+ return getFromSourceField(FIELD_TABLE);
+ }
+
+ @Nullable
+ @Override
+ protected String getDatabaseName() {
+ return getFromSourceField(FIELD_DB);
+ }
+
+ @Nullable
+ private String getFromSourceField(String key) {
+ GenericRecord source = (GenericRecord) valueRecord.get(FIELD_SOURCE);
+ if (Objects.isNull(source)) {
+ return null;
+ }
+ return source.get(key).toString();
+ }
+
+ 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/DebeziumRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumJsonRecordParser.java
similarity index 81%
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 c2b6587547e4..6349fc6a95d2 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
@@ -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.AbstractJsonRecordParser;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.types.RowKind;
import org.apache.paimon.types.RowType;
@@ -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 AbstractJsonRecordParser} 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,27 +75,14 @@
* operation type, and primary key field names are used to construct the details of each record
* event.
*/
-public class DebeziumRecordParser 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 AbstractJsonRecordParser {
private boolean hasSchema;
private final Map debeziumTypes = new HashMap<>();
private final Map classNames = new HashMap<>();
private final Map> parameters = new HashMap<>();
- public DebeziumRecordParser(TypeMapping typeMapping, List computedColumns) {
+ public DebeziumJsonRecordParser(TypeMapping typeMapping, List computedColumns) {
super(typeMapping, computedColumns);
}
@@ -146,10 +145,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 e61b33d0ed1e..b705bc9e1d88 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,24 @@
import io.debezium.time.MicroTimestamp;
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;
+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.HashMap;
import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING;
@@ -56,7 +63,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 +87,52 @@ 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);
+ }
+
+ 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,
+ String debeziumType,
+ @Nullable String className,
+ TypeMapping typeMapping,
+ Supplier geometryGetter,
+ ZoneId serverTimeZone) {
if (rawValue == null) {
return null;
}
@@ -158,8 +227,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);
@@ -252,4 +320,69 @@ 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";
+
+ 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";
+ static final String SCHEMA_PARAMETER_COLUMN_NAME = "__debezium.source.column.name";
+
+ 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
+ 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());
+ 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/json/JsonRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/json/JsonRecordParser.java
index 1a67f82f4ccf..399d38a10564 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/json/JsonRecordParser.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/json/JsonRecordParser.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.AbstractJsonRecordParser;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.types.RowKind;
@@ -28,13 +28,13 @@
import java.util.List;
/**
- * The {@code JsonRecordParser} class extends the abstract {@link RecordParser} class and is
- * designed to parse JSON records.
+ * The {@code JsonRecordParser} class extends the abstract {@link AbstractJsonRecordParser} class
+ * and is designed to parse JSON records.
*
* This class treats JSON records as special CDC data with only insert operation type and
* generates {@link RichCdcMultiplexRecord} objects with only INSERT operation types.
*/
-public class JsonRecordParser extends RecordParser {
+public class JsonRecordParser extends AbstractJsonRecordParser {
public JsonRecordParser(TypeMapping typeMapping, List computedColumns) {
super(typeMapping, computedColumns);
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 0c9393ee6ebf..a90088a41916 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.AbstractJsonRecordParser;
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 AbstractJsonRecordParser} 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 AbstractJsonRecordParser {
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 bc1efdf7c100..3ec45fc2f44d 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.AbstractJsonRecordParser;
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 AbstractJsonRecordParser} 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 AbstractJsonRecordParser {
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..78c298b99b71
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java
@@ -0,0 +1,101 @@
+/*
+ * 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 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;
+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 {
+ initAvroDeserializer();
+ }
+
+ @Override
+ public CdcSourceRecord deserialize(ConsumerRecord message) throws IOException {
+ if (message.value() == null) {
+ // skip tombstone messages
+ return null;
+ }
+
+ if (this.avroDeserializer == null) {
+ initAvroDeserializer();
+ }
+
+ 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);
+ }
+
+ @Override
+ public boolean isEndOfStream(CdcSourceRecord nextElement) {
+ return false;
+ }
+
+ @Override
+ 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/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 ed5a3afd4b6e..f81046962c39 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
@@ -345,7 +345,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..7adf7f38da00
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java
@@ -0,0 +1,92 @@
+/*
+ * 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 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 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 {
+ initAvroDeserializer();
+ }
+
+ @Override
+ public CdcSourceRecord deserialize(byte[] message) throws IOException {
+ if (message == null) {
+ return null;
+ }
+
+ if (this.avroDeserializer == null) {
+ initAvroDeserializer();
+ }
+
+ GenericContainerWithVersion valueContainerWithVersion =
+ this.avroDeserializer.deserialize(topic, false, message);
+ GenericRecord value = (GenericRecord) valueContainerWithVersion.container();
+ return new CdcSourceRecord(topic, null, value);
+ }
+
+ @Override
+ public boolean isEndOfStream(CdcSourceRecord nextElement) {
+ return false;
+ }
+
+ @Override
+ 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/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 getBasicKafkaConfig() {
return config;
}
+ protected String getSchemaRegistryUrl() {
+ return SCHEMA_REGISTRY_CONTAINER.getSchemaRegistryUrl();
+ }
+
protected KafkaSyncTableActionBuilder syncTableActionBuilder(Map kafkaConfig) {
return new KafkaSyncTableActionBuilder(kafkaConfig);
}
@@ -311,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()
@@ -319,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);
@@ -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..d4e9b6bac0d1
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroSyncTableActionITCase.java
@@ -0,0 +1,800 @@
+/*
+ * 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.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)
+ 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();
+ }
+
+ 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 =
+ 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.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
+ 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.TIME(), // _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
+ + "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, "
+ // 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, 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, "
+ + "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}, "
+ + "{\"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"));
+ }
+
+ 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/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 cb33f6259525..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
@@ -56,7 +56,8 @@ public void testKafkaSchema() 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/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java
index 5f7df79e48ed..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
@@ -486,7 +486,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<>();
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..5fe51e5e9a67
--- /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}}
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