|
27 | 27 | import org.apache.flink.cdc.common.data.RecordData; |
28 | 28 | import org.apache.flink.cdc.common.data.TimeData; |
29 | 29 | import org.apache.flink.cdc.common.data.TimestampData; |
| 30 | +import org.apache.flink.cdc.common.data.ZonedTimestampData; |
30 | 31 | import org.apache.flink.cdc.common.data.binary.BinaryStringData; |
31 | 32 | import org.apache.flink.cdc.common.event.ChangeEvent; |
32 | 33 | import org.apache.flink.cdc.common.event.CreateTableEvent; |
|
54 | 55 | import io.debezium.time.NanoTime; |
55 | 56 | import io.debezium.time.NanoTimestamp; |
56 | 57 | import io.debezium.time.Timestamp; |
| 58 | +import io.debezium.time.ZonedTimestamp; |
57 | 59 | import org.apache.kafka.connect.data.Decimal; |
58 | 60 | import org.apache.kafka.connect.data.Field; |
59 | 61 | import org.apache.kafka.connect.data.Schema; |
|
65 | 67 | import java.math.BigDecimal; |
66 | 68 | import java.nio.ByteBuffer; |
67 | 69 | import java.time.Instant; |
| 70 | +import java.time.OffsetDateTime; |
68 | 71 | import java.util.Collections; |
69 | 72 | import java.util.Date; |
70 | 73 | import java.util.HashMap; |
@@ -196,6 +199,8 @@ protected DeserializationRuntimeConverter createNotNullConverter(DataType type) |
196 | 199 | return this::convertToTime; |
197 | 200 | case TIMESTAMP_WITHOUT_TIME_ZONE: |
198 | 201 | return this::convertToTimestamp; |
| 202 | + case TIMESTAMP_WITH_TIME_ZONE: |
| 203 | + return this::convertToZonedTimestamp; |
199 | 204 | case TIMESTAMP_WITH_LOCAL_TIME_ZONE: |
200 | 205 | return this::convertToLocalTimeZoneTimestamp; |
201 | 206 | case FLOAT: |
@@ -367,6 +372,28 @@ protected Object convertToTimestamp(Object dbzObj, Schema schema) { |
367 | 372 | + dbzObj.getClass().getName()); |
368 | 373 | } |
369 | 374 |
|
| 375 | + protected Object convertToZonedTimestamp(Object dbzObj, Schema schema) { |
| 376 | + if (dbzObj instanceof String) { |
| 377 | + String str = (String) dbzObj; |
| 378 | + // ZonedTimestamp type is encoded in string type with timezone offset |
| 379 | + // Format: ISO-8601 with timezone offset (e.g., "2020-07-17T18:00:22+00:00") |
| 380 | + if (ZonedTimestamp.SCHEMA_NAME.equals(schema.name())) { |
| 381 | + // Parse using Debezium's ZonedTimestamp formatter |
| 382 | + OffsetDateTime offsetDateTime = OffsetDateTime.parse(str, ZonedTimestamp.FORMATTER); |
| 383 | + return ZonedTimestampData.fromOffsetDateTime(offsetDateTime); |
| 384 | + } else { |
| 385 | + // Fallback to standard ISO-8601 parsing |
| 386 | + OffsetDateTime offsetDateTime = OffsetDateTime.parse(str); |
| 387 | + return ZonedTimestampData.fromOffsetDateTime(offsetDateTime); |
| 388 | + } |
| 389 | + } |
| 390 | + throw new IllegalArgumentException( |
| 391 | + "Unable to convert to TIMESTAMP WITH TIME ZONE from unexpected value '" |
| 392 | + + dbzObj |
| 393 | + + "' of type " |
| 394 | + + dbzObj.getClass().getName()); |
| 395 | + } |
| 396 | + |
370 | 397 | protected Object convertToLocalTimeZoneTimestamp(Object dbzObj, Schema schema) { |
371 | 398 | if (dbzObj instanceof String) { |
372 | 399 | String str = (String) dbzObj; |
|
0 commit comments