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


##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/schema/SchemaChangeProvider.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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) {

Review Comment:
   the `options` parameter could be removed



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.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.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("Unsupported RowKind type: 
" + 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 -> 
DateData.fromEpochDay(row.getInt(fieldPos));
+                break;
+            case TIME_WITHOUT_TIME_ZONE:
+                fieldGetter = row -> 
TimeData.fromMillisOfDay(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:

Review Comment:
   Shall we support nested types like ROW, ARRAY and MAP?



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/cdc/connectors/base/relational/connection/JdbcConnectionPools.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.base.relational.connection;
+
+import com.zaxxer.hikari.HikariDataSource;
+import org.apache.flink.cdc.common.annotation.VisibleForTesting;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Copied from Flink CDC 3.5.0: {@code
+ * 
org/apache/flink/cdc/connectors/base/relational/connection/JdbcConnectionPools.java}.
 Modified
+ * method {@link JdbcConnectionPools#clear()} at line 92 ~ 94.

Review Comment:
   Maybe better to add a stable link that points to the 3.5.0 branch/tag, 
instead of pointing to the master branch.



##########
paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/pipeline/cdc/util/PaimonToFlinkCDCDataConverter.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.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.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:
   When rowkind == UPDATE_AFTER, I'm not sure whether `insertEvent` should be a 
better choice than `updateEvent`.



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/pipeline/cdc/util/DataConvertTest.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.paimon.types.RowKind;
+import org.apache.paimon.utils.InternalRowUtils;
+
+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.RecordData;
+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.schema.Schema;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Data convert test for {@link PaimonToFlinkCDCDataConverter} and {@link
+ * FlinkCDCToPaimonDataConverter}.
+ */
+public class DataConvertTest {
+
+    @Test
+    void testFullTypesConversion() {
+        Schema fullTypesSchema =
+                Schema.newBuilder()
+                        .physicalColumn("pk_string", 
DataTypes.STRING().notNull())
+                        .physicalColumn("boolean", DataTypes.BOOLEAN())
+                        .physicalColumn("binary", DataTypes.BINARY(3))
+                        .physicalColumn("varbinary", DataTypes.VARBINARY(10))
+                        .physicalColumn("bytes", DataTypes.BYTES())
+                        .physicalColumn("tinyint", DataTypes.TINYINT())
+                        .physicalColumn("smallint", DataTypes.SMALLINT())
+                        .physicalColumn("int", DataTypes.INT())
+                        .physicalColumn("bigint", DataTypes.BIGINT())
+                        .physicalColumn("float", DataTypes.FLOAT())
+                        .physicalColumn("double", DataTypes.DOUBLE())
+                        .physicalColumn("decimal", DataTypes.DECIMAL(6, 3))
+                        .physicalColumn("char", DataTypes.CHAR(5))
+                        .physicalColumn("varchar", DataTypes.VARCHAR(10))
+                        .physicalColumn("string", DataTypes.STRING())
+                        .physicalColumn("date", DataTypes.DATE())
+                        .physicalColumn("time", DataTypes.TIME())
+                        .physicalColumn("time_with_precision", 
DataTypes.TIME(6))
+                        .physicalColumn("timestamp", DataTypes.TIMESTAMP())
+                        .physicalColumn("timestamp_with_precision_3", 
DataTypes.TIMESTAMP(3))
+                        .physicalColumn("timestamp_with_precision_6", 
DataTypes.TIMESTAMP(6))
+                        .physicalColumn("timestamp_with_precision_9", 
DataTypes.TIMESTAMP(9))
+                        .physicalColumn("timestamp_ltz", 
DataTypes.TIMESTAMP_LTZ())
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_3", 
DataTypes.TIMESTAMP_LTZ(3))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_6", 
DataTypes.TIMESTAMP_LTZ(6))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_9", 
DataTypes.TIMESTAMP_LTZ(9))
+                        .primaryKey("pk_string")
+                        .partitionKey("boolean")
+                        .build();
+
+        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),
+                    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),
+                };
+        testConvertBackAndForth(
+                fullTypesSchema,
+                testData,
+                "[pk_string, true, [1, 2, 3], [4, 5, 6], [7, 8, 9], 1, 2, 3, 
4, 5.1, 6.2, 7.123, test1, test2, test3, 1000, 200, 300, 
1970-01-01T00:00:00.100000001, 1970-01-01T00:00:00.200000002, 
1970-01-01T00:00:00.300000003, 1970-01-01T00:00:00.400000004, 
1970-01-01T00:00:00.300000003, 1970-01-01T00:00:00.400000004, 
1970-01-01T00:00:00.500000005, 1970-01-01T00:00:00.600000006]");

Review Comment:
   Would it be better to directly validate against the original objects or a 
Paimon InternalRow, instead of the string representation of them?



##########
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:
   Is this change the result of some Flink CDC version update?



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

Review Comment:
   Better to change `tableSchema` into `rowType` to avoid confusion.



##########
paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/pipeline/cdc/util/DataConvertTest.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.paimon.types.RowKind;
+import org.apache.paimon.utils.InternalRowUtils;
+
+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.RecordData;
+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.schema.Schema;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Data convert test for {@link PaimonToFlinkCDCDataConverter} and {@link
+ * FlinkCDCToPaimonDataConverter}.
+ */
+public class DataConvertTest {
+
+    @Test
+    void testFullTypesConversion() {
+        Schema fullTypesSchema =
+                Schema.newBuilder()
+                        .physicalColumn("pk_string", 
DataTypes.STRING().notNull())
+                        .physicalColumn("boolean", DataTypes.BOOLEAN())
+                        .physicalColumn("binary", DataTypes.BINARY(3))
+                        .physicalColumn("varbinary", DataTypes.VARBINARY(10))
+                        .physicalColumn("bytes", DataTypes.BYTES())
+                        .physicalColumn("tinyint", DataTypes.TINYINT())
+                        .physicalColumn("smallint", DataTypes.SMALLINT())
+                        .physicalColumn("int", DataTypes.INT())
+                        .physicalColumn("bigint", DataTypes.BIGINT())
+                        .physicalColumn("float", DataTypes.FLOAT())
+                        .physicalColumn("double", DataTypes.DOUBLE())
+                        .physicalColumn("decimal", DataTypes.DECIMAL(6, 3))
+                        .physicalColumn("char", DataTypes.CHAR(5))
+                        .physicalColumn("varchar", DataTypes.VARCHAR(10))
+                        .physicalColumn("string", DataTypes.STRING())
+                        .physicalColumn("date", DataTypes.DATE())
+                        .physicalColumn("time", DataTypes.TIME())
+                        .physicalColumn("time_with_precision", 
DataTypes.TIME(6))
+                        .physicalColumn("timestamp", DataTypes.TIMESTAMP())
+                        .physicalColumn("timestamp_with_precision_3", 
DataTypes.TIMESTAMP(3))
+                        .physicalColumn("timestamp_with_precision_6", 
DataTypes.TIMESTAMP(6))
+                        .physicalColumn("timestamp_with_precision_9", 
DataTypes.TIMESTAMP(9))
+                        .physicalColumn("timestamp_ltz", 
DataTypes.TIMESTAMP_LTZ())
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_3", 
DataTypes.TIMESTAMP_LTZ(3))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_6", 
DataTypes.TIMESTAMP_LTZ(6))
+                        .physicalColumn(
+                                "timestamp_ltz_with_precision_9", 
DataTypes.TIMESTAMP_LTZ(9))
+                        .primaryKey("pk_string")
+                        .partitionKey("boolean")

Review Comment:
   Let's also add tests for nested types like MAP and ARRAY.



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