yunfengzhou-hub commented on code in PR #6358:
URL: https://github.com/apache/paimon/pull/6358#discussion_r2488470036


##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.apache.paimon.data.InternalRow;
+
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypeChecks;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/** Converter from Paimon to Flink CDC data. */
+public class PaimonToFlinkCDCDataConverter {
+
+    /** Convert Paimon row to Flink CDC data. */
+    public static DataChangeEvent convertRowToDataChangeEvent(
+            TableId tableId,
+            InternalRow row,
+            List<InternalRow.FieldGetter> fieldGetters,
+            BinaryRecordDataGenerator recordDataGenerator) {
+        Object[] objects = new Object[row.getFieldCount()];
+        for (int i = 0; i < row.getFieldCount(); i++) {
+            objects[i] = fieldGetters.get(i).getFieldOrNull(row);
+        }
+        BinaryRecordData binaryRecordData = 
recordDataGenerator.generate(objects);
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                {
+                    return DataChangeEvent.insertEvent(tableId, 
binaryRecordData, new HashMap<>());

Review Comment:
   Should it be `DataChangeEvent.updateEvent`? Same for the `UPDATE_BEFORE` 
below.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.apache.paimon.data.InternalRow;
+
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypeChecks;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/** Converter from Paimon to Flink CDC data. */
+public class PaimonToFlinkCDCDataConverter {
+
+    /** Convert Paimon row to Flink CDC data. */
+    public static DataChangeEvent convertRowToDataChangeEvent(
+            TableId tableId,
+            InternalRow row,
+            List<InternalRow.FieldGetter> fieldGetters,
+            BinaryRecordDataGenerator recordDataGenerator) {
+        Object[] objects = new Object[row.getFieldCount()];
+        for (int i = 0; i < row.getFieldCount(); i++) {
+            objects[i] = fieldGetters.get(i).getFieldOrNull(row);
+        }
+        BinaryRecordData binaryRecordData = 
recordDataGenerator.generate(objects);
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                {
+                    return DataChangeEvent.insertEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            case DELETE:
+            case UPDATE_BEFORE:
+                {
+                    return DataChangeEvent.deleteEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            default:
+                throw new IllegalArgumentException("don't support type of " + 
row.getRowKind());

Review Comment:
   nit: Capitalize the first letter of don't. Same as 
`FlinkCDCToPaimonDataConverter`.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.apache.paimon.data.InternalRow;
+
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypeChecks;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/** Converter from Paimon to Flink CDC data. */
+public class PaimonToFlinkCDCDataConverter {
+
+    /** Convert Paimon row to Flink CDC data. */
+    public static DataChangeEvent convertRowToDataChangeEvent(
+            TableId tableId,
+            InternalRow row,
+            List<InternalRow.FieldGetter> fieldGetters,
+            BinaryRecordDataGenerator recordDataGenerator) {
+        Object[] objects = new Object[row.getFieldCount()];
+        for (int i = 0; i < row.getFieldCount(); i++) {
+            objects[i] = fieldGetters.get(i).getFieldOrNull(row);
+        }
+        BinaryRecordData binaryRecordData = 
recordDataGenerator.generate(objects);
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                {
+                    return DataChangeEvent.insertEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            case DELETE:
+            case UPDATE_BEFORE:
+                {
+                    return DataChangeEvent.deleteEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            default:
+                throw new IllegalArgumentException("don't support type of " + 
row.getRowKind());
+        }
+    }
+
+    public static List<InternalRow.FieldGetter> 
createFieldGetters(List<DataType> fieldTypes) {
+        List<InternalRow.FieldGetter> fieldGetters = new ArrayList<>();
+        for (int i = 0; i < fieldTypes.size(); i++) {
+            fieldGetters.add(createFieldGetter(fieldTypes.get(i), i));
+        }
+        return fieldGetters;
+    }
+
+    public static InternalRow.FieldGetter createFieldGetter(DataType 
fieldType, int fieldPos) {
+        final InternalRow.FieldGetter fieldGetter;
+        // ordered by type root definition
+        switch (fieldType.getTypeRoot()) {
+            case CHAR:
+            case VARCHAR:
+                fieldGetter =
+                        row -> 
BinaryStringData.fromString(row.getString(fieldPos).toString());
+                break;
+            case BOOLEAN:
+                fieldGetter = row -> row.getBoolean(fieldPos);
+                break;
+            case BINARY:
+            case VARBINARY:
+                fieldGetter = row -> row.getBinary(fieldPos);
+                break;
+            case DECIMAL:
+                final int decimalPrecision = 
DataTypeChecks.getPrecision(fieldType);
+                final int decimalScale = DataTypeChecks.getScale(fieldType);
+                fieldGetter =
+                        row ->
+                                DecimalData.fromBigDecimal(
+                                        row.getDecimal(fieldPos, 
decimalPrecision, decimalScale)
+                                                .toBigDecimal(),
+                                        decimalPrecision,
+                                        decimalScale);
+                break;
+            case TINYINT:
+                fieldGetter = row -> row.getByte(fieldPos);
+                break;
+            case SMALLINT:
+                fieldGetter = row -> row.getShort(fieldPos);
+                break;
+            case BIGINT:
+                fieldGetter = row -> row.getLong(fieldPos);
+                break;
+            case FLOAT:
+                fieldGetter = row -> row.getFloat(fieldPos);
+                break;
+            case DOUBLE:
+                fieldGetter = row -> row.getDouble(fieldPos);
+                break;
+            case INTEGER:
+                fieldGetter = row -> row.getInt(fieldPos);
+                break;
+            case DATE:
+                fieldGetter = row -> row.getInt(fieldPos);
+                break;
+            case TIME_WITHOUT_TIME_ZONE:
+                fieldGetter = row -> row.getInt(fieldPos);
+                break;
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                fieldGetter =
+                        row ->
+                                TimestampData.fromTimestamp(
+                                        row.getTimestamp(
+                                                        fieldPos,
+                                                        
DataTypeChecks.getPrecision(fieldType))
+                                                .toSQLTimestamp());
+                break;
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                fieldGetter =
+                        row ->
+                                LocalZonedTimestampData.fromInstant(
+                                        row.getTimestamp(
+                                                        fieldPos,
+                                                        
DataTypeChecks.getPrecision(fieldType))
+                                                .toInstant());
+                break;
+            default:
+                throw new IllegalArgumentException(
+                        "don't support type of " + fieldType.getTypeRoot());
+        }
+        if (!fieldType.isNullable()) {
+            return fieldGetter;

Review Comment:
   Let's add a test to verify the behavior when values are null but types 
require not null.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/schema/PaimonMetadataApplier.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.pipeline.cdc.schema;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.flink.FlinkCatalogFactory;
+import org.apache.paimon.flink.LogicalTypeConversion;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.schema.SchemaChange;
+import org.apache.paimon.table.Table;
+import org.apache.paimon.table.sink.BatchTableCommit;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.DropTableEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEventType;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.event.TruncateTableEvent;
+import org.apache.flink.cdc.common.event.visitor.SchemaChangeEventVisitor;
+import org.apache.flink.cdc.common.exceptions.SchemaEvolveException;
+import 
org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.sink.MetadataApplier;
+import org.apache.flink.cdc.common.types.utils.DataTypeUtils;
+import org.apache.flink.shaded.guava31.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.cdc.common.utils.Preconditions.checkArgument;
+import static org.apache.flink.cdc.common.utils.Preconditions.checkNotNull;
+
+/** A {@code MetadataApplier} that applies schema changes to Paimon table. */
+public class PaimonMetadataApplier implements MetadataApplier {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(PaimonMetadataApplier.class);
+
+    // Catalog is unSerializable.
+    private transient Catalog catalog;
+
+    // currently, we set table options for all tables using the same options.
+    private final Map<String, String> tableOptions;
+
+    private final Options catalogOptions;
+
+    private final Map<TableId, List<String>> partitionMaps;
+
+    private Set<SchemaChangeEventType> enabledSchemaEvolutionTypes;
+
+    public PaimonMetadataApplier(Options catalogOptions) {
+        this.catalogOptions = catalogOptions;
+        this.tableOptions = new HashMap<>();
+        this.partitionMaps = new HashMap<>();
+        this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes();
+    }
+
+    public PaimonMetadataApplier(
+            Options catalogOptions,
+            Map<String, String> tableOptions,
+            Map<TableId, List<String>> partitionMaps) {
+        this.catalogOptions = catalogOptions;
+        this.tableOptions = tableOptions;
+        this.partitionMaps = partitionMaps;
+        this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes();
+    }
+
+    @Override
+    public MetadataApplier setAcceptedSchemaEvolutionTypes(
+            Set<SchemaChangeEventType> schemaEvolutionTypes) {
+        this.enabledSchemaEvolutionTypes = schemaEvolutionTypes;
+        return this;
+    }
+
+    @Override
+    public boolean acceptsSchemaEvolutionType(SchemaChangeEventType 
schemaChangeEventType) {
+        return enabledSchemaEvolutionTypes.contains(schemaChangeEventType);
+    }
+
+    @Override
+    public Set<SchemaChangeEventType> getSupportedSchemaEvolutionTypes() {
+        return Sets.newHashSet(
+                SchemaChangeEventType.CREATE_TABLE,
+                SchemaChangeEventType.ADD_COLUMN,
+                SchemaChangeEventType.DROP_COLUMN,
+                SchemaChangeEventType.RENAME_COLUMN,
+                SchemaChangeEventType.ALTER_COLUMN_TYPE);
+    }
+
+    @Override
+    public void applySchemaChange(SchemaChangeEvent schemaChangeEvent)
+            throws SchemaEvolveException {
+        if (catalog == null) {
+            catalog = FlinkCatalogFactory.createPaimonCatalog(catalogOptions);
+        }
+        SchemaChangeEventVisitor.visit(
+                schemaChangeEvent,
+                addColumnEvent -> {
+                    applyAddColumn(addColumnEvent);
+                    return null;
+                },
+                alterColumnTypeEvent -> {
+                    applyAlterColumnType(alterColumnTypeEvent);
+                    return null;
+                },
+                createTableEvent -> {
+                    applyCreateTable(createTableEvent);
+                    return null;
+                },
+                dropColumnEvent -> {
+                    applyDropColumn(dropColumnEvent);
+                    return null;
+                },
+                dropTableEvent -> {
+                    applyDropTable(dropTableEvent);
+                    return null;
+                },
+                renameColumnEvent -> {
+                    applyRenameColumn(renameColumnEvent);
+                    return null;
+                },
+                truncateTableEvent -> {
+                    applyTruncateTable(truncateTableEvent);
+                    return null;
+                });
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (catalog != null) {
+            catalog.close();
+        }
+    }
+
+    private void applyCreateTable(CreateTableEvent event) throws 
SchemaEvolveException {
+        try {
+            if 
(!catalog.listDatabases().contains(event.tableId().getSchemaName())) {
+                catalog.createDatabase(event.tableId().getSchemaName(), true);
+            }
+            Schema schema = event.getSchema();
+            org.apache.paimon.schema.Schema.Builder builder =
+                    new org.apache.paimon.schema.Schema.Builder();
+            schema.getColumns()
+                    .forEach(
+                            (column) ->
+                                    builder.column(
+                                            column.getName(),
+                                            LogicalTypeConversion.toDataType(
+                                                    
DataTypeUtils.toFlinkDataType(column.getType())
+                                                            .getLogicalType()),
+                                            column.getComment()));
+            List<String> partitionKeys = new ArrayList<>();
+            List<String> primaryKeys = schema.primaryKeys();
+            if (partitionMaps.containsKey(event.tableId())) {
+                partitionKeys.addAll(partitionMaps.get(event.tableId()));
+            } else if (schema.partitionKeys() != null && 
!schema.partitionKeys().isEmpty()) {
+                partitionKeys.addAll(schema.partitionKeys());
+            }
+            builder.primaryKey(primaryKeys)

Review Comment:
   I noticed that in Flink CDC, this implementation originally looks like this
   ```
               for (String partitionColumn : partitionKeys) {
                   if (!primaryKeys.contains(partitionColumn)) {
                       primaryKeys.add(partitionColumn);
                   }
               }
               builder.partitionKeys(partitionKeys)
                       .primaryKey(primaryKeys)
   ```
   
   Why do we change the implementation here?



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/pipeline/cdc/schema/PaimonMetadataApplierTest.java:
##########
@@ -0,0 +1,609 @@
+/*
+ * 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.pipeline.cdc.schema;
+
+import org.apache.paimon.CoreOptions;
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.FileSystemCatalogFactory;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.flink.FlinkCatalogFactory;
+import org.apache.paimon.flink.pipeline.cdc.util.FlinkCDCToPaimonTypeConverter;
+import org.apache.paimon.options.CatalogOptions;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.table.Table;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.DataTypes;
+import org.apache.paimon.types.RowType;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.exceptions.SchemaEvolveException;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.sink.MetadataApplier;
+import org.apache.flink.cdc.common.types.DataType;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/** Tests for {@link PaimonMetadataApplier}. */
+class PaimonMetadataApplierTest {
+
+    @TempDir public static java.nio.file.Path temporaryFolder;
+
+    private static Catalog catalog;
+
+    private static Options catalogOptions;
+
+    private static MetadataApplier metadataApplier;
+
+    @BeforeAll
+    public static void initialize() {
+        catalogOptions = new Options();
+        catalogOptions.setString(
+                CatalogOptions.METASTORE.key(), 
FileSystemCatalogFactory.IDENTIFIER);
+        catalogOptions.setString(
+                CatalogOptions.WAREHOUSE.key(),
+                new File(temporaryFolder.toFile(), 
UUID.randomUUID().toString()).toString());
+        catalogOptions.set(CatalogOptions.CACHE_ENABLED.key(), "false");
+        catalog = FlinkCatalogFactory.createPaimonCatalog(catalogOptions);
+
+        Map<String, String> tableOptions = new HashMap<>();
+        tableOptions.put(CoreOptions.BUCKET.key(), "-1");
+        metadataApplier = new PaimonMetadataApplier(catalogOptions, 
tableOptions, new HashMap<>());
+    }
+
+    @AfterAll
+    public static void close() throws Exception {
+        if (catalog != null) {
+            catalog.close();
+        }
+        metadataApplier.close();
+    }
+
+    @Test
+    void testApplySchemaChange() throws Catalog.TableNotExistException, 
SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, "table1");
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                                .primaryKey("col1")
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.INT())));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+        List<AddColumnEvent.ColumnWithPosition> addedColumns = new 
ArrayList<>();
+        addedColumns.add(
+                new AddColumnEvent.ColumnWithPosition(
+                        Column.physicalColumn(
+                                "col3",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING(),
+                                null,
+                                "col3DefValue")));
+        AddColumnEvent addColumnEvent = new AddColumnEvent(tableId, 
addedColumns);
+        metadataApplier.applySchemaChange(addColumnEvent);
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.INT()),
+                                new DataField(
+                                        2, "col3", DataTypes.STRING(), null, 
"col3DefValue")));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+
+        Map<String, String> nameMapping = new HashMap<>();
+        nameMapping.put("col2", "newcol2");
+        nameMapping.put("col3", "newcol3");
+        RenameColumnEvent renameColumnEvent = new RenameColumnEvent(tableId, 
nameMapping);
+        metadataApplier.applySchemaChange(renameColumnEvent);
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "newcol2", DataTypes.INT()),
+                                new DataField(
+                                        2, "newcol3", DataTypes.STRING(), 
null, "col3DefValue")));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+
+        Map<String, DataType> typeMapping = new HashMap<>();
+        typeMapping.put("newcol2", 
org.apache.flink.cdc.common.types.DataTypes.STRING());
+        AlterColumnTypeEvent alterColumnTypeEvent =
+                new AlterColumnTypeEvent(TableId.parse(tableId.identifier()), 
typeMapping);
+        metadataApplier.applySchemaChange(alterColumnTypeEvent);
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "newcol2", 
DataTypes.STRING()),
+                                new DataField(
+                                        2, "newcol3", DataTypes.STRING(), 
null, "col3DefValue")));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+
+        DropColumnEvent dropColumnEvent =
+                new DropColumnEvent(tableId, 
Collections.singletonList("newcol2"));
+        metadataApplier.applySchemaChange(dropColumnEvent);
+        // id of DataField should keep the same as before dropping column
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(
+                                        2, "newcol3", DataTypes.STRING(), 
null, "col3DefValue")));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+
+        // Create table with partition column.
+        tableId = TableId.tableId(databaseName, "table_with_partition");
+        createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                                .physicalColumn(
+                                        "dt",
+                                        
org.apache.flink.cdc.common.types.DataTypes.INT().notNull())
+                                .primaryKey("col1", "dt")
+                                .partitionKey("dt")
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.INT()),
+                                new DataField(2, "dt", 
DataTypes.INT().notNull())));
+        Table tableWithPartition = 
catalog.getTable(Identifier.fromString(tableId.identifier()));
+        
Assertions.assertThat(tableWithPartition.rowType()).isEqualTo(tableSchema);
+        Assertions.assertThat(tableWithPartition.primaryKeys())
+                .isEqualTo(Arrays.asList("col1", "dt"));
+        // Create table with upper case.
+        tableId = TableId.tableId(databaseName, "table_with_upper_case");
+        catalogOptions.setString(CatalogOptions.CASE_SENSITIVE.key(), "true");
+        PaimonMetadataApplier anotherMetadataApplier = new 
PaimonMetadataApplier(catalogOptions);
+        createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "COL1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                                .primaryKey("COL1")
+                                .build());
+        anotherMetadataApplier.applySchemaChange(createTableEvent);
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "COL1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.INT())));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+    }
+
+    @Test
+    public void testCreateTableWithoutPrimaryKey()
+            throws Catalog.TableNotExistException, SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, "table1");
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .physicalColumn(
+                                        "col3",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .physicalColumn(
+                                        "col4",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+        Table table = 
catalog.getTable(Identifier.fromString(tableId.identifier()));
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.STRING()),
+                                new DataField(2, "col3", DataTypes.STRING()),
+                                new DataField(3, "col4", DataTypes.STRING())));
+        Assertions.assertThat(table.rowType()).isEqualTo(tableSchema);
+        Assertions.assertThat(table.primaryKeys()).isEmpty();
+        Assertions.assertThat(table.partitionKeys()).isEmpty();
+        
Assertions.assertThat(table.options()).containsEntry(CoreOptions.BUCKET.key(), 
"-1");
+    }
+
+    @Test
+    void testCreateTableWithOptions() throws Catalog.TableNotExistException, 
SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, "table1");
+        Map<TableId, List<String>> partitionMaps = new HashMap<>();
+        partitionMaps.put(tableId, Arrays.asList("col3", "col4"));
+        Map<String, String> tableOptions = new HashMap<>();
+        tableOptions.put(CoreOptions.BUCKET.key(), "-1");
+        MetadataApplier anotherMetadataApplier =
+                new PaimonMetadataApplier(catalogOptions, tableOptions, 
partitionMaps);
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .physicalColumn(
+                                        "col3",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .physicalColumn(
+                                        "col4",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .primaryKey("col1", "col3", "col4")
+                                .build());
+        anotherMetadataApplier.applySchemaChange(createTableEvent);
+        Table table = 
catalog.getTable(Identifier.fromString(tableId.identifier()));
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.STRING()),
+                                new DataField(2, "col3", 
DataTypes.STRING().notNull()),
+                                new DataField(3, "col4", 
DataTypes.STRING().notNull())));
+        Assertions.assertThat(table.rowType()).isEqualTo(tableSchema);
+        
Assertions.assertThat(table.primaryKeys()).isEqualTo(Arrays.asList("col1", 
"col3", "col4"));
+        
Assertions.assertThat(table.partitionKeys()).isEqualTo(Arrays.asList("col3", 
"col4"));
+        
Assertions.assertThat(table.options()).containsEntry(CoreOptions.BUCKET.key(), 
"-1");
+    }
+
+    @Test
+    void testCreateTableWithAllDataTypes()
+            throws Catalog.TableNotExistException, SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, "table1");
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "boolean",
+                                        
org.apache.flink.cdc.common.types.DataTypes.BOOLEAN())
+                                .physicalColumn(
+                                        "binary",
+                                        
org.apache.flink.cdc.common.types.DataTypes.BINARY(3))
+                                .physicalColumn(
+                                        "varbinary",
+                                        
org.apache.flink.cdc.common.types.DataTypes.VARBINARY(10))
+                                .physicalColumn(
+                                        "bytes",
+                                        
org.apache.flink.cdc.common.types.DataTypes.BYTES())
+                                .physicalColumn(
+                                        "tinyint",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TINYINT())
+                                .physicalColumn(
+                                        "smallint",
+                                        
org.apache.flink.cdc.common.types.DataTypes.SMALLINT())
+                                .physicalColumn(
+                                        "int", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                                .physicalColumn(
+                                        "float",
+                                        
org.apache.flink.cdc.common.types.DataTypes.FLOAT())
+                                .physicalColumn(
+                                        "double",
+                                        
org.apache.flink.cdc.common.types.DataTypes.DOUBLE())
+                                .physicalColumn(
+                                        "decimal",
+                                        
org.apache.flink.cdc.common.types.DataTypes.DECIMAL(6, 3))
+                                .physicalColumn(
+                                        "char", 
org.apache.flink.cdc.common.types.DataTypes.CHAR(5))
+                                .physicalColumn(
+                                        "varchar",
+                                        
org.apache.flink.cdc.common.types.DataTypes.VARCHAR(10))
+                                .physicalColumn(
+                                        "string",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                                .physicalColumn(
+                                        "date", 
org.apache.flink.cdc.common.types.DataTypes.DATE())
+                                .physicalColumn(
+                                        "time", 
org.apache.flink.cdc.common.types.DataTypes.TIME())
+                                .physicalColumn(
+                                        "time_with_precision",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TIME(6))
+                                .physicalColumn(
+                                        "timestamp",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP())
+                                .physicalColumn(
+                                        "timestamp_with_precision",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(3))
+                                .physicalColumn(
+                                        "timestamp_ltz",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ())
+                                .physicalColumn(
+                                        "timestamp_ltz_with_precision",
+                                        
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(
+                                                3))
+                                .primaryKey("col1")
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "boolean", 
DataTypes.BOOLEAN()),
+                                new DataField(2, "binary", 
DataTypes.BINARY(3)),
+                                new DataField(3, "varbinary", 
DataTypes.VARBINARY(10)),
+                                new DataField(4, "bytes", DataTypes.BYTES()),
+                                new DataField(5, "tinyint", 
DataTypes.TINYINT()),
+                                new DataField(6, "smallint", 
DataTypes.SMALLINT()),
+                                new DataField(7, "int", DataTypes.INT()),
+                                new DataField(8, "float", DataTypes.FLOAT()),
+                                new DataField(9, "double", DataTypes.DOUBLE()),
+                                new DataField(10, "decimal", 
DataTypes.DECIMAL(6, 3)),
+                                new DataField(11, "char", DataTypes.CHAR(5)),
+                                new DataField(12, "varchar", 
DataTypes.VARCHAR(10)),
+                                new DataField(13, "string", 
DataTypes.STRING()),
+                                new DataField(14, "date", DataTypes.DATE()),
+                                new DataField(15, "time", DataTypes.TIME(0)),
+                                new DataField(16, "time_with_precision", 
DataTypes.TIME(6)),
+                                new DataField(17, "timestamp", 
DataTypes.TIMESTAMP(6)),
+                                new DataField(
+                                        18, "timestamp_with_precision", 
DataTypes.TIMESTAMP(3)),
+                                new DataField(
+                                        19,
+                                        "timestamp_ltz",
+                                        
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6)),
+                                new DataField(
+                                        20,
+                                        "timestamp_ltz_with_precision",
+                                        
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3))));
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+    }
+
+    @Test
+    void testAddColumnWithPosition() throws Catalog.TableNotExistException, 
SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, "table1");
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull())
+                                .physicalColumn(
+                                        "col2", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                                .primaryKey("col1")
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+
+        List<AddColumnEvent.ColumnWithPosition> addedColumns = new 
ArrayList<>();
+        addedColumns.add(
+                new AddColumnEvent.ColumnWithPosition(
+                        Column.physicalColumn(
+                                "col3",
+                                org.apache.flink.cdc.common.types.DataTypes
+                                        .STRING()))); // default last position.
+        AddColumnEvent addColumnEvent = new AddColumnEvent(tableId, 
addedColumns);
+        metadataApplier.applySchemaChange(addColumnEvent);
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(1, "col2", DataTypes.INT()),
+                                new DataField(2, "col3", DataTypes.STRING())));
+
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+
+        addedColumns.clear();
+
+        addedColumns.add(
+                AddColumnEvent.before(
+                        Column.physicalColumn(
+                                "col4_first_before",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING()),
+                        "col1"));
+        addedColumns.add(
+                AddColumnEvent.first(
+                        Column.physicalColumn(
+                                "col4_first",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING())));
+        addedColumns.add(
+                AddColumnEvent.last(
+                        Column.physicalColumn(
+                                "col5_last",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING())));
+        addedColumns.add(
+                AddColumnEvent.before(
+                        Column.physicalColumn(
+                                "col6_before",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING()),
+                        "col2"));
+        addedColumns.add(
+                AddColumnEvent.after(
+                        Column.physicalColumn(
+                                "col7_after", 
org.apache.flink.cdc.common.types.DataTypes.STRING()),
+                        "col2"));
+
+        addColumnEvent = new AddColumnEvent(tableId, addedColumns);
+        metadataApplier.applySchemaChange(addColumnEvent);
+
+        tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(4, "col4_first", 
DataTypes.STRING()),
+                                new DataField(3, "col4_first_before", 
DataTypes.STRING()),
+                                new DataField(0, "col1", 
DataTypes.STRING().notNull()),
+                                new DataField(6, "col6_before", 
DataTypes.STRING()),
+                                new DataField(1, "col2", DataTypes.INT()),
+                                new DataField(7, "col7_after", 
DataTypes.STRING()),
+                                new DataField(2, "col3", DataTypes.STRING()),
+                                new DataField(5, "col5_last", 
DataTypes.STRING())));
+
+        Assertions.assertThat(
+                        
catalog.getTable(Identifier.fromString(tableId.identifier())).rowType())
+                .isEqualTo(tableSchema);
+    }
+
+    @Test
+    public void testCreateTableWithComment()
+            throws Catalog.TableNotExistException, SchemaEvolveException {
+        String databaseName = "test_" + UUID.randomUUID();
+        TableId tableId = TableId.tableId(databaseName, 
"test.table_with_comment");
+        CreateTableEvent createTableEvent =
+                new CreateTableEvent(
+                        tableId,
+                        org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                                .physicalColumn(
+                                        "col1",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING()
+                                                .notNull(),
+                                        "comment of col1")
+                                .physicalColumn(
+                                        "col2",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING(),
+                                        "comment of col2")
+                                .physicalColumn(
+                                        "col3",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING(),
+                                        "comment of col3")
+                                .physicalColumn(
+                                        "col4",
+                                        
org.apache.flink.cdc.common.types.DataTypes.STRING(),
+                                        "comment of col4")
+                                .comment("comment of table_with_comment")
+                                .build());
+        metadataApplier.applySchemaChange(createTableEvent);
+        Table table = 
catalog.getTable(Identifier.fromString(tableId.identifier()));
+        RowType tableSchema =
+                new RowType(
+                        Arrays.asList(
+                                new DataField(
+                                        0, "col1", 
DataTypes.STRING().notNull(), "comment of col1"),
+                                new DataField(1, "col2", DataTypes.STRING(), 
"comment of col2"),
+                                new DataField(2, "col3", DataTypes.STRING(), 
"comment of col3"),
+                                new DataField(3, "col4", DataTypes.STRING(), 
"comment of col4")));
+        Assertions.assertThat(table.rowType()).isEqualTo(tableSchema);
+        Assertions.assertThat(table.primaryKeys()).isEmpty();
+        Assertions.assertThat(table.partitionKeys()).isEmpty();
+        Assertions.assertThat(table.options()).containsEntry("bucket", "-1");
+        Assertions.assertThat(table.comment()).contains("comment of 
table_with_comment");
+    }
+
+    @Test
+    public void testMysqlDefaultTimestampValueConversionInAddColumn()

Review Comment:
   Is "Mysql" a typo? Or maybe better to add a comment describing why this 
paimon test is related to mysql.



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/pipeline/cdc/util/DataConvertTest.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.apache.flink.cdc.common.data.DateData;
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.TimeData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+
+/**
+ * Data convert test for {@link PaimonToFlinkCDCDataConverter} and {@link
+ * FlinkCDCToPaimonDataConverter}.
+ */
+public class DataConvertTest {
+
+    @Test
+    public void testFullTypesConverter() {
+        org.apache.flink.cdc.common.schema.Schema fullTypesSchema =
+                org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                        .physicalColumn(
+                                "pk_string",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING().notNull())
+                        .physicalColumn(
+                                "boolean", 
org.apache.flink.cdc.common.types.DataTypes.BOOLEAN())
+                        .physicalColumn(
+                                "binary", 
org.apache.flink.cdc.common.types.DataTypes.BINARY(3))
+                        .physicalColumn(
+                                "varbinary",
+                                
org.apache.flink.cdc.common.types.DataTypes.VARBINARY(10))
+                        .physicalColumn(
+                                "bytes", 
org.apache.flink.cdc.common.types.DataTypes.BYTES())
+                        .physicalColumn(
+                                "tinyint", 
org.apache.flink.cdc.common.types.DataTypes.TINYINT())
+                        .physicalColumn(
+                                "smallint", 
org.apache.flink.cdc.common.types.DataTypes.SMALLINT())
+                        .physicalColumn("int", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                        .physicalColumn(
+                                "bigint", 
org.apache.flink.cdc.common.types.DataTypes.BIGINT())
+                        .physicalColumn(
+                                "float", 
org.apache.flink.cdc.common.types.DataTypes.FLOAT())
+                        .physicalColumn(
+                                "double", 
org.apache.flink.cdc.common.types.DataTypes.DOUBLE())
+                        .physicalColumn(
+                                "decimal",
+                                
org.apache.flink.cdc.common.types.DataTypes.DECIMAL(6, 3))
+                        .physicalColumn("char", 
org.apache.flink.cdc.common.types.DataTypes.CHAR(5))
+                        .physicalColumn(
+                                "varchar", 
org.apache.flink.cdc.common.types.DataTypes.VARCHAR(10))
+                        .physicalColumn(
+                                "string", 
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                        .physicalColumn("date", 
org.apache.flink.cdc.common.types.DataTypes.DATE())
+                        .physicalColumn("time", 
org.apache.flink.cdc.common.types.DataTypes.TIME())
+                        .physicalColumn(
+                                "time_with_precision",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIME(6))
+                        .physicalColumn(
+                                "timestamp",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP())
+                        .physicalColumn(
+                                "timestamp_with_precision_3",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(3))
+                        .physicalColumn(
+                                "timestamp_with_precision_6",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(6))
+                        .physicalColumn(
+                                "timestamp_with_precision_9",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(9))
+                        .physicalColumn(
+                                "timestamp_ltz",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ())
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_3",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(3))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_6",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(6))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_9",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(9))
+                        .primaryKey("pk_string")
+                        .partitionKey("boolean")
+                        .build();
+        TableId tableId = TableId.tableId("testDatabase", "testTable");
+        BinaryRecordDataGenerator recordDataGenerator =
+                new BinaryRecordDataGenerator(
+                        fullTypesSchema.getColumnDataTypes().toArray(new 
DataType[0]));
+        Object[] testData =
+                new Object[] {
+                    BinaryStringData.fromString("pk_string"),
+                    true,
+                    new byte[] {1, 2, 3},
+                    new byte[] {4, 5, 6},
+                    new byte[] {7, 8, 9},
+                    (byte) 1,
+                    (short) 2,
+                    3,
+                    4L,
+                    5.1f,
+                    6.2,
+                    DecimalData.fromBigDecimal(new BigDecimal("7.123"), 6, 3),
+                    BinaryStringData.fromString("test1"),
+                    BinaryStringData.fromString("test2"),
+                    BinaryStringData.fromString("test3"),
+                    DateData.fromEpochDay(1000),
+                    TimeData.fromMillisOfDay(200),
+                    TimeData.fromMillisOfDay(300).toMillisOfDay(),
+                    TimestampData.fromMillis(100, 1),
+                    TimestampData.fromMillis(200, 2),
+                    TimestampData.fromMillis(300, 3),
+                    TimestampData.fromMillis(400, 4),
+                    LocalZonedTimestampData.fromEpochMillis(300, 3),
+                    LocalZonedTimestampData.fromEpochMillis(400, 4),
+                    LocalZonedTimestampData.fromEpochMillis(500, 5),
+                    LocalZonedTimestampData.fromEpochMillis(600, 6),
+                };
+        org.apache.flink.cdc.common.event.DataChangeEvent dataChangeEvent =
+                DataChangeEvent.insertEvent(tableId, 
recordDataGenerator.generate(testData));
+
+        Assertions.assertEquals(

Review Comment:
   Let's also verify the converted data in the middle phase, after the original 
CDC data is converted to paimon data and before it's converted back.



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java:
##########
@@ -298,7 +298,7 @@ public void testNewlyAddedTablesOptionsChange() throws 
Exception {
         JobClient jobClient = runActionWithDefaultEnv(action1);
 
         waitingTables("t3");
-        jobClient.cancel();
+        jobClient.cancel().get();

Review Comment:
   Better add a timeout to avoid infinite blocking. Same for other test cases.



##########
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

Review Comment:
   Maybe need to revert this change.



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/pipeline/cdc/util/TypeConverterTest.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Type converter test for {@link FlinkCDCToPaimonTypeConverter} and {@link
+ * PaimonToFlinkCDCTypeConverter}.
+ */
+public class TypeConverterTest {
+
+    @Test
+    public void testFullTypesConverter() {
+        org.apache.flink.cdc.common.schema.Schema fullTypesSchema =
+                org.apache.flink.cdc.common.schema.Schema.newBuilder()
+                        .physicalColumn(
+                                "pk_string",
+                                
org.apache.flink.cdc.common.types.DataTypes.STRING().notNull())
+                        .physicalColumn(
+                                "boolean", 
org.apache.flink.cdc.common.types.DataTypes.BOOLEAN())
+                        .physicalColumn(
+                                "binary", 
org.apache.flink.cdc.common.types.DataTypes.BINARY(3))
+                        .physicalColumn(
+                                "varbinary",
+                                
org.apache.flink.cdc.common.types.DataTypes.VARBINARY(10))
+                        .physicalColumn(
+                                "bytes", 
org.apache.flink.cdc.common.types.DataTypes.BYTES())
+                        .physicalColumn(
+                                "tinyint", 
org.apache.flink.cdc.common.types.DataTypes.TINYINT())
+                        .physicalColumn(
+                                "smallint", 
org.apache.flink.cdc.common.types.DataTypes.SMALLINT())
+                        .physicalColumn("int", 
org.apache.flink.cdc.common.types.DataTypes.INT())
+                        .physicalColumn(
+                                "bigint", 
org.apache.flink.cdc.common.types.DataTypes.BIGINT())
+                        .physicalColumn(
+                                "float", 
org.apache.flink.cdc.common.types.DataTypes.FLOAT())
+                        .physicalColumn(
+                                "double", 
org.apache.flink.cdc.common.types.DataTypes.DOUBLE())
+                        .physicalColumn(
+                                "decimal",
+                                
org.apache.flink.cdc.common.types.DataTypes.DECIMAL(6, 3))
+                        .physicalColumn("char", 
org.apache.flink.cdc.common.types.DataTypes.CHAR(5))
+                        .physicalColumn(
+                                "varchar", 
org.apache.flink.cdc.common.types.DataTypes.VARCHAR(10))
+                        .physicalColumn(
+                                "string", 
org.apache.flink.cdc.common.types.DataTypes.STRING())
+                        .physicalColumn("date", 
org.apache.flink.cdc.common.types.DataTypes.DATE())
+                        .physicalColumn("time", 
org.apache.flink.cdc.common.types.DataTypes.TIME())
+                        .physicalColumn(
+                                "time_with_precision",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIME(6))
+                        .physicalColumn(
+                                "timestamp",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP())
+                        .physicalColumn(
+                                "timestamp_with_precision_3",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(3))
+                        .physicalColumn(
+                                "timestamp_with_precision_6",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(6))
+                        .physicalColumn(
+                                "timestamp_with_precision_9",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP(9))
+                        .physicalColumn(
+                                "timestamp_ltz",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ())
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_3",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(3))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_6",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(6))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_9",
+                                
org.apache.flink.cdc.common.types.DataTypes.TIMESTAMP_LTZ(9))
+                        .primaryKey("pk_string")
+                        .partitionKey("boolean")
+                        .build();
+
+        Assertions.assertEquals(
+                fullTypesSchema,
+                
PaimonToFlinkCDCTypeConverter.convertPaimonSchemaToFlinkCDCSchema(
+                        
FlinkCDCToPaimonTypeConverter.convertFlinkCDCSchemaToPaimonSchema(

Review Comment:
   Same as above.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.pipeline.cdc.util;
+
+import org.apache.paimon.data.InternalRow;
+
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypeChecks;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/** Converter from Paimon to Flink CDC data. */
+public class PaimonToFlinkCDCDataConverter {
+
+    /** Convert Paimon row to Flink CDC data. */
+    public static DataChangeEvent convertRowToDataChangeEvent(
+            TableId tableId,
+            InternalRow row,
+            List<InternalRow.FieldGetter> fieldGetters,
+            BinaryRecordDataGenerator recordDataGenerator) {
+        Object[] objects = new Object[row.getFieldCount()];
+        for (int i = 0; i < row.getFieldCount(); i++) {
+            objects[i] = fieldGetters.get(i).getFieldOrNull(row);
+        }
+        BinaryRecordData binaryRecordData = 
recordDataGenerator.generate(objects);
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                {
+                    return DataChangeEvent.insertEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            case DELETE:
+            case UPDATE_BEFORE:
+                {
+                    return DataChangeEvent.deleteEvent(tableId, 
binaryRecordData, new HashMap<>());
+                }
+            default:
+                throw new IllegalArgumentException("don't support type of " + 
row.getRowKind());
+        }
+    }
+
+    public static List<InternalRow.FieldGetter> 
createFieldGetters(List<DataType> fieldTypes) {
+        List<InternalRow.FieldGetter> fieldGetters = new ArrayList<>();
+        for (int i = 0; i < fieldTypes.size(); i++) {
+            fieldGetters.add(createFieldGetter(fieldTypes.get(i), i));
+        }
+        return fieldGetters;
+    }
+
+    public static InternalRow.FieldGetter createFieldGetter(DataType 
fieldType, int fieldPos) {
+        final InternalRow.FieldGetter fieldGetter;
+        // ordered by type root definition
+        switch (fieldType.getTypeRoot()) {
+            case CHAR:
+            case VARCHAR:
+                fieldGetter =
+                        row -> 
BinaryStringData.fromString(row.getString(fieldPos).toString());
+                break;
+            case BOOLEAN:
+                fieldGetter = row -> row.getBoolean(fieldPos);
+                break;
+            case BINARY:
+            case VARBINARY:
+                fieldGetter = row -> row.getBinary(fieldPos);
+                break;
+            case DECIMAL:
+                final int decimalPrecision = 
DataTypeChecks.getPrecision(fieldType);
+                final int decimalScale = DataTypeChecks.getScale(fieldType);
+                fieldGetter =
+                        row ->
+                                DecimalData.fromBigDecimal(
+                                        row.getDecimal(fieldPos, 
decimalPrecision, decimalScale)
+                                                .toBigDecimal(),
+                                        decimalPrecision,
+                                        decimalScale);
+                break;
+            case TINYINT:
+                fieldGetter = row -> row.getByte(fieldPos);
+                break;
+            case SMALLINT:
+                fieldGetter = row -> row.getShort(fieldPos);
+                break;
+            case BIGINT:
+                fieldGetter = row -> row.getLong(fieldPos);
+                break;
+            case FLOAT:
+                fieldGetter = row -> row.getFloat(fieldPos);
+                break;
+            case DOUBLE:
+                fieldGetter = row -> row.getDouble(fieldPos);
+                break;
+            case INTEGER:
+                fieldGetter = row -> row.getInt(fieldPos);
+                break;
+            case DATE:
+                fieldGetter = row -> row.getInt(fieldPos);

Review Comment:
   Should the result type be something like DateTime or Instant? Same for below



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/schema/SchemaChangeProvider.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.pipeline.cdc.schema;
+
+import org.apache.paimon.flink.pipeline.cdc.util.FlinkCDCToPaimonTypeConverter;
+import org.apache.paimon.schema.SchemaChange;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.types.DataType;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * The SchemaChangeProvider class provides static methods to create 
SchemaChange objects that
+ * represent different types of schema modifications.
+ */
+public class SchemaChangeProvider {
+
+    /**
+     * Creates a SchemaChange object for adding a column without specifying 
its position.
+     *
+     * @param columnWithPosition The ColumnWithPosition object containing the 
column details and its
+     *     intended position within the schema.
+     * @return A SchemaChange object representing the addition of a column.
+     */
+    public static List<SchemaChange> add(AddColumnEvent.ColumnWithPosition 
columnWithPosition) {
+        List<SchemaChange> result = new ArrayList<>();
+        result.add(
+                SchemaChange.addColumn(
+                        columnWithPosition.getAddColumn().getName(),
+                        
FlinkCDCToPaimonTypeConverter.convertFlinkCDCDataTypeToPaimonDataType(
+                                columnWithPosition.getAddColumn().getType()),
+                        columnWithPosition.getAddColumn().getComment()));
+        // if default value express exists, we need to set the default value 
to the table
+        // option
+        Column column = columnWithPosition.getAddColumn();
+        Optional.ofNullable(
+                        
FlinkCDCToPaimonTypeConverter.convertFlinkCDCDefaultValueToValidValue(
+                                column.getDefaultValueExpression(), 
column.getType()))
+                .ifPresent(
+                        value -> {
+                            result.add(
+                                    SchemaChange.updateColumnDefaultValue(
+                                            new String[] {column.getName()}, 
value));
+                        });
+        return result;
+    }
+
+    /**
+     * Creates a SchemaChange object for adding a column with a specified 
position.
+     *
+     * @param columnWithPosition The ColumnWithPosition object containing the 
column details and its
+     *     intended position within the schema.
+     * @param move The move operation to indicate the column's new position.
+     * @return A SchemaChange object representing the addition of a column 
with position
+     *     information.
+     */
+    public static List<SchemaChange> add(
+            AddColumnEvent.ColumnWithPosition columnWithPosition, 
SchemaChange.Move move) {
+        List<SchemaChange> result = new ArrayList<>();
+        result.add(
+                SchemaChange.addColumn(
+                        columnWithPosition.getAddColumn().getName(),
+                        
FlinkCDCToPaimonTypeConverter.convertFlinkCDCDataTypeToPaimonDataType(
+                                columnWithPosition.getAddColumn().getType()),
+                        columnWithPosition.getAddColumn().getComment(),
+                        move));
+        // if default value express exists, we need to set the default value 
to the table
+        // option
+        Column column = columnWithPosition.getAddColumn();
+        Optional.ofNullable(
+                        
FlinkCDCToPaimonTypeConverter.convertFlinkCDCDefaultValueToValidValue(
+                                column.getDefaultValueExpression(), 
column.getType()))
+                .ifPresent(
+                        value -> {
+                            result.add(
+                                    SchemaChange.updateColumnDefaultValue(
+                                            new String[] {column.getName()}, 
value));
+                        });
+        return result;
+    }
+
+    /**
+     * Creates a SchemaChange object to update the data type of a column.
+     *
+     * @param oldColumnName The name of the column whose data type is to be 
updated.
+     * @param newType The new DataType for the column.
+     * @return A SchemaChange object representing the update of the column's 
data type.
+     */
+    public static SchemaChange updateColumnType(String oldColumnName, DataType 
newType) {
+        return SchemaChange.updateColumnType(
+                oldColumnName,
+                
FlinkCDCToPaimonTypeConverter.convertFlinkCDCDataTypeToPaimonDataType(newType));
+    }
+
+    /**
+     * Creates a SchemaChange object for renaming a column.
+     *
+     * @param oldColumnName The current name of the column to be renamed.
+     * @param newColumnName The new name for the column.
+     * @return A SchemaChange object representing the renaming of a column.
+     */
+    public static List<SchemaChange> rename(
+            String oldColumnName, String newColumnName, Map<String, String> 
options) {
+        List<SchemaChange> result = new ArrayList<>();
+        result.add(SchemaChange.renameColumn(oldColumnName, newColumnName));
+        return result;
+    }
+
+    /**
+     * Creates a SchemaChange object for dropping a column.
+     *
+     * @param columnName The name of the column to be dropped.
+     * @return A SchemaChange object representing the deletion of a column.
+     */
+    public static List<SchemaChange> drop(String columnName) {
+        List<SchemaChange> result = new ArrayList<>();
+        result.add(SchemaChange.dropColumn(columnName));
+        return result;
+    }
+
+    /**
+     * Creates a SchemaChange object for setting an option.
+     *
+     * @param key The key of the option to be set.
+     * @param value The value of the option to be set.
+     * @return A SchemaChange object representing the setting of an option.
+     */
+    public static SchemaChange setOption(String key, String value) {

Review Comment:
   unused method.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresActionUtils.java:
##########
@@ -186,7 +186,7 @@ public static JdbcIncrementalSource<CdcSourceRecord> 
buildPostgresSource(
         customConverterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, 
"numeric");
         CdcDebeziumDeserializationSchema schema =
                 new CdcDebeziumDeserializationSchema(true, 
customConverterConfigs);
-        return 
sourceBuilder.deserializer(schema).includeSchemaChanges(true).build();
+        return sourceBuilder.deserializer(schema).build();

Review Comment:
   Shall we describe why we need to make changes like this in 
non-Paimon-Flink-CDC classes? I suppose these changes are along with the Flink 
CDC version upgrade from 3.1 to 3.5, but not quite sure why we need to make 
such changes. We can add some details to the description section of this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to