loserwang1024 commented on code in PR #3968: URL: https://github.com/apache/flink-cdc/pull/3968#discussion_r2154104544
########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/utils/PostgresTypeUtils.java: ########## @@ -0,0 +1,170 @@ +/* + * 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.postgres.utils; + +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.ZonedTimestampType; +import org.apache.flink.table.types.logical.DecimalType; + +import io.debezium.relational.Column; + +/** A utility class for converting Postgres types to Flink types. */ +public class PostgresTypeUtils { + private static final String PG_SMALLSERIAL = "smallserial"; + private static final String PG_SERIAL = "serial"; + private static final String PG_BIGSERIAL = "bigserial"; + private static final String PG_BYTEA = "bytea"; + private static final String PG_BYTEA_ARRAY = "_bytea"; + private static final String PG_SMALLINT = "int2"; + private static final String PG_SMALLINT_ARRAY = "_int2"; + private static final String PG_INTEGER = "int4"; + private static final String PG_INTEGER_ARRAY = "_int4"; + private static final String PG_BIGINT = "int8"; + private static final String PG_BIGINT_ARRAY = "_int8"; + private static final String PG_REAL = "float4"; + private static final String PG_REAL_ARRAY = "_float4"; + private static final String PG_DOUBLE_PRECISION = "float8"; + private static final String PG_DOUBLE_PRECISION_ARRAY = "_float8"; + private static final String PG_NUMERIC = "numeric"; + private static final String PG_NUMERIC_ARRAY = "_numeric"; + private static final String PG_BOOLEAN = "bool"; + private static final String PG_BOOLEAN_ARRAY = "_bool"; + private static final String PG_TIMESTAMP = "timestamp"; + private static final String PG_TIMESTAMP_ARRAY = "_timestamp"; + private static final String PG_TIMESTAMPTZ = "timestamptz"; + private static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz"; + private static final String PG_DATE = "date"; + private static final String PG_DATE_ARRAY = "_date"; + private static final String PG_TIME = "time"; + private static final String PG_TIME_ARRAY = "_time"; + private static final String PG_TEXT = "text"; + private static final String PG_TEXT_ARRAY = "_text"; + private static final String PG_CHAR = "bpchar"; + private static final String PG_CHAR_ARRAY = "_bpchar"; + private static final String PG_CHARACTER = "character"; + private static final String PG_CHARACTER_ARRAY = "_character"; + private static final String PG_CHARACTER_VARYING = "varchar"; + private static final String PG_CHARACTER_VARYING_ARRAY = "_varchar"; + private static final String PG_UUID = "uuid"; + private static final String PG_GEOMETRY = "geometry"; + private static final String PG_GEOGRAPHY = "geography"; + + /** Returns a corresponding Flink data type from a debezium {@link Column}. */ + public static DataType fromDbzColumn(Column column) { + DataType dataType = convertFromColumn(column); + if (column.isOptional()) { + return dataType; + } else { + return dataType.notNull(); + } + } + + /** + * Returns a corresponding Flink data type from a debezium {@link Column} with nullable always + * be true. + */ + private static DataType convertFromColumn(Column column) { + String typeName = column.typeName(); + + int precision = column.length(); + int scale = column.scale().orElse(0); + + switch (typeName) { + case PG_BOOLEAN: + return DataTypes.BOOLEAN(); Review Comment: I am a little bit concerned about it. Because PostgreSQL's WAL log does not contain DDL statements that include the PostgreSQL type for this mapping. We can only infer the types from Debezium's records. When we later introduce type derivation, it may lead to fragmentation. Could you please check how Debezium performs this mapping? Perhaps we can also use the same mapping approach (this might involve converting fine-grained types into coarser types). @lvyanquan , WDYT? -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org