You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
[FLINK-38742][cdc-pipeline/postgres] Fix TIMESTAMPTZ type mapping to TIMESTAMP_LTZ
PostgreSQL TIMESTAMPTZ columns were causing NumberFormatException because
PostgresTypeUtils incorrectly mapped them to ZonedTimestampType (TIMESTAMP_WITH_TIME_ZONE).
PostgreSQL TIMESTAMPTZ stores values internally in UTC and converts on display
based on session timezone, which semantically matches TIMESTAMP_LTZ, not
TIMESTAMP_WITH_TIME_ZONE. Additionally, Debezium's PostgreSQL connector always
converts TIMESTAMPTZ values to UTC format (e.g., '2025-12-30T05:59:50.724893Z')
before serialization.
The type mismatch caused sinks to call getZonedTimestamp() on LocalZonedTimestampData,
resulting in NumberFormatException when trying to parse binary data as a string.
This commit fixes PostgresTypeUtils to correctly map TIMESTAMPTZ and TIMESTAMPTZ_ARRAY
to TIMESTAMP_LTZ type, matching both PostgreSQL semantics and Debezium's serialization
format. The E2E test is updated to verify the fix.
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/utils/PostgresTypeUtils.java
Copy file name to clipboardExpand all lines: flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/PostgresE2eITCase.java
"CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL 'nextval('inventory.products_id_seq'::regclass)',`name` VARCHAR(255) NOT NULL,`description` VARCHAR(512),`weight` FLOAT}, primaryKeys=id, options=()}",
"DataChangeEvent{tableId=%s.products, before=[], after=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8], op=INSERT, meta=()}",
"DataChangeEvent{tableId=%s.products, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14], op=INSERT, meta=()}",
154
-
"DataChangeEvent{tableId=%s.products, before=[], after=[102, car battery, 12V car battery, 8.1], op=INSERT, meta=()}");
145
+
"CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL 'nextval('inventory.products_id_seq'::regclass)',`name` VARCHAR(255) NOT NULL,`description` VARCHAR(512),`weight` FLOAT,`created_at` TIMESTAMP_LTZ(6)}, primaryKeys=id, options=()}",
0 commit comments