diff --git a/.gitignore b/.gitignore index bf83032a9c98..9ea48e4b1dcd 100644 --- a/.gitignore +++ b/.gitignore @@ -23,6 +23,12 @@ logs.zip *.iml .idea/* +# Eclipse / M2E files +.classpath +.factorypath +.project +.settings/ + .DS_Store metastore_db/ diff --git a/docs/en/introduction/concepts/incompatible-changes.md b/docs/en/introduction/concepts/incompatible-changes.md index 7cdb776ddea5..93a439372efc 100644 --- a/docs/en/introduction/concepts/incompatible-changes.md +++ b/docs/en/introduction/concepts/incompatible-changes.md @@ -5,6 +5,18 @@ You need to check this document before you upgrade to related version. ## dev +### JDBC Connector + +- **Breaking Change: Mapping of timezone-aware timestamp columns to `TIMESTAMP_TZ` type** + - **Affected component**: `seatunnel-connectors-v2/connector-jdbc`, `seatunnel-connectors-v2/connector-iceberg`, `seatunnel-connectors-v2/connector-cdc-base`, `seatunnel-connectors-v2/connector-cdc-tidb`, `seatunnel-connectors-v2/connector-starrocks`, `seatunnel-connectors-v2/connector-hudi`, `seatunnel-connectors-v2/connector-snowflake` (via JDBC dialect) + - **Description**: Previously, JDBC sources mapped both timezone-naive (e.g., MySQL `DATETIME`) and timezone-aware (e.g., MySQL `TIMESTAMP`) timestamp columns to SeaTunnel's internal `TIMESTAMP` type. Now, timezone-aware columns like MySQL `TIMESTAMP`, PostgreSQL `timestamptz`, Oracle `TIMESTAMP WITH LOCAL TIME ZONE`, SQL Server `datetimeoffset`, Snowflake `TIMESTAMP_LTZ/TZ`, and others are explicitly mapped to `TIMESTAMP_TZ`. This ensures that timezone semantics are accurately preserved when writing to formats like Iceberg, where `TIMESTAMP` is saved as `timestamp` (without timezone) and `TIMESTAMP_TZ` is saved as `timestamptz` (with timezone). + - **Impact**: If your downstream Sink relies on receiving `TIMESTAMP` types and does not support `TIMESTAMP_TZ` natively, you may encounter type mismatch errors. For Iceberg users, this means columns previously written as `timestamp` (without timezone) may now be written as `timestamptz` (with timezone) and change the table schema. You may need to cast the column in sql transform or update your sink configurations. (#10685) + - **Connector-specific behavior changes**: + - **Snowflake**: `TIMESTAMP_LTZ` and `TIMESTAMP_TZ` columns are now mapped to `OFFSET_DATE_TIME_TYPE` (`TIMESTAMP_TZ`) instead of `LOCAL_DATE_TIME_TYPE`. This affects both Source and Sink paths for Snowflake. + - **StarRocks**: `TIMESTAMP_TZ` values written to StarRocks Sink are stored as `DATETIME` (wall-clock only, timezone offset is dropped) due to StarRocks not having a native timezone-aware datetime type. + - **Hudi**: `TIMESTAMP_TZ` is now mapped to Avro `timestampMillis` (UTC epoch). Existing Hudi tables written with the old schema may need to be re-created if schema evolution is not supported. + - **CDC (Debezium-based, TiDB)**: CDC connectors now correctly handle `TIMESTAMP_TZ` type in the Debezium deserialization layer. Previously, `TIMESTAMP_TZ` was unsupported and would throw `UnsupportedOperationException`. Users who were previously unable to use timezone-aware columns in CDC pipelines can now do so. + ### API Changes - **Breaking Change: Engine REST table metrics key format** diff --git a/docs/zh/introduction/concepts/incompatible-changes.md b/docs/zh/introduction/concepts/incompatible-changes.md index 784729cec657..c439e4f42b25 100644 --- a/docs/zh/introduction/concepts/incompatible-changes.md +++ b/docs/zh/introduction/concepts/incompatible-changes.md @@ -4,6 +4,18 @@ ## dev +### JDBC Connector + +- **破坏性变更:带时区的时间戳列映射为 `TIMESTAMP_TZ` 类型** + - **影响范围**:`seatunnel-connectors-v2/connector-jdbc`、`seatunnel-connectors-v2/connector-iceberg`、`seatunnel-connectors-v2/connector-cdc-base`、`seatunnel-connectors-v2/connector-cdc-tidb`、`seatunnel-connectors-v2/connector-starrocks`、`seatunnel-connectors-v2/connector-hudi`、`seatunnel-connectors-v2/connector-snowflake`(通过 JDBC 方言) + - **变更说明**:以前,JDBC Source 将无时区(如 MySQL `DATETIME`)和带时区(如 MySQL `TIMESTAMP`)的时间戳列都映射为 SeaTunnel 内部的 `TIMESTAMP` 类型。现在,带时区的列(如 MySQL `TIMESTAMP`、PostgreSQL `timestamptz`、Oracle `TIMESTAMP WITH LOCAL TIME ZONE`、SQL Server `datetimeoffset`、Snowflake `TIMESTAMP_LTZ/TZ` 等)被显式映射为 `TIMESTAMP_TZ`。这确保了在写入 Iceberg 等格式时,时区语义得到准确保留(在 Iceberg 中 `TIMESTAMP` 存为无时区的 `timestamp`,`TIMESTAMP_TZ` 存为带时区的 `timestamptz`)。 + - **影响**:如果您的下游 Sink 依赖接收 `TIMESTAMP` 类型且不支持 `TIMESTAMP_TZ`,您可能会遇到类型不匹配错误。对于 Iceberg 用户,这意味着以前作为 `timestamp`(无时区)写入的列现在可能会作为 `timestamptz`(带时区)写入,从而改变表结构。您可能需要在 SQL Transform 中转换该列或更新您的 Sink 配置。(#10685) + - **各连接器具体行为变更**: + - **Snowflake**:`TIMESTAMP_LTZ` 和 `TIMESTAMP_TZ` 列现在映射为 `OFFSET_DATE_TIME_TYPE`(`TIMESTAMP_TZ`),而不是原来的 `LOCAL_DATE_TIME_TYPE`。这同时影响 Snowflake 的 Source 和 Sink 路径。 + - **StarRocks**:写入 StarRocks Sink 的 `TIMESTAMP_TZ` 值以 `DATETIME`(仅保留时钟时间,时区偏移量丢失)形式存储,这是由于 StarRocks 不支持原生带时区的日期时间类型。 + - **Hudi**:`TIMESTAMP_TZ` 现在映射为 Avro `timestampMillis`(UTC 纪元时间)。如果 Hudi 表不支持 Schema Evolution,以旧 Schema 写入的现有表可能需要重新创建。 + - **CDC(基于 Debezium,TiDB)**:CDC 连接器现在可以正确处理 Debezium 反序列化层中的 `TIMESTAMP_TZ` 类型。以前,`TIMESTAMP_TZ` 不受支持,会抛出 `UnsupportedOperationException`。现在,在 CDC 管道中使用带时区列的用户可以正常使用。 + ### API 变更 - **破坏性变更:Engine REST 表级指标 key 格式变化** diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConverters.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConverters.java index b5982736d50b..137ae98cdf06 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConverters.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConverters.java @@ -165,6 +165,8 @@ public Object convert(Object dbzObj, Schema schema) throws Exception { return convertToTime(); case TIMESTAMP: return convertToTimestamp(serverTimeZone); + case TIMESTAMP_TZ: + return convertToTimestampTz(serverTimeZone); case FLOAT: return wrapNumericConverter(convertToFloat()); case DOUBLE: @@ -392,6 +394,122 @@ public Object convert(Object dbzObj, Schema schema) { }; } + /** + * Flexible fallback formatter that covers non-ISO variants emitted by Debezium: + * + * + */ + private static final java.time.format.DateTimeFormatter FLEXIBLE_OFFSET_FORMATTER = + new java.time.format.DateTimeFormatterBuilder() + .parseLenient() + .append(java.time.format.DateTimeFormatter.ISO_LOCAL_DATE) + .optionalStart() + .appendLiteral('T') + .optionalEnd() + .optionalStart() + .appendLiteral(' ') + .optionalEnd() + .append(java.time.format.DateTimeFormatter.ISO_LOCAL_TIME) + .appendPattern("[XXX][XX][X]") + .toFormatter(); + + private static DebeziumDeserializationConverter convertToTimestampTz(ZoneId serverTimeZone) { + return new DebeziumDeserializationConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(Object dbzObj, Schema schema) { + if (dbzObj instanceof String) { + return parseOffsetDateTimeFromString((String) dbzObj, serverTimeZone); + } + java.time.LocalDateTime localDateTime = + TemporalConversions.toLocalDateTime(dbzObj, serverTimeZone); + return localDateTime + .atZone(serverTimeZone) + .toOffsetDateTime() + .withOffsetSameInstant(java.time.ZoneOffset.UTC); + } + }; + } + + /** + * Parses a Debezium-emitted timestamp string into an {@link java.time.OffsetDateTime} using a + * fallback chain that tolerates the variety of formats Debezium may produce: + * + *
    + *
  1. {@link java.time.OffsetDateTime#parse} — strict ISO-8601 with numeric offset, e.g. + * {@code 2024-01-01T12:00:00+08:00} + *
  2. {@link java.time.ZonedDateTime#parse} — IANA zone-region id, e.g. {@code + * 2024-01-01T12:00:00 Asia/Shanghai} (the space before the zone id is normalized to 'T' + * first) + *
  3. {@link #FLEXIBLE_OFFSET_FORMATTER} — space date/time separator or short offset, e.g. + * {@code 2024-01-01 12:00:00+08:00} or {@code 2024-01-01T12:00:00+08} + *
  4. {@link Instant#parse} — UTC epoch literal, e.g. {@code 2024-01-01T12:00:00Z} + *
+ * + * If all attempts fail, an {@link IllegalArgumentException} is thrown with the raw value + * included so that the failing CDC task carries enough context for diagnosis. + */ + @VisibleForTesting + static java.time.OffsetDateTime parseOffsetDateTimeFromString( + String str, ZoneId serverTimeZone) { + // 1. Strict ISO-8601 with numeric offset: 2024-01-01T12:00:00+08:00 / Z + try { + return java.time.OffsetDateTime.parse(str) + .withOffsetSameInstant(java.time.ZoneOffset.UTC); + } catch (java.time.format.DateTimeParseException ignored) { + // fall through + } + + // 2. IANA zone-region id: 2024-01-01T12:00:00 Asia/Shanghai + // Debezium separates the zone id with a space; replace the *last* space before a + // letter-only token so that the date/time part is unaffected. + try { + String normalized = str.replaceFirst(" ([A-Za-z])", "[$1").replace(" ", "/") + "]"; + // Build a formatter that accepts [...] as a zone-region + java.time.format.DateTimeFormatter zoneRegionFmt = + new java.time.format.DateTimeFormatterBuilder() + .append(java.time.format.DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendLiteral('[') + .parseCaseSensitive() + .appendZoneRegionId() + .appendLiteral(']') + .toFormatter(); + return java.time.ZonedDateTime.parse(normalized, zoneRegionFmt) + .toOffsetDateTime() + .withOffsetSameInstant(java.time.ZoneOffset.UTC); + } catch (Exception ignored) { + // fall through + } + + // 3. Space separator or short offset: 2024-01-01 12:00:00+08:00 / +08 + try { + return java.time.OffsetDateTime.parse(str, FLEXIBLE_OFFSET_FORMATTER) + .withOffsetSameInstant(java.time.ZoneOffset.UTC); + } catch (java.time.format.DateTimeParseException ignored) { + // fall through + } + + // 4. UTC epoch literal: 2024-01-01T12:00:00Z + try { + return Instant.parse(str).atOffset(java.time.ZoneOffset.UTC); + } catch (java.time.format.DateTimeParseException ignored) { + // fall through + } + + throw new IllegalArgumentException( + "Unable to parse OffsetDateTime from CDC TIMESTAMP_TZ value: '" + + str + + "'. Supported formats: ISO-8601 with numeric offset, IANA zone-region" + + " id, space-separated date/time, short-form hour-only offset, UTC" + + " epoch literal."); + } + private static DebeziumDeserializationConverter convertToTimestamp(ZoneId serverTimeZone) { return new DebeziumDeserializationConverter() { private static final long serialVersionUID = 1L; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConvertersTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConvertersTest.java index c17243b2bda6..e243b8fe17da 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConvertersTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializationConvertersTest.java @@ -37,7 +37,10 @@ import io.debezium.data.geometry.Geography; import io.debezium.data.geometry.Geometry; +import java.time.Instant; +import java.time.OffsetDateTime; import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -206,4 +209,57 @@ void testGeographyStringConversion() throws Exception { Assertions.assertTrue(fieldValue instanceof String); Assertions.assertEquals("0102FF", fieldValue); } + + /** + * Verifies the fallback chain in {@code parseOffsetDateTimeFromString} against the four + * Debezium timestamp formats called out in the review: + * + *
    + *
  1. Standard ISO-8601 with numeric offset (baseline) + *
  2. Oracle TIMESTAMP WITH LOCAL TIME ZONE — IANA zone-region id + *
  3. MySQL TIMESTAMP in certain schema-history modes — space date/time separator + *
  4. PostgreSQL timestamptz — short-form hour-only offset + *
+ * + * All four variants must parse to the same UTC instant. + */ + @Test + void testParseOffsetDateTimeFromStringFallbackChain() { + ZoneId serverTz = ZoneId.of("Asia/Shanghai"); + // Expected UTC instant: 2024-01-01 04:00:00Z (2024-01-01 12:00:00+08:00) + OffsetDateTime expected = Instant.parse("2024-01-01T04:00:00Z").atOffset(ZoneOffset.UTC); + + // 1. ISO-8601 with numeric offset — handled by OffsetDateTime.parse + OffsetDateTime r1 = + SeaTunnelRowDebeziumDeserializationConverters.parseOffsetDateTimeFromString( + "2024-01-01T12:00:00+08:00", serverTz); + Assertions.assertEquals(expected, r1, "ISO-8601 numeric offset failed"); + + // 2. IANA zone-region id — Oracle TIMESTAMP WITH LOCAL TIME ZONE + OffsetDateTime r2 = + SeaTunnelRowDebeziumDeserializationConverters.parseOffsetDateTimeFromString( + "2024-01-01T12:00:00 Asia/Shanghai", serverTz); + Assertions.assertEquals(expected, r2, "IANA zone-region id failed"); + + // 3. Space separator — MySQL in certain schema-history modes + OffsetDateTime r3 = + SeaTunnelRowDebeziumDeserializationConverters.parseOffsetDateTimeFromString( + "2024-01-01 12:00:00+08:00", serverTz); + Assertions.assertEquals(expected, r3, "Space date/time separator failed"); + + // 4. Short-form hour-only offset — PostgreSQL timestamptz + OffsetDateTime r4 = + SeaTunnelRowDebeziumDeserializationConverters.parseOffsetDateTimeFromString( + "2024-01-01T12:00:00+08", serverTz); + Assertions.assertEquals(expected, r4, "Short-form hour-only offset failed"); + } + + @Test + void testParseOffsetDateTimeFromStringThrowsOnUnknownFormat() { + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + SeaTunnelRowDebeziumDeserializationConverters.parseOffsetDateTimeFromString( + "not-a-timestamp", ZoneId.systemDefault())); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java index 850b8add8446..5bcf98938b07 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java @@ -221,7 +221,7 @@ public void testParseDDLForAddColumn() { addEvent3.get(12), "col13".toUpperCase(), "timestamp with time zone(6)", - "TIMESTAMP", + "TIMESTAMP_TZ", null, 6, false, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-tidb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/tidb/source/converter/DefaultDataConverter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-tidb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/tidb/source/converter/DefaultDataConverter.java index f17c6c4a341f..f71437a903a3 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-tidb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/tidb/source/converter/DefaultDataConverter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-tidb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/tidb/source/converter/DefaultDataConverter.java @@ -37,6 +37,8 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; @Slf4j public class DefaultDataConverter implements DataConverter { @@ -101,6 +103,11 @@ public SeaTunnelRow convert(Object[] values, TiTableInfo tableInfo, SeaTunnelRow case TIMESTAMP: fields[fieldIndex] = convertToTimestamp(value, dataType); break; + case TIMESTAMP_TZ: + // TiDB TIMESTAMP is stored as UTC (LTZ). + // Convert to OffsetDateTime with UTC offset to preserve timezone semantics. + fields[fieldIndex] = convertToOffsetDateTime(value, dataType); + break; case BYTES: fields[fieldIndex] = convertToBinary(value); break; @@ -259,6 +266,45 @@ private static Object convertToTime(Object value) { return TemporalConversions.toLocalTime(value); } + private static Object convertToOffsetDateTime( + Object value, org.tikv.common.types.DataType dataType) { + if (value instanceof Timestamp) { + // TiDB TIMESTAMP is stored in UTC; convert to OffsetDateTime with UTC zone. + Instant instant = ((Timestamp) value).toInstant(); + return OffsetDateTime.ofInstant(instant, ZoneOffset.UTC); + } + if (value instanceof Long) { + // TiDB may emit TIMESTAMP as epoch milliseconds in some snapshot modes. + return OffsetDateTime.ofInstant(Instant.ofEpochMilli((Long) value), ZoneOffset.UTC); + } + if (value instanceof LocalDateTime) { + // LocalDateTime without explicit zone — treat as UTC wall-clock value. + return ((LocalDateTime) value).atOffset(ZoneOffset.UTC); + } + if (value instanceof String) { + // String representation from TiDB CDC — attempt ISO-8601 parse. + try { + return OffsetDateTime.parse((String) value); + } catch (java.time.format.DateTimeParseException e) { + throw new IllegalArgumentException( + "Unable to convert TIMESTAMP_TZ from String value: '" + + value + + "' for TiDB dataType: " + + dataType, + e); + } + } + // Unknown type — fail fast with enough context for diagnosis instead of silently + // returning the raw value which would cause a ClassCastException downstream. + throw new IllegalArgumentException( + "Unsupported value type for TIMESTAMP_TZ conversion: " + + value.getClass().getName() + + ", value='" + + value + + "', TiDB dataType=" + + dataType); + } + private static Object convertToTimestamp( Object value, org.tikv.common.types.DataType dataType) { switch (dataType.getType()) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/AbstractDorisTypeConverter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/AbstractDorisTypeConverter.java index df057b38c430..20ab1a517bd8 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/AbstractDorisTypeConverter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/AbstractDorisTypeConverter.java @@ -353,6 +353,12 @@ protected BasicTypeDefine sampleReconvert( builder.columnType(String.format("%s(%s)", DORIS_VARCHAR, 8)); builder.dataType(DORIS_VARCHAR); break; + case TIMESTAMP_TZ: + // Doris has no timezone-aware datetime type; store as DATETIME (wall-clock value) + builder.columnType(String.format("%s(%s)", DORIS_DATETIME, MAX_DATETIME_SCALE)); + builder.dataType(DORIS_DATETIME); + builder.scale(MAX_DATETIME_SCALE); + break; case ARRAY: SeaTunnelDataType dataType = column.getDataType(); SeaTunnelDataType elementType = null; @@ -426,6 +432,7 @@ private void reconvertBuildArrayInternal( builder.dataType(DORIS_DATEV2_ARRAY); break; case TIMESTAMP: + case TIMESTAMP_TZ: builder.columnType(DORIS_DATETIMEV2_ARRAY); builder.dataType(DORIS_DATETIMEV2_ARRAY); break; diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV1.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV1.java index a3c4684f7da1..f7c5b80368bd 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV1.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV1.java @@ -103,6 +103,7 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(DORIS_DATEV2); break; case TIMESTAMP: + case TIMESTAMP_TZ: if (column.getScale() != null && column.getScale() > 0 && column.getScale() <= MAX_DATETIME_SCALE) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV2.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV2.java index feef2c3956f8..65e9756d4ecc 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV2.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/datatype/DorisTypeConverterV2.java @@ -227,6 +227,7 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(DORIS_DATE); break; case TIMESTAMP: + case TIMESTAMP_TZ: if (column.getScale() != null && column.getScale() >= 0 && column.getScale() <= MAX_DATETIME_SCALE) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java index 023724ada09f..6dcbc7959553 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java @@ -88,7 +88,7 @@ public SeaTunnelRowSerializer( if (JSON.equals(type)) { JsonSerializationSchema jsonSerializationSchema = - new JsonSerializationSchema(this.seaTunnelRowType); + new JsonSerializationSchema(this.seaTunnelRowType, true); ObjectMapper mapper = jsonSerializationSchema.getMapper(); mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); this.serialize = jsonSerializationSchema; diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java index acb102127573..d9128487c477 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java @@ -89,7 +89,10 @@ public static Schema convertToSchema(SeaTunnelDataType dataType, String rowNa Schema binary = SchemaBuilder.builder().bytesType(); return nullableSchema(binary); case TIMESTAMP: - // use long to represents Timestamp + case TIMESTAMP_TZ: + // use long to represents Timestamp / Timestamp with timezone + // TIMESTAMP_TZ (OffsetDateTime/LTZ) is stored as timestampMillis (UTC epoch) + // same as TIMESTAMP, as Avro/Hudi does not have a native timezone-aware type LogicalType avroLogicalType; avroLogicalType = LogicalTypes.timestampMillis(); Schema timestamp = avroLogicalType.addToSchema(SchemaBuilder.builder().longType()); diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java index 5c0636266934..930fcf39ed6a 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java @@ -162,6 +162,19 @@ public Object convert(Schema schema, Object object) { } }; break; + case TIMESTAMP_TZ: + converter = + new RowDataToAvroConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(Schema schema, Object object) { + return ((java.time.OffsetDateTime) object) + .toInstant() + .toEpochMilli(); + } + }; + break; case DECIMAL: converter = new RowDataToAvroConverter() { diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java index 4243c794656f..2899c4acb4e1 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java @@ -90,8 +90,10 @@ private Object convert( case TIMESTAMP: Types.TimestampType timestampType = (Types.TimestampType) icebergType; if (timestampType.shouldAdjustToUTC()) { - return OffsetDateTime.class.cast(icebergValue).toLocalDateTime(); + // withZone() → LTZ → return OffsetDateTime to preserve timezone info + return OffsetDateTime.class.cast(icebergValue); } + // withoutZone() → NTZ → return LocalDateTime as-is return LocalDateTime.class.cast(icebergValue); case STRING: return String.class.cast(icebergValue); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java index 20b8ba3adcaf..6bf9f90367a4 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java @@ -55,6 +55,12 @@ public static SeaTunnelDataType mapping(String field, @NonNull Type icebergTy case TIME: return LocalTimeType.LOCAL_TIME_TYPE; case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) icebergType; + if (timestampType.shouldAdjustToUTC()) { + // withZone() → LTZ → TIMESTAMP_TZ + return LocalTimeType.OFFSET_DATE_TIME_TYPE; + } + // withoutZone() → NTZ → TIMESTAMP return LocalTimeType.LOCAL_DATE_TIME_TYPE; case STRING: return BasicType.STRING_TYPE; @@ -173,6 +179,10 @@ public static Type toIcebergType(SeaTunnelDataType dataType, AtomicInteger nextI case TIME: return Types.TimeType.get(); case TIMESTAMP: + // NTZ → Iceberg withoutZone() + return Types.TimestampType.withoutZone(); + case TIMESTAMP_TZ: + // LTZ → Iceberg withZone() return Types.TimestampType.withZone(); case STRING: default: diff --git a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapperTest.java b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapperTest.java index baf9b9441fc7..1f96140cb793 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapperTest.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapperTest.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.iceberg.types.Type; @@ -80,4 +81,44 @@ void throwsExceptionWhenSinkTypeIsInvalid() { IcebergTypeMapper.toIcebergType(column, new AtomicInteger(1)); }); } + + @Test + void timestampNtzMapsToWithoutZone() { + // TIMESTAMP (NTZ) → Iceberg withoutZone() + Column column = mock(Column.class); + when(column.getSinkType()).thenReturn(null); + when(column.getDataType()) + .thenReturn((SeaTunnelDataType) LocalTimeType.LOCAL_DATE_TIME_TYPE); + + Type result = IcebergTypeMapper.toIcebergType(column, new AtomicInteger(1)); + assertEquals(Types.TimestampType.withoutZone(), result); + } + + @Test + void timestampTzMapsToWithZone() { + // TIMESTAMP_TZ (LTZ) → Iceberg withZone() + Column column = mock(Column.class); + when(column.getSinkType()).thenReturn(null); + when(column.getDataType()) + .thenReturn((SeaTunnelDataType) LocalTimeType.OFFSET_DATE_TIME_TYPE); + + Type result = IcebergTypeMapper.toIcebergType(column, new AtomicInteger(1)); + assertEquals(Types.TimestampType.withZone(), result); + } + + @Test + void icebergWithoutZoneMapsToLocalDateTimeType() { + // Iceberg withoutZone() → SeaTunnel LOCAL_DATE_TIME_TYPE (TIMESTAMP / NTZ) + SeaTunnelDataType result = + IcebergTypeMapper.mapping("ts_ntz", Types.TimestampType.withoutZone()); + assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, result); + } + + @Test + void icebergWithZoneMapsToOffsetDateTimeType() { + // Iceberg withZone() → SeaTunnel OFFSET_DATE_TIME_TYPE (TIMESTAMP_TZ / LTZ) + SeaTunnelDataType result = + IcebergTypeMapper.mapping("ts_ltz", Types.TimestampType.withZone()); + assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, result); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/snowflake/SnowflakeDataTypeConvertor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/snowflake/SnowflakeDataTypeConvertor.java index 52dd03cef0df..dd8e89886f38 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/snowflake/SnowflakeDataTypeConvertor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/snowflake/SnowflakeDataTypeConvertor.java @@ -143,10 +143,11 @@ public SeaTunnelDataType toSeaTunnelType( return LocalTimeType.LOCAL_TIME_TYPE; case SNOWFLAKE_DATE_TIME: case SNOWFLAKE_TIMESTAMP: - case SNOWFLAKE_TIMESTAMP_LTZ: case SNOWFLAKE_TIMESTAMP_NTZ: - case SNOWFLAKE_TIMESTAMP_TZ: return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case SNOWFLAKE_TIMESTAMP_LTZ: + case SNOWFLAKE_TIMESTAMP_TZ: + return LocalTimeType.OFFSET_DATE_TIME_TYPE; default: throw CommonError.convertToSeaTunnelTypeError( DatabaseIdentifier.SNOWFLAKE, connectorDataType, field); @@ -186,7 +187,9 @@ public String toConnectorType( case TIME: return SNOWFLAKE_TIME; case TIMESTAMP: - return SNOWFLAKE_TIMESTAMP; + return SNOWFLAKE_TIMESTAMP_NTZ; + case TIMESTAMP_TZ: + return SNOWFLAKE_TIMESTAMP_TZ; default: throw CommonError.convertToSeaTunnelTypeError( DatabaseIdentifier.SNOWFLAKE, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/converter/AbstractJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/converter/AbstractJdbcRowConverter.java index 882288524cd1..c2a3fb04abba 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/converter/AbstractJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/converter/AbstractJdbcRowConverter.java @@ -114,11 +114,9 @@ public SeaTunnelRow toInternal(ResultSet rs, TableSchema tableSchema) throws SQL fields[fieldIndex] = readTime(rs, resultSetIndex); break; case TIMESTAMP: - Timestamp sqlTimestamp = JdbcFieldTypeUtils.getTimestamp(rs, resultSetIndex); - fields[fieldIndex] = - Optional.ofNullable(sqlTimestamp) - .map(e -> e.toLocalDateTime()) - .orElse(null); + // Use getLocalDateTime() which avoids JVM-default-timezone influence. + // See JdbcFieldTypeUtils.getLocalDateTime() for full strategy details. + fields[fieldIndex] = JdbcFieldTypeUtils.getLocalDateTime(rs, resultSetIndex); break; case TIMESTAMP_TZ: OffsetDateTime offsetDateTime = @@ -311,12 +309,19 @@ protected void setValueToStatementByDataType( case TIMESTAMP_TZ: OffsetDateTime offsetDateTime = (OffsetDateTime) value; try { - // Try to use setObject first for better timezone support + // Try to use setObject first for better timezone support. + // Modern Oracle JDBC (12.2+) and most other drivers accept OffsetDateTime + // directly and preserve the offset accurately. statement.setObject(statementIndex, offsetDateTime); } catch (SQLException e) { - // Fallback to setTimestamp if setObject is not supported + // Fallback for older drivers that do not support OffsetDateTime via setObject. + // Pass an explicit UTC Calendar so the driver does not apply the JVM default + // timezone when interpreting the Timestamp, which would silently corrupt the + // stored instant in any non-UTC environment. statement.setTimestamp( - statementIndex, Timestamp.from(offsetDateTime.toInstant())); + statementIndex, + Timestamp.from(offsetDateTime.toInstant()), + java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC"))); } break; case BYTES: diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverter.java index 1ed2c3a8b8d5..8ea3de9b6f45 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverter.java @@ -311,7 +311,7 @@ public Column convert(BasicTypeDefine typeDefine) { builder.sourceType( String.format("DATETIME(%s) WITH TIME ZONE", typeDefine.getScale())); } - builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); builder.scale(typeDefine.getScale()); break; default: @@ -491,6 +491,28 @@ public BasicTypeDefine reconvert(Column column) { builder.columnType(DM_TIMESTAMP); } break; + case TIMESTAMP_TZ: + builder.dataType(DM_DATETIME_WITH_TIME_ZONE); + if (column.getScale() != null && column.getScale() > 0) { + Integer timestampTzScale = column.getScale(); + if (timestampTzScale > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type datetime_tz({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to datetime_tz({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType( + String.format("DATETIME(%s) WITH TIME ZONE", timestampTzScale)); + builder.scale(timestampTzScale); + } else { + builder.columnType(DM_DATETIME_WITH_TIME_ZONE); + } + break; default: throw CommonError.convertToConnectorTypeError( DatabaseIdentifier.DAMENG, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverter.java index 6debaf90f3fa..88dddab00518 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverter.java @@ -164,9 +164,11 @@ public Column convert(BasicTypeDefine typeDefine) { builder.dataType(LocalTimeType.LOCAL_TIME_TYPE); break; case DUCKDB_TIMESTAMP: - case DUCKDB_TIMESTAMP_WITH_TZ: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); break; + case DUCKDB_TIMESTAMP_WITH_TZ: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + break; case DUCKDB_INTERVAL: builder.dataType(BasicType.STRING_TYPE); builder.columnLength(50L); @@ -281,6 +283,10 @@ public BasicTypeDefine reconvert(Column column) { builder.columnType(DUCKDB_TIMESTAMP); builder.dataType(DUCKDB_TIMESTAMP); break; + case TIMESTAMP_TZ: + builder.columnType(DUCKDB_TIMESTAMP_WITH_TZ); + builder.dataType(DUCKDB_TIMESTAMP_WITH_TZ); + break; case BYTES: builder.columnType(DUCKDB_BLOB); builder.dataType(DUCKDB_BLOB); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverter.java index f46fccda931d..abf624fe8b86 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverter.java @@ -79,7 +79,7 @@ public Column convert(BasicTypeDefine typeDefine) { case MySqlTypeConverter.MYSQL_YEAR_UNSIGNED: builder.dataType(BasicType.INT_TYPE); break; - // DATETIME not in PG (PG has TIMESTAMP) + // DATETIME not in PG (PG has TIMESTAMP) — NTZ case MySqlTypeConverter.MYSQL_DATETIME: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); if (typeDefine.getScale() != null @@ -155,10 +155,14 @@ public Column convert(BasicTypeDefine typeDefine) { builder.columnLength((long) (1024 * 1024 * 1024)); } break; - // SQLServer compatibility - SQLServer specific types + // MySQL TIMESTAMP — LTZ (timezone-aware) + case MySqlTypeConverter.MYSQL_TIMESTAMP: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; + // SQLServer compatibility - NTZ types case SqlServerTypeConverter.SQLSERVER_DATETIME2: case SqlServerTypeConverter.SQLSERVER_SMALLDATETIME: - case SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); if (typeDefine.getScale() != null && typeDefine.getScale() > MAX_TIMESTAMP_SCALE) { @@ -175,6 +179,11 @@ public Column convert(BasicTypeDefine typeDefine) { builder.scale(typeDefine.getScale()); } break; + // SQLServer DATETIMEOFFSET — LTZ (timezone-aware) + case SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; case KB_TINYINT: builder.dataType(BasicType.BYTE_TYPE); break; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverter.java index 4076a2303e6a..f1344cb58011 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverter.java @@ -317,10 +317,14 @@ public Column convert(BasicTypeDefine typeDefine) { builder.scale(typeDefine.getScale()); break; case MYSQL_DATETIME: - case MYSQL_TIMESTAMP: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); builder.scale(typeDefine.getScale()); break; + case MYSQL_TIMESTAMP: + // MySQL TIMESTAMP is LTZ (stored as UTC, displayed in session timezone) + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; default: throw CommonError.convertToSeaTunnelTypeError( DatabaseIdentifier.MYSQL, mysqlDataType, typeDefine.getName()); @@ -537,6 +541,31 @@ public BasicTypeDefine reconvert(Column column) { builder.columnType(MYSQL_DATETIME); } break; + case TIMESTAMP_TZ: + // TIMESTAMP_TZ (LTZ) maps back to MySQL TIMESTAMP + builder.nativeType(MysqlType.TIMESTAMP); + builder.dataType(MYSQL_TIMESTAMP); + if (version.isAtOrBefore(MySqlVersion.V_5_5)) { + builder.columnType(MYSQL_TIMESTAMP); + } else if (column.getScale() != null && column.getScale() > 0) { + int timestampTzScale = column.getScale(); + if (timestampTzScale > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type timestamp({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType(String.format("%s(%s)", MYSQL_TIMESTAMP, timestampTzScale)); + builder.scale(timestampTzScale); + } else { + builder.columnType(MYSQL_TIMESTAMP); + } + break; default: throw CommonError.convertToConnectorTypeError( DatabaseIdentifier.MYSQL, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java index e3463fc914d9..ca426551b4c1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java @@ -291,10 +291,13 @@ public Column convert(BasicTypeDefine typeDefine) { builder.scale(typeDefine.getScale()); break; case MYSQL_DATETIME: - case MYSQL_TIMESTAMP: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); builder.scale(typeDefine.getScale()); break; + case MYSQL_TIMESTAMP: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; case VECTOR_NAME: String columnType = typeDefine.getColumnType().toUpperCase(); if (columnType.startsWith("VECTOR(") && columnType.endsWith(")")) { @@ -518,6 +521,28 @@ public BasicTypeDefine reconvert(Column column) { builder.columnType(MYSQL_DATETIME); } break; + case TIMESTAMP_TZ: + builder.nativeType(OceanBaseMysqlType.TIMESTAMP); + builder.dataType(MYSQL_TIMESTAMP); + if (column.getScale() != null && column.getScale() > 0) { + int timestampTzScale = column.getScale(); + if (timestampTzScale > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type timestamp({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType(String.format("%s(%s)", MYSQL_TIMESTAMP, timestampTzScale)); + builder.scale(timestampTzScale); + } else { + builder.columnType(MYSQL_TIMESTAMP); + } + break; case FLOAT_VECTOR: builder.nativeType(VECTOR_NAME); builder.columnType(String.format("%s(%s)", VECTOR_NAME, column.getScale())); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java index e51211e8df7f..6939aee4d155 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java @@ -126,6 +126,10 @@ public SeaTunnelRow toInternal(ResultSet rs, TableSchema tableSchema) throws SQL .map(e -> e.toLocalDateTime()) .orElse(null); break; + case TIMESTAMP_TZ: + // OceanBase MySQL TIMESTAMP (LTZ) → read as OffsetDateTime + fields[fieldIndex] = JdbcFieldTypeUtils.getOffsetDateTime(rs, resultSetIndex); + break; case BYTES: fields[fieldIndex] = JdbcFieldTypeUtils.getBytes(rs, resultSetIndex); break; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleJdbcRowConverter.java index 8d91e311083f..c0a3d8865335 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleJdbcRowConverter.java @@ -66,6 +66,12 @@ protected void setValueToStatementByDataType( } else { statement.setString(statementIndex, (String) value); } + } else if (seaTunnelDataType.getSqlType().equals(SqlType.TIMESTAMP_TZ)) { + // Delegate to AbstractJdbcRowConverter which uses setObject() first (preserving the + // offset for Oracle JDBC 12.2+) and falls back to setTimestamp() with an explicit UTC + // Calendar for older drivers, avoiding JVM-default-timezone corruption. + super.setValueToStatementByDataType( + value, statement, seaTunnelDataType, statementIndex, sourceType); } else { super.setValueToStatementByDataType( value, statement, seaTunnelDataType, statementIndex, sourceType); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java index 6d2b56992161..2b91fdd48449 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java @@ -240,9 +240,18 @@ public Column convert(BasicTypeDefine typeDefine) { builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); break; case ORACLE_TIMESTAMP: + // TIMESTAMP without timezone is NTZ + builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + if (typeDefine.getScale() == null) { + builder.scale(TIMESTAMP_DEFAULT_SCALE); + } else { + builder.scale(typeDefine.getScale()); + } + break; case ORACLE_TIMESTAMP_WITH_TIME_ZONE: case ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE: - builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + // TIMESTAMP WITH (LOCAL) TIME ZONE is LTZ + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); if (typeDefine.getScale() == null) { builder.scale(TIMESTAMP_DEFAULT_SCALE); } else { @@ -376,25 +385,48 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(ORACLE_DATE); break; case TIMESTAMP: + // NTZ: maps to ORACLE_TIMESTAMP (without timezone) if (column.getScale() == null || column.getScale() <= 0) { - builder.columnType(ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE); + builder.columnType(ORACLE_TIMESTAMP); } else { int timestampScale = column.getScale(); - if (column.getScale() > MAX_TIMESTAMP_SCALE) { - timestampScale = MAX_TIMESTAMP_SCALE; + if (timestampScale > MAX_TIMESTAMP_SCALE) { log.warn( "The timestamp column {} type timestamp({}) is out of range, " + "which exceeds the maximum scale of {}, " + "it will be converted to timestamp({})", column.getName(), - column.getScale(), + timestampScale, MAX_TIMESTAMP_SCALE, - timestampScale); + MAX_TIMESTAMP_SCALE); + timestampScale = MAX_TIMESTAMP_SCALE; } - builder.columnType( - String.format("TIMESTAMP(%s) WITH LOCAL TIME ZONE", timestampScale)); + builder.columnType(String.format("TIMESTAMP(%s)", timestampScale)); builder.scale(timestampScale); } + builder.dataType(ORACLE_TIMESTAMP); + break; + case TIMESTAMP_TZ: + // LTZ: maps to ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE + if (column.getScale() == null || column.getScale() <= 0) { + builder.columnType(ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE); + } else { + int timestampTzScale = column.getScale(); + if (timestampTzScale > MAX_TIMESTAMP_SCALE) { + log.warn( + "The timestamp_tz column {} type timestamp({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp({})", + column.getName(), + timestampTzScale, + MAX_TIMESTAMP_SCALE, + MAX_TIMESTAMP_SCALE); + timestampTzScale = MAX_TIMESTAMP_SCALE; + } + builder.columnType( + String.format("TIMESTAMP(%s) WITH LOCAL TIME ZONE", timestampTzScale)); + builder.scale(timestampTzScale); + } builder.dataType(ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE); break; default: diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresJdbcRowConverter.java index a19c4814f1ed..ffe1ffec9674 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresJdbcRowConverter.java @@ -150,11 +150,9 @@ public SeaTunnelRow toInternal(ResultSet rs, TableSchema tableSchema) throws SQL Optional.ofNullable(sqlTime).map(e -> e.toLocalTime()).orElse(null); break; case TIMESTAMP: - Timestamp sqlTimestamp = JdbcFieldTypeUtils.getTimestamp(rs, resultSetIndex); - fields[fieldIndex] = - Optional.ofNullable(sqlTimestamp) - .map(e -> e.toLocalDateTime()) - .orElse(null); + // Use getLocalDateTime() which avoids JVM-default-timezone influence. + // See JdbcFieldTypeUtils.getLocalDateTime() for full strategy details. + fields[fieldIndex] = JdbcFieldTypeUtils.getLocalDateTime(rs, resultSetIndex); break; case TIMESTAMP_TZ: // Enhanced PostgreSQL TIMESTAMP_TZ handling @@ -390,9 +388,6 @@ private OffsetDateTime getPostgresOffsetDateTime(ResultSet rs, int columnIndex) if (obj instanceof OffsetDateTime) { return (OffsetDateTime) obj; } - if (obj instanceof Timestamp) { - return ((Timestamp) obj).toInstant().atOffset(ZoneOffset.UTC); - } if (obj instanceof java.time.ZonedDateTime) { return ((java.time.ZonedDateTime) obj).toOffsetDateTime(); } @@ -400,6 +395,22 @@ private OffsetDateTime getPostgresOffsetDateTime(ResultSet rs, int columnIndex) return ((java.util.Date) obj).toInstant().atOffset(ZoneOffset.UTC); } + // Handle java.sql.Timestamp: avoid using toInstant() directly because the Timestamp + // was constructed with JVM-default-timezone semantics, which would shift the value. + // Instead, re-read as string and parse the timezone info explicitly. + if (obj instanceof Timestamp) { + String strVal = rs.getString(columnIndex); + if (strVal == null) { + return null; + } + try { + return JdbcFieldTypeUtils.parseOffsetDateTimeFromString(strVal); + } catch (Exception e) { + // Last resort: fall back to instant-based conversion + return ((Timestamp) obj).toInstant().atOffset(ZoneOffset.UTC); + } + } + // Remaining PostgreSQL-specific or driver types: fall back to string representation return parseTimestampFromObjectString(obj); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverter.java index f48d0c77cb4d..f82291198cef 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverter.java @@ -177,7 +177,7 @@ public Column convert(BasicTypeDefine typeDefine) { break; case REDSHIFT_TIMESTAMPTZ: builder.sourceType(REDSHIFT_TIMESTAMPTZ); - builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); builder.scale(MAX_TIMESTAMP_SCALE); break; default: @@ -373,6 +373,23 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(REDSHIFT_TIMESTAMP); builder.scale(timestampScale); break; + case TIMESTAMP_TZ: + Integer timestampTzScale = column.getScale(); + if (timestampTzScale != null && timestampTzScale > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type timestamptz({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamptz({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType(REDSHIFT_TIMESTAMPTZ); + builder.dataType(REDSHIFT_TIMESTAMPTZ); + builder.scale(timestampTzScale); + break; case MAP: case ARRAY: case ROW: diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java index c9e4adcadd29..ac0a74624771 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java @@ -171,12 +171,15 @@ public Column convert(BasicTypeDefine typeDefine) { break; case SNOWFLAKE_DATE_TIME: case SNOWFLAKE_TIMESTAMP: - case SNOWFLAKE_TIMESTAMP_LTZ: case SNOWFLAKE_TIMESTAMP_NTZ: - case SNOWFLAKE_TIMESTAMP_TZ: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); builder.scale(9); break; + case SNOWFLAKE_TIMESTAMP_LTZ: + case SNOWFLAKE_TIMESTAMP_TZ: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + builder.scale(9); + break; default: throw CommonError.convertToSeaTunnelTypeError( DatabaseIdentifier.SNOWFLAKE, dataType, typeDefine.getName()); @@ -306,7 +309,7 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(SNOWFLAKE_GEOMETRY); break; case TIME: - if (column.getScale() > 9) { + if (column.getScale() != null && column.getScale() > 9) { log.warn( "The timestamp column {} type time({}) is out of range, " + "which exceeds the maximum scale of {}, " @@ -320,7 +323,7 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(SNOWFLAKE_TIME); break; case TIMESTAMP: - if (column.getScale() > 9) { + if (column.getScale() != null && column.getScale() > 9) { log.warn( "The timestamp column {} type timestamp({}) is out of range, " + "which exceeds the maximum scale of {}, " @@ -330,8 +333,22 @@ public BasicTypeDefine reconvert(Column column) { 9, 9); } - builder.columnType(SNOWFLAKE_TIMESTAMP); - builder.dataType(SNOWFLAKE_TIMESTAMP); + builder.columnType(SNOWFLAKE_TIMESTAMP_NTZ); + builder.dataType(SNOWFLAKE_TIMESTAMP_NTZ); + break; + case TIMESTAMP_TZ: + if (column.getScale() != null && column.getScale() > 9) { + log.warn( + "The timestamp_tz column {} type timestamp_tz({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp_tz({})", + column.getName(), + column.getScale(), + 9, + 9); + } + builder.columnType(SNOWFLAKE_TIMESTAMP_TZ); + builder.dataType(SNOWFLAKE_TIMESTAMP_TZ); break; default: throw CommonError.convertToSeaTunnelTypeError( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverter.java index 59eb19cc4ad5..fd4e9d913408 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverter.java @@ -290,9 +290,10 @@ public Column convert(BasicTypeDefine typeDefine) { builder.scale(typeDefine.getScale()); break; case SQLSERVER_DATETIMEOFFSET: + // DATETIMEOFFSET is LTZ (includes timezone offset) builder.sourceType( String.format("%s(%s)", SQLSERVER_DATETIMEOFFSET, typeDefine.getScale())); - builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); builder.scale(typeDefine.getScale()); break; case SQLSERVER_SMALLDATETIME: @@ -458,6 +459,7 @@ public BasicTypeDefine reconvert(Column column) { builder.dataType(SQLSERVER_TIME); break; case TIMESTAMP: + // NTZ: maps to DATETIME2 if (column.getScale() != null && column.getScale() > 0) { int timestampScale = column.getScale(); if (timestampScale > MAX_TIMESTAMP_SCALE) { @@ -479,6 +481,29 @@ public BasicTypeDefine reconvert(Column column) { } builder.dataType(SQLSERVER_DATETIME2); break; + case TIMESTAMP_TZ: + // LTZ: maps to DATETIMEOFFSET + if (column.getScale() != null && column.getScale() > 0) { + int timestampTzScale = column.getScale(); + if (timestampTzScale > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type datetimeoffset({}) is out of" + + " range, which exceeds the maximum scale of {}, " + + "it will be converted to datetimeoffset({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType( + String.format("%s(%s)", SQLSERVER_DATETIMEOFFSET, timestampTzScale)); + builder.scale(timestampTzScale); + } else { + builder.columnType(SQLSERVER_DATETIMEOFFSET); + } + builder.dataType(SQLSERVER_DATETIMEOFFSET); + break; default: throw CommonError.convertToConnectorTypeError( DatabaseIdentifier.SQLSERVER, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java index 28826e700c1e..304465f3cbcc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java @@ -49,6 +49,24 @@ protected LocalTime readTime(ResultSet rs, int resultSetIndex) throws SQLExcepti .map(e -> e.toLocalDateTime().toLocalTime()) .orElse(null); } + + @Override + protected void setValueToStatementByDataType( + Object value, + PreparedStatement statement, + SeaTunnelDataType seaTunnelDataType, + int statementIndex, + @Nullable String sourceType) + throws SQLException { + if (seaTunnelDataType.getSqlType().equals(SqlType.TIMESTAMP_TZ)) { + // DATETIMEOFFSET supports OffsetDateTime directly via setObject + statement.setObject(statementIndex, (java.time.OffsetDateTime) value); + } else { + super.setValueToStatementByDataType( + value, statement, seaTunnelDataType, statementIndex, sourceType); + } + } + /** * {@inheritDoc} * diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java index 4590761965c8..75b14f95a80d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java @@ -17,13 +17,44 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SqlType; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import javax.annotation.Nullable; + +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.time.OffsetDateTime; + public class XuguJdbcRowConverter extends AbstractJdbcRowConverter { @Override public String converterName() { return DatabaseIdentifier.XUGU; } + + @Override + protected void setValueToStatementByDataType( + Object value, + PreparedStatement statement, + SeaTunnelDataType seaTunnelDataType, + int statementIndex, + @Nullable String sourceType) + throws SQLException { + if (seaTunnelDataType.getSqlType().equals(SqlType.TIMESTAMP_TZ)) { + // Xugu JDBC driver has a batch execution bug when receiving OffsetDateTime or + // a timezone-formatted string for TIMESTAMP WITH TIME ZONE columns ([E19138]). + // Fallback: convert OffsetDateTime to LocalDateTime (dropping timezone offset) + // and write as a plain Timestamp. Timezone info is lost on the Xugu Sink side, + // but this avoids the driver crash. + OffsetDateTime odt = (OffsetDateTime) value; + statement.setTimestamp(statementIndex, Timestamp.valueOf(odt.toLocalDateTime())); + } else { + super.setValueToStatementByDataType( + value, statement, seaTunnelDataType, statementIndex, sourceType); + } + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java index 38b0553124d9..8074e6ce4e72 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java @@ -187,11 +187,12 @@ public Column convert(BasicTypeDefine typeDefine) { builder.dataType(LocalTimeType.LOCAL_TIME_TYPE); break; case XUGU_DATETIME: - case XUGU_DATETIME_WITH_TIME_ZONE: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); break; + case XUGU_DATETIME_WITH_TIME_ZONE: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + break; case XUGU_TIMESTAMP: - case XUGU_TIMESTAMP_WITH_TIME_ZONE: builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); if (typeDefine.getScale() == null) { builder.scale(TIMESTAMP_DEFAULT_SCALE); @@ -199,6 +200,14 @@ public Column convert(BasicTypeDefine typeDefine) { builder.scale(typeDefine.getScale()); } break; + case XUGU_TIMESTAMP_WITH_TIME_ZONE: + builder.dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE); + if (typeDefine.getScale() == null) { + builder.scale(TIMESTAMP_DEFAULT_SCALE); + } else { + builder.scale(typeDefine.getScale()); + } + break; default: throw CommonError.convertToSeaTunnelTypeError( DatabaseIdentifier.XUGU, xuguDataType, typeDefine.getName()); @@ -373,6 +382,28 @@ public BasicTypeDefine reconvert(Column column) { } builder.dataType(XUGU_TIMESTAMP); break; + case TIMESTAMP_TZ: + if (column.getScale() == null || column.getScale() <= 0) { + builder.columnType(XUGU_TIMESTAMP_WITH_TIME_ZONE); + } else { + int timestampTzScale = column.getScale(); + if (column.getScale() > MAX_TIMESTAMP_SCALE) { + timestampTzScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp_tz column {} type timestamp_tz({}) is out of" + + " range, which exceeds the maximum scale of {}, " + + "it will be converted to timestamp_tz({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampTzScale); + } + builder.columnType( + String.format("TIMESTAMP(%s) WITH TIME ZONE", timestampTzScale)); + builder.scale(timestampTzScale); + } + builder.dataType(XUGU_TIMESTAMP_WITH_TIME_ZONE); + break; default: throw CommonError.convertToConnectorTypeError( DatabaseIdentifier.XUGU, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtils.java index 92874511714e..08e875a462eb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtils.java @@ -23,10 +23,13 @@ import java.sql.Time; import java.sql.Timestamp; import java.time.Instant; +import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.format.DateTimeParseException; +import java.util.Calendar; +import java.util.TimeZone; public final class JdbcFieldTypeUtils { @@ -96,6 +99,52 @@ public static Timestamp getTimestamp(ResultSet resultSet, int columnIndex) throw return resultSet.getTimestamp(columnIndex); } + /** + * Reads a NTZ (No Time Zone) timestamp column as {@link LocalDateTime}, free from JVM default + * timezone influence. + * + *

Strategy: + * + *

    + *
  1. Try {@code getObject(index, LocalDateTime.class)} first — supported by modern JDBC + * drivers (PostgreSQL ≥ 42.2, MySQL Connector/J ≥ 8.0, MariaDB Connector/J ≥ 3.x). This + * returns the wall-clock value exactly as stored, with no timezone conversion. + *
  2. Try {@code getTimestamp(index, utcCalendar)}: passing a UTC {@link Calendar} forces the + * driver to treat the raw bytes as UTC epoch millis, then {@link + * Timestamp#toLocalDateTime()} reconstructs the wall-clock via UTC — again + * timezone-neutral. A new {@link Calendar} is created per call to avoid thread-safety + * issues with a shared mutable instance. Not supported by all drivers (e.g. Hive JDBC). + *
  3. Last resort: plain {@code getTimestamp(index)} — may be affected by JVM timezone for + * drivers that apply session/JVM timezone conversion internally (e.g. Hive JDBC). + *
+ * + * @param resultSet the JDBC result set + * @param columnIndex 1-based column index + * @return the wall-clock {@link LocalDateTime} exactly as stored in the DB, or {@code null} + */ + public static LocalDateTime getLocalDateTime(ResultSet resultSet, int columnIndex) + throws SQLException { + // Prefer the modern JDBC 4.2 API — returns wall-clock value directly, no TZ involved + try { + return resultSet.getObject(columnIndex, LocalDateTime.class); + } catch (SQLException | UnsupportedOperationException ignored) { + // Driver does not support getObject(index, LocalDateTime.class) — fall back + } + // Try UTC Calendar to avoid JVM-default-timezone influence. + // A new Calendar is created per call to avoid thread-safety issues with a shared instance. + try { + Timestamp ts = + resultSet.getTimestamp( + columnIndex, Calendar.getInstance(TimeZone.getTimeZone("UTC"))); + return ts == null ? null : ts.toLocalDateTime(); + } catch (SQLException | UnsupportedOperationException ignored) { + // Driver does not support getTimestamp(index, Calendar) — fall back (e.g. Hive JDBC) + } + // Last resort: plain getTimestamp() — may be affected by JVM timezone for some drivers + Timestamp ts = resultSet.getTimestamp(columnIndex); + return ts == null ? null : ts.toLocalDateTime(); + } + public static byte[] getBytes(ResultSet resultSet, int columnIndex) throws SQLException { return resultSet.getBytes(columnIndex); } @@ -123,8 +172,20 @@ public static OffsetDateTime getOffsetDateTime(ResultSet resultSet, int columnIn } // Handle java.sql.Timestamp + // Avoid using Timestamp.toInstant() directly because the Timestamp was constructed + // with JVM-default-timezone semantics, which would shift the value by the JVM offset. + // Instead, try to re-read the column as a string and parse it with timezone info preserved. if (obj instanceof Timestamp) { - return ((Timestamp) obj).toInstant().atOffset(ZoneOffset.UTC); + String strVal = resultSet.getString(columnIndex); + if (strVal == null) { + return null; + } + try { + return parseOffsetDateTimeFromString(strVal); + } catch (Exception e) { + // Last resort: use the instant-based conversion (may shift by JVM offset) + return ((Timestamp) obj).toInstant().atOffset(ZoneOffset.UTC); + } } // Handle java.util.Date @@ -137,6 +198,23 @@ public static OffsetDateTime getOffsetDateTime(ResultSet resultSet, int columnIn return Instant.ofEpochMilli((Long) obj).atOffset(ZoneOffset.UTC); } + // Handle Oracle-specific TIMESTAMPLTZ / TIMESTAMPTZ types. + // oracle.sql.TIMESTAMPLTZ and oracle.sql.TIMESTAMPTZ do not implement standard interfaces + // and their toString() returns the Java object reference (e.g. + // "oracle.sql.TIMESTAMPLTZ@xxx"). + // Fall back to ResultSet.getTimestamp() which the Oracle JDBC driver converts correctly. + String objClassName = obj.getClass().getName(); + if (objClassName.equals("oracle.sql.TIMESTAMPLTZ") + || objClassName.equals("oracle.sql.TIMESTAMPTZ")) { + Timestamp oracleTs = + resultSet.getTimestamp( + columnIndex, Calendar.getInstance(TimeZone.getTimeZone("UTC"))); + if (oracleTs == null) { + return null; + } + return oracleTs.toInstant().atOffset(ZoneOffset.UTC); + } + // Try to parse as string String str = obj.toString(); try { @@ -201,7 +279,11 @@ private static String normalizeOffsetDateTimeString(String value) { if (normalized.endsWith(" UTC")) { normalized = normalized.substring(0, normalized.length() - 4) + "Z"; } - normalized = normalized.replace(' ', 'T'); + // Only replace the first space (between date and time) with 'T'. + // Then remove any space before the timezone offset (+/-). + // e.g. "2026-04-15 04:53:44.407 +08:00" → "2026-04-15T04:53:44.407+08:00" + normalized = normalized.replaceFirst(" ", "T"); + normalized = normalized.replace(" +", "+").replace(" -", "-"); if (normalized.matches(".*[+-]\\d{2}$")) { normalized = normalized + ":00"; } else if (normalized.matches(".*[+-]\\d{4}$")) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java index 255ef1ae9e00..54589f8cf579 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java @@ -115,14 +115,17 @@ public void testBuild() { List sqls = oracleCreateTableSqlBuilder.build(tablePath); String createTableSql = sqls.get(0); // create table sql is change; The old unit tests are no longer applicable + // After the NTZ/LTZ fix (#10685), LOCAL_DATE_TIME_TYPE (NTZ) maps to TIMESTAMP + // (without timezone), while OFFSET_DATE_TIME_TYPE (LTZ) maps to TIMESTAMP WITH LOCAL + // TIME ZONE. String expect = "CREATE TABLE \"test_table\" (\n" + "\"id\" INTEGER NOT NULL,\n" + "\"name\" VARCHAR2(128) NOT NULL,\n" + "\"age\" INTEGER,\n" + "\"blob_v\" BLOB,\n" - + "\"createTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" - + "\"lastUpdateTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" + + "\"createTime\" TIMESTAMP,\n" + + "\"lastUpdateTime\" TIMESTAMP,\n" + "CONSTRAINT id_9a8b PRIMARY KEY (\"id\")\n" + ")"; @@ -146,8 +149,8 @@ public void testBuild() { + "\"name\" VARCHAR2(128) NOT NULL,\n" + "\"age\" INTEGER,\n" + "\"blob_v\" BLOB,\n" - + "\"createTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" - + "\"lastUpdateTime\" TIMESTAMP WITH LOCAL TIME ZONE\n" + + "\"createTime\" TIMESTAMP,\n" + + "\"lastUpdateTime\" TIMESTAMP\n" + ")"; CONSOLE.println(expectSkipIndex); Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverterTest.java index 0fa537d08fa2..75d42d071d33 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeConverterTest.java @@ -600,7 +600,8 @@ public void testConvertDatetime() { .build(); column = DmdbTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // DATETIME WITH TIME ZONE is LTZ → maps to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getScale(), column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType().toLowerCase()); @@ -613,7 +614,8 @@ public void testConvertDatetime() { .build(); column = DmdbTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // DATETIME WITH TIME ZONE is LTZ → maps to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getScale(), column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType().toLowerCase()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverterTest.java index 4024612d8a1e..bcb0b00f15f5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverterTest.java @@ -242,7 +242,8 @@ void testConvertTimestamp() { @Test void testConvertTimestampWithTimezone() { Column column = convert("f_timestamp_tz", "timestamp with time zone"); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP WITH TIME ZONE is LTZ → must map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); } @Test @@ -467,6 +468,21 @@ void testReconvertTimestamp() { Assertions.assertEquals(DuckDBTypeConverter.DUCKDB_TIMESTAMP, typeDefine.getDataType()); } + @Test + void testReconvertTimestampTz() { + BasicTypeDefine typeDefine = + DuckDBTypeConverter.INSTANCE.reconvert( + PhysicalColumn.builder() + .name("f_timestamp_tz") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build()); + // OFFSET_DATE_TIME_TYPE → DUCKDB_TIMESTAMP_WITH_TZ + Assertions.assertEquals( + DuckDBTypeConverter.DUCKDB_TIMESTAMP_WITH_TZ, typeDefine.getColumnType()); + Assertions.assertEquals( + DuckDBTypeConverter.DUCKDB_TIMESTAMP_WITH_TZ, typeDefine.getDataType()); + } + @Test void testReconvertUnsupportedType() { Column mapColumn = diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverterTest.java index 17176db80986..03302b48b9ea 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/kingbase/KingbaseTypeConverterTest.java @@ -862,4 +862,19 @@ public void testReconvertArray() { KingbaseTypeConverter.PG_SMALLINT_ARRAY, typeDefine.getColumnType()); Assertions.assertEquals(KingbaseTypeConverter.PG_SMALLINT_ARRAY, typeDefine.getDataType()); } + + @Test + public void testConvertSqlServerDatetimeoffsetIsLtz() { + // SQL Server DATETIMEOFFSET (LTZ) compatibility in Kingbase → must map to + // OFFSET_DATE_TIME_TYPE + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("DATETIMEOFFSET") + .dataType("DATETIMEOFFSET") + .build(); + Column column = KingbaseTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverterTest.java index 1093285d6ebf..1401a5894a0a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlTypeConverterTest.java @@ -626,6 +626,7 @@ public void testConvertDatetime() { @Test public void testConvertTimestamp() { + // MySQL TIMESTAMP is LTZ → should map to OFFSET_DATE_TIME_TYPE (TIMESTAMP_TZ) BasicTypeDefine typeDefine = BasicTypeDefine.builder() .name("test") @@ -634,7 +635,7 @@ public void testConvertTimestamp() { .build(); Column column = MySqlTypeConverter.DEFAULT_INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); typeDefine = @@ -646,7 +647,7 @@ public void testConvertTimestamp() { .build(); column = MySqlTypeConverter.DEFAULT_INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getScale(), column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); } @@ -1088,6 +1089,39 @@ public void testReconvertDatetimeForV55() { Assertions.assertEquals(MySqlTypeConverter.MYSQL_DATETIME, typeDefine.getDataType()); } + @Test + public void testReconvertTimestampTz() { + // TIMESTAMP_TZ (LTZ) should map back to MySQL TIMESTAMP + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = + MySqlTypeConverter.DEFAULT_INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(MysqlType.TIMESTAMP, typeDefine.getNativeType()); + Assertions.assertEquals(MySqlTypeConverter.MYSQL_TIMESTAMP, typeDefine.getColumnType()); + Assertions.assertEquals(MySqlTypeConverter.MYSQL_TIMESTAMP, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .scale(3) + .build(); + + typeDefine = MySqlTypeConverter.DEFAULT_INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(MysqlType.TIMESTAMP, typeDefine.getNativeType()); + Assertions.assertEquals( + String.format("%s(%s)", MySqlTypeConverter.MYSQL_TIMESTAMP, column.getScale()), + typeDefine.getColumnType()); + Assertions.assertEquals(MySqlTypeConverter.MYSQL_TIMESTAMP, typeDefine.getDataType()); + Assertions.assertEquals(column.getScale(), typeDefine.getScale()); + } + @Test public void testConvertSet() { BasicTypeDefine typeDefine = diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverterTest.java new file mode 100644 index 000000000000..6104e3b0baf9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverterTest.java @@ -0,0 +1,93 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.type.LocalTimeType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +/** + * Unit tests for {@link OceanBaseMySqlTypeConverter} verifying the NTZ/LTZ timestamp split + * introduced by the fix for https://github.com/apache/seatunnel/issues/10685. + */ +public class OceanBaseMySqlTypeConverterTest { + + @Test + public void testConvertDatetimeIsNtz() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("DATETIME") + .dataType("DATETIME") + .build(); + Column column = OceanBaseMySqlTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + // DATETIME is NTZ → must map to LOCAL_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + } + + @Test + public void testConvertTimestampIsLtz() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("TIMESTAMP") + .dataType("TIMESTAMP") + .build(); + Column column = OceanBaseMySqlTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + // TIMESTAMP is LTZ → must map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); + } + + @Test + public void testReconvertDatetime() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .build(); + BasicTypeDefine typeDefine = OceanBaseMySqlTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + // LOCAL_DATE_TIME_TYPE (NTZ) → DATETIME + Assertions.assertEquals( + OceanBaseMySqlTypeConverter.MYSQL_DATETIME, typeDefine.getColumnType()); + Assertions.assertEquals( + OceanBaseMySqlTypeConverter.MYSQL_DATETIME, typeDefine.getDataType()); + } + + @Test + public void testReconvertDatetimeTz() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + BasicTypeDefine typeDefine = OceanBaseMySqlTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + // OFFSET_DATE_TIME_TYPE (LTZ) → TIMESTAMP + Assertions.assertEquals( + OceanBaseMySqlTypeConverter.MYSQL_TIMESTAMP, typeDefine.getColumnType()); + Assertions.assertEquals( + OceanBaseMySqlTypeConverter.MYSQL_TIMESTAMP, typeDefine.getDataType()); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java index eddbddde89f8..a25dd327a7d5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java @@ -612,7 +612,8 @@ public void testConvertDatetime() { column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP WITH TIME ZONE is LTZ → should map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(6, column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -626,7 +627,8 @@ public void testConvertDatetime() { column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP WITH LOCAL TIME ZONE is LTZ → should map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(6, column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); } @@ -882,12 +884,42 @@ public void testReconvertDate() { @Test public void testReconvertDatetime() { + // TIMESTAMP (NTZ) should map to ORACLE_TIMESTAMP Column column = PhysicalColumn.builder() .name("test") .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) .build(); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(OracleTypeConverter.ORACLE_TIMESTAMP, typeDefine.getColumnType()); + Assertions.assertEquals(OracleTypeConverter.ORACLE_TIMESTAMP, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .scale(3) + .build(); + + typeDefine = INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("TIMESTAMP(%s)", column.getScale()), typeDefine.getColumnType()); + Assertions.assertEquals(OracleTypeConverter.ORACLE_TIMESTAMP, typeDefine.getDataType()); + Assertions.assertEquals(column.getScale(), typeDefine.getScale()); + } + + @Test + public void testReconvertTimestampTz() { + // TIMESTAMP_TZ (LTZ) should map to ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( @@ -900,7 +932,7 @@ public void testReconvertDatetime() { column = PhysicalColumn.builder() .name("test") - .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) .scale(3) .build(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverterTest.java index e491e1532840..8b3e9fc1f92d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverterTest.java @@ -359,7 +359,8 @@ public void testConvertTimestamp() { .build(); column = RedshiftTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP WITH TIME ZONE is LTZ → must map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(RedshiftTypeConverter.MAX_TIMESTAMP_SCALE, column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); } @@ -640,4 +641,35 @@ public void testReconvertDatetime() { Assertions.assertEquals(RedshiftTypeConverter.REDSHIFT_TIMESTAMP, typeDefine.getDataType()); Assertions.assertEquals(RedshiftTypeConverter.MAX_TIMESTAMP_SCALE, typeDefine.getScale()); } + + @Test + public void testReconvertDatetimeTz() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = RedshiftTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + RedshiftTypeConverter.REDSHIFT_TIMESTAMPTZ, typeDefine.getColumnType()); + Assertions.assertEquals( + RedshiftTypeConverter.REDSHIFT_TIMESTAMPTZ, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .scale(3) + .build(); + + typeDefine = RedshiftTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + RedshiftTypeConverter.REDSHIFT_TIMESTAMPTZ, typeDefine.getColumnType()); + Assertions.assertEquals( + RedshiftTypeConverter.REDSHIFT_TIMESTAMPTZ, typeDefine.getDataType()); + Assertions.assertEquals(3, typeDefine.getScale()); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverterTest.java index 308a80497671..6ebaae9aabb0 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerTypeConverterTest.java @@ -540,7 +540,8 @@ public void testConvertDatetime() { .build(); column = SqlServerTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // DATETIMEOFFSET is LTZ → should map to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getScale(), column.getScale()); Assertions.assertEquals( String.format("%s(%s)", typeDefine.getDataType(), typeDefine.getScale()), @@ -872,4 +873,39 @@ public void testReconvertDatetime() { SqlServerTypeConverter.SQLSERVER_DATETIME2, typeDefine.getDataType()); Assertions.assertEquals(7, typeDefine.getScale()); } + + @Test + public void testReconvertTimestampTz() { + // TIMESTAMP_TZ (LTZ) should map to DATETIMEOFFSET + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = SqlServerTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET, typeDefine.getColumnType()); + Assertions.assertEquals( + SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .scale(3) + .build(); + + typeDefine = SqlServerTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format( + "%s(%s)", + SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET, column.getScale()), + typeDefine.getColumnType()); + Assertions.assertEquals( + SqlServerTypeConverter.SQLSERVER_DATETIMEOFFSET, typeDefine.getDataType()); + Assertions.assertEquals(column.getScale(), typeDefine.getScale()); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java index 9dfd7079dfd2..41e3fafdec28 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java @@ -343,7 +343,8 @@ public void testConvertTimestamp() { .build(); column = XuguTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // DATETIME WITH TIME ZONE is LTZ → maps to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); typeDefine = @@ -379,7 +380,8 @@ public void testConvertTimestamp() { .build(); column = XuguTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP WITH TIME ZONE is LTZ → maps to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(3, column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -392,7 +394,8 @@ public void testConvertTimestamp() { .build(); column = XuguTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + // TIMESTAMP(n) WITH TIME ZONE is LTZ → maps to OFFSET_DATE_TIME_TYPE + Assertions.assertEquals(LocalTimeType.OFFSET_DATE_TIME_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getScale(), column.getScale()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtilsTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtilsTest.java index 8135b4b61877..f9ac9cb819b8 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtilsTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcFieldTypeUtilsTest.java @@ -40,6 +40,9 @@ public void testGetOffsetDateTimeFromTimestampUsesInstant() throws SQLException ResultSet rs = mock(ResultSet.class); when(rs.getObject(1)).thenReturn(timestamp); + // getString is called as the primary parse path for Timestamp objects; + // return an ISO-8601 string so parseOffsetDateTimeFromString can convert it. + when(rs.getString(1)).thenReturn("2025-01-01T00:00:00Z"); OffsetDateTime result = JdbcFieldTypeUtils.getOffsetDateTime(rs, 1); assertEquals(instant, result.toInstant()); diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java index 7766aef5464e..84e30b1efc05 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java @@ -49,15 +49,19 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.LocalZonedTimestampType; import org.apache.paimon.types.RowType; import org.apache.paimon.types.TimestampType; import org.apache.paimon.utils.DateTimeUtils; import java.math.BigDecimal; import java.math.RoundingMode; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -308,6 +312,24 @@ public static SeaTunnelRow convert( Timestamp timestamp = rowData.getTimestamp(i, precision); objects[i] = timestamp.toLocalDateTime(); break; + case TIMESTAMP_TZ: + int tzPrecision = LocalZonedTimestampType.DEFAULT_PRECISION; + Optional tzPrecisionOptional = + tableSchema.fields().stream() + .filter(dataField -> dataField.name().equals(fieldName)) + .findFirst(); + if (tzPrecisionOptional.isPresent() + && tzPrecisionOptional.get().type() + instanceof LocalZonedTimestampType) { + tzPrecision = + ((LocalZonedTimestampType) tzPrecisionOptional.get().type()) + .getPrecision(); + } + Timestamp tzTimestamp = rowData.getTimestamp(i, tzPrecision); + objects[i] = + Instant.ofEpochMilli(tzTimestamp.getMillisecond()) + .atOffset(ZoneOffset.UTC); + break; case ARRAY: InternalArray paimonArray = rowData.getArray(i); ArrayType seatunnelArray = (ArrayType) fieldType; @@ -445,6 +467,17 @@ public static InternalRow reconvert( binaryWriter.writeTimestamp( i, Timestamp.fromLocalDateTime(datetime), precision); break; + case TIMESTAMP_TZ: + DataField tzDataField = SchemaUtil.getDataField(sinkTotalFields, fieldName); + int tzWritePrecision = + ((LocalZonedTimestampType) tzDataField.type()).getPrecision(); + Instant instant = ((OffsetDateTime) fieldValue).toInstant(); + binaryWriter.writeTimestamp( + i, + Timestamp.fromEpochMillis( + instant.toEpochMilli(), instant.getNano() % 1_000_000), + tzWritePrecision); + break; case TIME: LocalTime time = (LocalTime) fieldValue; BinaryWriter.createValueSetter(DataTypes.TIME()) diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowTypeConverter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowTypeConverter.java index ef9d24c3d44a..09f1321c7600 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowTypeConverter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowTypeConverter.java @@ -290,6 +290,19 @@ public BasicTypeDefine visit(Column column) { builder.scale(timestampScale); builder.length(column.getColumnLength()); return builder.build(); + case TIMESTAMP_TZ: + int tzScale = + Objects.isNull(scale) + ? LocalZonedTimestampType.DEFAULT_PRECISION + : scale; + LocalZonedTimestampType tzType = + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(tzScale); + builder.nativeType(tzType.copy(column.isNullable())); + builder.dataType(tzType.getTypeRoot().name()); + builder.columnType(tzType.toString()); + builder.scale(tzScale); + builder.length(column.getColumnLength()); + return builder.build(); case TIME: int timeScale = Objects.isNull(scale) ? TimeType.DEFAULT_PRECISION : scale; TimeType timeType = DataTypes.TIME(timeScale); @@ -404,6 +417,9 @@ public DataType visit(String fieldName, SeaTunnelDataType dataType) { return DataTypes.TIME(TimeType.MAX_PRECISION); case TIMESTAMP: return DataTypes.TIMESTAMP(TimestampType.MAX_PRECISION); + case TIMESTAMP_TZ: + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE( + LocalZonedTimestampType.MAX_PRECISION); case MAP: SeaTunnelDataType keyType = ((org.apache.seatunnel.api.table.type.MapType) dataType) diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/datatypes/StarRocksTypeConverter.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/datatypes/StarRocksTypeConverter.java index 6d31e4337aea..16e552a09ce5 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/datatypes/StarRocksTypeConverter.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/datatypes/StarRocksTypeConverter.java @@ -319,6 +319,12 @@ public BasicTypeDefine reconvert(Column column) { builder.columnType(SR_DATETIME); builder.dataType(SR_DATETIME); break; + case TIMESTAMP_TZ: + // StarRocks DATETIME does not store timezone info; + // TIMESTAMP_TZ (LTZ) is mapped to DATETIME with potential timezone loss. + builder.columnType(SR_DATETIME); + builder.dataType(SR_DATETIME); + break; case MAP: reconvertMap(column, builder); break; @@ -550,6 +556,13 @@ private void reconvertBuildArrayInternal( builder.columnType(SR_DATETIME_ARRAY); builder.dataType(SR_DATETIME_ARRAY); break; + case TIMESTAMP_TZ: + // StarRocks DATETIME does not store timezone info; + // TIMESTAMP_TZ (LTZ) array is mapped to DATETIME array with potential timezone + // loss. + builder.columnType(SR_DATETIME_ARRAY); + builder.dataType(SR_DATETIME_ARRAY); + break; default: throw CommonError.convertToConnectorTypeError( identifier(), elementType.getSqlType().name(), columnName); diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksBaseSerializer.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksBaseSerializer.java index 04b5b4ec218b..871c01f9eda7 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksBaseSerializer.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksBaseSerializer.java @@ -64,6 +64,10 @@ protected Object convert(SeaTunnelDataType dataType, Object val) { return TimeUtils.toString((LocalTime) val, timeFormatter); case TIMESTAMP: return ((LocalDateTime) val).format(dateTimeFormatter); + case TIMESTAMP_TZ: + // StarRocks DATETIME does not store timezone info; + // convert OffsetDateTime to local wall-clock string. + return ((java.time.OffsetDateTime) val).toLocalDateTime().format(dateTimeFormatter); case ARRAY: case MAP: return JsonUtils.toJsonString(val); diff --git a/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/catalog/StarRocksTypeConverterTest.java b/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/catalog/StarRocksTypeConverterTest.java index 8bd4ca5f21bc..53bafec36983 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/catalog/StarRocksTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/catalog/StarRocksTypeConverterTest.java @@ -1026,6 +1026,23 @@ public void testReconvertDatetime() { Assertions.assertEquals(SR_DATETIME, typeDefine.getDataType()); } + @Test + public void testReconvertDatetimeTz() { + // OFFSET_DATE_TIME_TYPE (TIMESTAMP_TZ / LTZ) → StarRocks DATETIME + // StarRocks DATETIME does not support timezone, so LTZ maps to DATETIME with potential + // timezone loss. + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.OFFSET_DATE_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = converter.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(SR_DATETIME, typeDefine.getColumnType()); + Assertions.assertEquals(SR_DATETIME, typeDefine.getDataType()); + } + @Test public void testReconvertArray() { Column column = diff --git a/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksJsonSerializerTest.java b/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksJsonSerializerTest.java index 54c4401b8d0d..d85af7743e52 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksJsonSerializerTest.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/test/java/org/apache/seatunnel/connectors/seatunnel/starrocks/serialize/StarRocksJsonSerializerTest.java @@ -29,6 +29,8 @@ import org.junit.jupiter.api.Test; import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoField; @@ -70,4 +72,24 @@ public void serialize() { "{\"id\":1,\"name\":\"Tom\",\"array\":[\"tag1\",\"tag2\"],\"map\":{\"key1\":\"value1\"},\"timestamp\":\"2024-01-25 07:55:45.123\"}", jsonString); } + + @Test + public void serializeTimestampTz() { + // TIMESTAMP_TZ (OffsetDateTime / LTZ) → StarRocks DATETIME string (wall-clock, no tz) + String[] fieldNames = {"id", "ts_tz"}; + SeaTunnelDataType[] fieldTypes = { + BasicType.LONG_TYPE, LocalTimeType.OFFSET_DATE_TIME_TYPE + }; + + SeaTunnelRowType seaTunnelRowType = new SeaTunnelRowType(fieldNames, fieldTypes); + StarRocksJsonSerializer serializer = new StarRocksJsonSerializer(seaTunnelRowType, false); + + // 2026-04-15T04:15:23Z → toLocalDateTime() → "2026-04-15 04:15:23" + OffsetDateTime odt = OffsetDateTime.of(2026, 4, 15, 4, 15, 23, 0, ZoneOffset.UTC); + Object[] fields = {1L, odt}; + SeaTunnelRow row = new SeaTunnelRow(fields); + + String jsonString = serializer.serialize(row); + Assertions.assertEquals("{\"id\":1,\"ts_tz\":\"2026-04-15 04:15:23\"}", jsonString); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml index 11c147432d03..ce344d435a0f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml @@ -105,6 +105,19 @@ test + + org.testcontainers + postgresql + ${testcontainer.version} + test + + + + org.postgresql + postgresql + test + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/JdbcToIcebergTimestampIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/JdbcToIcebergTimestampIT.java new file mode 100644 index 000000000000..fc233ddcefad --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/JdbcToIcebergTimestampIT.java @@ -0,0 +1,275 @@ +/* + * 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.seatunnel.e2e.connector.iceberg; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.util.stream.Stream; + +/** + * E2E test verifying that NTZ (No Time Zone) and LTZ (Local Time Zone) timestamp columns from JDBC + * sources are stored with the correct Iceberg timestamp type: + * + *
    + *
  • NTZ → Iceberg {@code TimestampType.withoutZone()} (e.g. MySQL DATETIME, PG timestamp) + *
  • LTZ → Iceberg {@code TimestampType.withZone()} (e.g. MySQL TIMESTAMP, PG timestamptz) + *
+ * + *

Covers the fix for https://github.com/apache/seatunnel/issues/10685 + */ +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = + "Spark engine does not support TIMESTAMP_TZ (OffsetDateTime) natively; " + + "TIMESTAMP_TZ is serialized as a custom Decimal struct in Spark translation layer, " + + "which is incompatible with standard Sink connectors. " + + "Tested on Zeta and Flink engines only.") +@DisabledOnOs(OS.WINDOWS) +public class JdbcToIcebergTimestampIT extends TestSuiteBase implements TestResource { + + private static final Logger log = LoggerFactory.getLogger(JdbcToIcebergTimestampIT.class); + + // ------------------------------------------------------------------------- + // Catalog directories (inside the SeaTunnel container) + // ------------------------------------------------------------------------- + private static final String MYSQL_CATALOG_DIR = "/tmp/seatunnel_mnt/iceberg/hadoop-ts-mysql/"; + + private static final String PG_CATALOG_DIR = "/tmp/seatunnel_mnt/iceberg/hadoop-ts-pg/"; + + // ------------------------------------------------------------------------- + // MySQL container + // ------------------------------------------------------------------------- + private static final String MYSQL_IMAGE = "mysql:8.0"; + private static final String MYSQL_HOST = "mysql_timestamp_e2e"; + private static final String MYSQL_DATABASE = "ts_test"; + private static final String MYSQL_USER = "root"; + private static final String MYSQL_PASSWORD = "root"; + + private static final MySQLContainer MYSQL_CONTAINER = + new MySQLContainer<>(DockerImageName.parse(MYSQL_IMAGE)) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER) + .withPassword(MYSQL_PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("mysql-timestamp-image"))); + + // ------------------------------------------------------------------------- + // PostgreSQL container + // ------------------------------------------------------------------------- + private static final String PG_IMAGE = "postgres:14-alpine"; + private static final String PG_HOST = "pg_timestamp_e2e"; + private static final String PG_DATABASE = "ts_test"; + private static final String PG_USER = "postgres"; + private static final String PG_PASSWORD = "postgres"; + + private static final PostgreSQLContainer PG_CONTAINER = + new PostgreSQLContainer<>(DockerImageName.parse(PG_IMAGE)) + .withDatabaseName(PG_DATABASE) + .withUsername(PG_USER) + .withPassword(PG_PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(PG_HOST) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("pg-timestamp-image"))); + + // ------------------------------------------------------------------------- + // Driver / plugin JARs downloaded into the SeaTunnel container + // ------------------------------------------------------------------------- + private static final String MYSQL_DRIVER_URL = + "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + + private static final String PG_DRIVER_URL = + "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.3.3/postgresql-42.3.3.jar"; + + private static final String ZSTD_URL = + "https://repo1.maven.org/maven2/com/github/luben/zstd-jni/1.5.5-5/zstd-jni-1.5.5-5.jar"; + + // ------------------------------------------------------------------------- + // Container setup: create Iceberg dirs + download driver JARs + // ------------------------------------------------------------------------- + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + for (String dir : + new String[] { + MYSQL_CATALOG_DIR + "seatunnel_namespace/mysql_ts_sink/data", + MYSQL_CATALOG_DIR + "seatunnel_namespace/mysql_ts_sink/metadata", + PG_CATALOG_DIR + "seatunnel_namespace/pg_ts_sink/data", + PG_CATALOG_DIR + "seatunnel_namespace/pg_ts_sink/metadata", + }) { + container.execInContainer("sh", "-c", "mkdir -p " + dir); + } + container.execInContainer("sh", "-c", "chmod -R 777 /tmp/seatunnel_mnt/iceberg/"); + + // Download Iceberg compression codec + container.execInContainer( + "sh", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Iceberg/lib" + + " && cd /tmp/seatunnel/plugins/Iceberg/lib" + + " && wget -q " + + ZSTD_URL); + + // Download JDBC drivers into the Jdbc plugin directory + container.execInContainer( + "sh", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib" + + " && cd /tmp/seatunnel/plugins/Jdbc/lib" + + " && wget -q " + + MYSQL_DRIVER_URL + + " && wget -q " + + PG_DRIVER_URL); + }; + + // ------------------------------------------------------------------------- + // Lifecycle + // ------------------------------------------------------------------------- + @BeforeAll + @Override + public void startUp() throws Exception { + log.info("Starting MySQL and PostgreSQL containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER, PG_CONTAINER)).join(); + log.info("DB containers started. Initializing test data..."); + initMysqlData(); + initPostgresData(); + log.info("Test data initialised."); + } + + @AfterAll + @Override + public void tearDown() { + if (MYSQL_CONTAINER != null) { + MYSQL_CONTAINER.close(); + } + if (PG_CONTAINER != null) { + PG_CONTAINER.close(); + } + } + + // ------------------------------------------------------------------------- + // Test: MySQL DATETIME (NTZ) → Iceberg withoutZone() + // ------------------------------------------------------------------------- + @TestTemplate + public void testMysqlDatetimeToIcebergNtz(TestContainer container) + throws IOException, InterruptedException { + // Step 1: Run job to write data from MySQL to Iceberg + org.testcontainers.containers.Container.ExecResult result = + container.executeJob("/iceberg/mysql_jdbc_to_iceberg_timestamp.conf"); + Assertions.assertEquals( + 0, result.getExitCode(), "Write job failed:\n" + result.getStderr()); + + // Step 2: Run verification job (Iceberg -> Assert) + // This job verifies that the data in Iceberg matches expected types and values + org.testcontainers.containers.Container.ExecResult verifyResult = + container.executeJob("/iceberg/mysql_iceberg_to_assert.conf"); + Assertions.assertEquals( + 0, + verifyResult.getExitCode(), + "Verification job failed:\n" + verifyResult.getStderr()); + } + + // ------------------------------------------------------------------------- + // Test: PostgreSQL timestamp (NTZ) → Iceberg withoutZone() + // PostgreSQL timestamptz (LTZ) → Iceberg withZone() + // ------------------------------------------------------------------------- + @TestTemplate + public void testPgTimestampToIceberg(TestContainer container) + throws IOException, InterruptedException { + // Step 1: Run job to write data from PostgreSQL to Iceberg + org.testcontainers.containers.Container.ExecResult result = + container.executeJob("/iceberg/pg_jdbc_to_iceberg_timestamp.conf"); + Assertions.assertEquals( + 0, result.getExitCode(), "Write job failed:\n" + result.getStderr()); + + // Step 2: Run verification job (Iceberg -> Assert) + org.testcontainers.containers.Container.ExecResult verifyResult = + container.executeJob("/iceberg/pg_iceberg_to_assert.conf"); + Assertions.assertEquals( + 0, + verifyResult.getExitCode(), + "Verification job failed:\n" + verifyResult.getStderr()); + } + + // ------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------- + private void initMysqlData() throws Exception { + try (Connection conn = + DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), MYSQL_USER, MYSQL_PASSWORD); + Statement stmt = conn.createStatement()) { + stmt.execute( + "CREATE TABLE IF NOT EXISTS ts_table (" + + " id INT PRIMARY KEY," + + " dt_col DATETIME," + + " ts_col TIMESTAMP" + + ")"); + stmt.execute( + "INSERT INTO ts_table (id, dt_col, ts_col) VALUES" + + " (1, '2026-01-01 00:00:00', '2026-01-01 00:00:00')"); + } + } + + private void initPostgresData() throws Exception { + try (Connection conn = + DriverManager.getConnection( + PG_CONTAINER.getJdbcUrl(), PG_USER, PG_PASSWORD); + Statement stmt = conn.createStatement()) { + stmt.execute( + "CREATE TABLE IF NOT EXISTS ts_table (" + + " id INT PRIMARY KEY," + + " ts_col TIMESTAMP WITHOUT TIME ZONE," + + " tstz_col TIMESTAMP WITH TIME ZONE" + + ")"); + stmt.execute( + "INSERT INTO ts_table (id, ts_col, tstz_col) VALUES" + + " (1, '2026-01-01 00:00:00', '2026-01-01 00:00:00+00')"); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf index fcec73e5d01e..3e42c56f69de 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -34,7 +34,7 @@ source { f5 = "float" f6 = "double" f7 = "date" - f9 = "timestamp" + f9 = "timestamp_tz" f10 = "timestamp" f11 = "string" f12 = "bytes" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_iceberg_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_iceberg_to_assert.conf new file mode 100644 index 000000000000..3ba2afb306fa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_iceberg_to_assert.conf @@ -0,0 +1,60 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Iceberg { + catalog_name = "seatunnel_test" + iceberg.catalog.config = { + "type" = "hadoop" + "warehouse" = "file:///tmp/seatunnel_mnt/iceberg/hadoop-ts-mysql/" + } + namespace = "seatunnel_namespace" + table = "mysql_ts_sink" + } +} + +sink { + Assert { + rules = { + row_rules = [ + { rule_type = MIN_ROW, rule_value = 1 }, + { rule_type = MAX_ROW, rule_value = 1 } + ] + field_rules = [ + { + field_name = dt_col + field_type = timestamp + field_value = [ + { rule_type = NOT_NULL, equals_to = "2026-01-01T00:00:00" } + ] + }, + { + field_name = ts_col + field_type = timestamp_tz + field_value = [ + { rule_type = NOT_NULL, equals_to = "2026-01-01T00:00:00Z" } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_jdbc_to_iceberg_timestamp.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_jdbc_to_iceberg_timestamp.conf new file mode 100644 index 000000000000..545a54e0e74b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_jdbc_to_iceberg_timestamp.conf @@ -0,0 +1,45 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:mysql://mysql_timestamp_e2e:3306/ts_test?useSSL=false&serverTimezone=UTC" + driver = "com.mysql.cj.jdbc.Driver" + user = "root" + password = "root" + query = "SELECT id, dt_col, ts_col FROM ts_table" + plugin_output = "jdbc_source" + } +} + +sink { + Iceberg { + catalog_name = "seatunnel_test" + iceberg.catalog.config = { + "type" = "hadoop" + "warehouse" = "file:///tmp/seatunnel_mnt/iceberg/hadoop-ts-mysql/" + } + namespace = "seatunnel_namespace" + table = "mysql_ts_sink" + plugin_input = "jdbc_source" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_iceberg_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_iceberg_to_assert.conf new file mode 100644 index 000000000000..647dc1da8e96 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_iceberg_to_assert.conf @@ -0,0 +1,60 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Iceberg { + catalog_name = "seatunnel_test" + iceberg.catalog.config = { + "type" = "hadoop" + "warehouse" = "file:///tmp/seatunnel_mnt/iceberg/hadoop-ts-pg/" + } + namespace = "seatunnel_namespace" + table = "pg_ts_sink" + } +} + +sink { + Assert { + rules = { + row_rules = [ + { rule_type = MIN_ROW, rule_value = 1 }, + { rule_type = MAX_ROW, rule_value = 1 } + ] + field_rules = [ + { + field_name = ts_col + field_type = timestamp + field_value = [ + { rule_type = NOT_NULL, equals_to = "2026-01-01T00:00:00" } + ] + }, + { + field_name = tstz_col + field_type = timestamp_tz + field_value = [ + { rule_type = NOT_NULL, equals_to = "2026-01-01T00:00:00Z" } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_jdbc_to_iceberg_timestamp.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_jdbc_to_iceberg_timestamp.conf new file mode 100644 index 000000000000..7640356e601c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/pg_jdbc_to_iceberg_timestamp.conf @@ -0,0 +1,45 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:postgresql://pg_timestamp_e2e:5432/ts_test" + driver = "org.postgresql.Driver" + user = "postgres" + password = "postgres" + query = "SELECT id, ts_col, tstz_col FROM ts_table" + plugin_output = "jdbc_source" + } +} + +sink { + Iceberg { + catalog_name = "seatunnel_test" + iceberg.catalog.config = { + "type" = "hadoop" + "warehouse" = "file:///tmp/seatunnel_mnt/iceberg/hadoop-ts-pg/" + } + namespace = "seatunnel_namespace" + table = "pg_ts_sink" + plugin_input = "jdbc_source" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf index 1430d77e505c..0be4ac65ed53 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -38,7 +38,7 @@ source { f5 = "float" f6 = "double" f7 = "date" - f9 = "timestamp" + f9 = "timestamp_tz" f10 = "timestamp" f11 = "string" f12 = "bytes" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-s3-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-s3-e2e/src/test/resources/iceberg/iceberg_source.conf index 6b50aba96fbe..36386dd25f43 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-s3-e2e/src/test/resources/iceberg/iceberg_source.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-s3-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -38,7 +38,7 @@ source { f5 = "float" f6 = "double" f7 = "date" - f9 = "timestamp" + f9 = "timestamp_tz" f10 = "timestamp" f11 = "string" f12 = "bytes" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index c92c887ca6ef..0f18eb0deea9 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -366,9 +366,25 @@ public void tearDown() throws SQLException { } } + /** + * Hook for subclasses to skip testJdbcDb for specific engine types without overriding + * the @TestTemplate method itself. Overriding a @TestTemplate method in a subclass causes JUnit + * 5 to register and run the test twice (once from the parent, once from the child), leading to + * duplicate data in the sink table and incorrect row-count assertions. + * + * @param container the current test container + * @return true if testJdbcDb should be skipped for this container + */ + protected boolean isDisabledOnContainer(TestContainer container) { + return false; + } + @TestTemplate public void testJdbcDb(TestContainer container) throws IOException, InterruptedException, SQLException { + if (isDisabledOnContainer(container)) { + return; + } List configFiles = jdbcCase.getConfigFile(); for (String configFile : configFiles) { try { @@ -572,7 +588,23 @@ private Object checkData(Object data) throws SQLException, IOException { javaArray[index] = checkData(jdbcArray[index]); } return javaArray; + } else if (data instanceof java.time.OffsetDateTime) { + // Normalize OffsetDateTime to Timestamp for comparison + return java.sql.Timestamp.valueOf(((java.time.OffsetDateTime) data).toLocalDateTime()); } else { + // oracle.sql.TIMESTAMPLTZ / TIMESTAMPTZ objects do not override equals() correctly + // for cross-object comparison. Normalize to byte[] via toBytes() so that + // assertArrayEquals() can compare the raw timestamp bytes directly. + String className = data.getClass().getName(); + if (className.equals("oracle.sql.TIMESTAMPLTZ") + || className.equals("oracle.sql.TIMESTAMPTZ")) { + try { + java.lang.reflect.Method toBytes = data.getClass().getMethod("toBytes"); + return toBytes.invoke(data); + } catch (Exception e) { + return data; + } + } return data; } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java index 01bcf2db2f07..c300b3dd2c23 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java @@ -503,6 +503,23 @@ private List> querySink(String sql) { Object object = resultSet.getObject(i); if (object instanceof NClob) { objects.add(readNClobAsString((NClob) object)); + } else if (object instanceof java.time.OffsetDateTime) { + // TIMESTAMP_TZ (OffsetDateTime) → normalize to Timestamp for comparison + // with MySQL source which returns java.sql.Timestamp + objects.add( + java.sql.Timestamp.valueOf( + ((java.time.OffsetDateTime) object).toLocalDateTime())); + } else if (object != null + && object.getClass().getName().equals("microsoft.sql.DateTimeOffset")) { + // SQL Server DATETIMEOFFSET → normalize to Timestamp for comparison + // microsoft.sql.DateTimeOffset.getTimestamp() returns java.sql.Timestamp + try { + java.lang.reflect.Method getTimestamp = + object.getClass().getMethod("getTimestamp"); + objects.add(getTimestamp.invoke(object)); + } catch (Exception e) { + objects.add(object); + } } else { objects.add(object); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlTimestampIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlTimestampIT.java new file mode 100644 index 000000000000..7aaba3b8c1da --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlTimestampIT.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.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.given; + +/** + * E2E test verifying that MySQL NTZ/LTZ timestamp types are correctly distinguished by the JDBC + * connector after the fix for https://github.com/apache/seatunnel/issues/10685. + * + *

    + *
  • MySQL {@code DATETIME} (NTZ) → SeaTunnel internal {@code TIMESTAMP} type + *
  • MySQL {@code TIMESTAMP} (LTZ) → SeaTunnel internal {@code TIMESTAMP_TZ} type + *
+ * + *

The Assert sink's {@code field_type} check is used to validate the internal type mapping. + */ +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = + "Spark engine does not support TIMESTAMP_TZ (OffsetDateTime) natively; " + + "TIMESTAMP_TZ is serialized as a custom Decimal struct in Spark translation layer, " + + "which is incompatible with standard Sink connectors. " + + "Tested on Zeta and Flink engines only.") +@Slf4j +public class JdbcMysqlTimestampIT extends TestSuiteBase implements TestResource { + + private static final String MYSQL_IMAGE = "mysql:8.0"; + private static final String MYSQL_HOST = "mysql_ts_e2e"; + private static final String MYSQL_DATABASE = "ts_test"; + private static final String MYSQL_USER = "root"; + private static final String MYSQL_PASSWORD = "Abc!@#135_seatunnel"; + + private static final String MYSQL_DRIVER_URL = + "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + + private MySQLContainer mysqlContainer; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult result = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib" + + " && cd /tmp/seatunnel/plugins/Jdbc/lib" + + " && wget -q " + + MYSQL_DRIVER_URL); + Assertions.assertEquals( + 0, + result.getExitCode(), + "Failed to download MySQL driver: " + result.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + mysqlContainer = + new MySQLContainer<>(DockerImageName.parse(MYSQL_IMAGE)) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER) + .withPassword(MYSQL_PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MYSQL_IMAGE))); + + Startables.deepStart(Stream.of(mysqlContainer)).join(); + + given().ignoreExceptions() + .await() + .atMost(60, TimeUnit.SECONDS) + .untilAsserted(() -> initMysqlData()); + log.info("MySQL container started and test data initialised."); + } + + @AfterAll + @Override + public void tearDown() { + if (mysqlContainer != null) { + mysqlContainer.close(); + } + } + + /** + * Verifies that MySQL {@code DATETIME} (NTZ) columns are read as SeaTunnel {@code TIMESTAMP} + * (i.e. {@code LOCAL_DATE_TIME_TYPE}), not {@code TIMESTAMP_TZ}. + * + *

The Assert sink's {@code field_type = timestamp} assertion will fail if the connector + * incorrectly maps {@code DATETIME} to {@code TIMESTAMP_TZ}. + */ + @TestTemplate + public void testMysqlDatetimeIsNtz(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult result = container.executeJob("/jdbc_mysql_datetime_to_assert.conf"); + Assertions.assertEquals( + 0, + result.getExitCode(), + "MySQL DATETIME (NTZ) assertion failed:\n" + result.getStderr()); + } + + /** + * Verifies that MySQL {@code TIMESTAMP} (LTZ) columns are read as SeaTunnel {@code + * TIMESTAMP_TZ} (i.e. {@code OFFSET_DATE_TIME_TYPE}). + * + *

The Assert sink's {@code field_type = timestamp_tz} assertion will fail if the connector + * incorrectly maps {@code TIMESTAMP} to plain {@code TIMESTAMP}. + */ + @TestTemplate + public void testMysqlTimestampIsLtz(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult result = container.executeJob("/jdbc_mysql_timestamp_to_assert.conf"); + Assertions.assertEquals( + 0, + result.getExitCode(), + "MySQL TIMESTAMP (LTZ) assertion failed:\n" + result.getStderr()); + } + + /** + * Core fix scenario: verifies that MySQL {@code TIMESTAMP} (LTZ) preserves the correct UTC + * instant when the JDBC connection uses a non-UTC {@code serverTimezone} (Asia/Seoul, +09:00). + * + *

Before the fix, {@code JdbcFieldTypeUtils.getOffsetDateTime()} applied the JVM default + * timezone during {@code ResultSet} traversal. In a Seoul-timezone session a value stored as + * UTC midnight would be shifted by +09:00 and read back incorrectly. + * + *

After the fix the UTC instant must be preserved regardless of the JDBC session timezone. + * If the fix regresses, the {@code field_type = timestamp_tz} assertion inside the Assert sink + * will fail because the column will be read as a plain {@code TIMESTAMP} (LocalDateTime) + * shifted by the Seoul offset. + */ + @TestTemplate + public void testMysqlTimestampIsLtzInNonUtcSession(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult result = + container.executeJob("/jdbc_mysql_timestamp_non_utc_to_assert.conf"); + Assertions.assertEquals( + 0, + result.getExitCode(), + "MySQL TIMESTAMP (LTZ) assertion failed with non-UTC serverTimezone (Asia/Seoul):\n" + + result.getStderr()); + } + + // ------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------- + + private void initMysqlData() throws Exception { + String jdbcUrl = + String.format( + "jdbc:mysql://%s:%d/%s?useSSL=false&serverTimezone=UTC", + mysqlContainer.getHost(), + mysqlContainer.getFirstMappedPort(), + MYSQL_DATABASE); + try (Connection conn = DriverManager.getConnection(jdbcUrl, MYSQL_USER, MYSQL_PASSWORD); + Statement stmt = conn.createStatement()) { + stmt.execute( + "CREATE TABLE IF NOT EXISTS ts_source (" + + " id INT PRIMARY KEY," + + " dt_col DATETIME," + + " ts_col TIMESTAMP NULL" + + ")"); + // Insert a fixed wall-clock value: 2026-01-01 00:00:00 + // DATETIME stores it as-is (NTZ); TIMESTAMP stores UTC and displays in session TZ. + stmt.execute( + "INSERT INTO ts_source (id, dt_col, ts_col) VALUES" + + " (1, '2026-01-01 00:00:00', '2026-01-01 00:00:00')"); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java index 255bcc9c00de..1f42bf9973e5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.junit.jupiter.api.Assertions; @@ -166,6 +167,16 @@ public void testSampleDataFromColumnSuccess() throws Exception { dialect.sampleDataFromColumn(connection, table, "INTEGER_COL", 1, 1024); } + /** + * Disabled on Spark: TIMESTAMP WITH LOCAL TIME ZONE is now mapped to TIMESTAMP_TZ + * (OffsetDateTime). Spark encodes TIMESTAMP_TZ as DecimalType(18, 5) internally, causing + * byte-level mismatch on Oracle round-trip. See JdbcMysqlTimestampIT for the same limitation. + */ + @Override + protected boolean isDisabledOnContainer(TestContainer container) { + return container.identifier().getEngineType() == EngineType.SPARK; + } + @TestTemplate public void testOracleWithoutDecimalTypeNarrowing(TestContainer container) throws Exception { Container.ExecResult execResult = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_datetime_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_datetime_to_assert.conf new file mode 100644 index 000000000000..cb2d002a3a8f --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_datetime_to_assert.conf @@ -0,0 +1,60 @@ +# +# 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. +# + +# Verifies that MySQL DATETIME (NTZ) is read as SeaTunnel TIMESTAMP (LOCAL_DATE_TIME_TYPE). +# Covers the fix for https://github.com/apache/seatunnel/issues/10685 + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:mysql://mysql_ts_e2e:3306/ts_test?useSSL=false&serverTimezone=UTC" + driver = "com.mysql.cj.jdbc.Driver" + user = "root" + password = "Abc!@#135_seatunnel" + query = "SELECT id, dt_col FROM ts_source" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 1 + } + ] + field_rules = [ + { + # dt_col is MySQL DATETIME → must map to SeaTunnel TIMESTAMP (NTZ) + field_name = dt_col + field_type = timestamp + field_value = [ + { + rule_type = NOT_NULL + equals_to = "2026-01-01T00:00:00" + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_non_utc_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_non_utc_to_assert.conf new file mode 100644 index 000000000000..1e1e85b19f8e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_non_utc_to_assert.conf @@ -0,0 +1,70 @@ +# +# 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. +# + +# Verifies that MySQL TIMESTAMP (LTZ) preserves the correct UTC instant even when the +# JDBC connection is made with a non-UTC serverTimezone (Asia/Seoul, +09:00). +# +# Core fix scenario: before the fix, JdbcFieldTypeUtils.getOffsetDateTime() used the +# JVM default timezone during ResultSet traversal, so a value stored as UTC midnight +# would be read back as 09:00 (+09:00) in a Seoul-timezone environment — a 9-hour shift. +# After the fix, the UTC instant must be preserved regardless of serverTimezone. +# +# Covers the fix for https://github.com/apache/seatunnel/issues/10685 + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + # serverTimezone=Asia/Seoul simulates a non-UTC JDBC session. + # The stored UTC epoch must still be read correctly as TIMESTAMP_TZ. + url = "jdbc:mysql://mysql_ts_e2e:3306/ts_test?useSSL=false&serverTimezone=Asia%2FSeoul" + driver = "com.mysql.cj.jdbc.Driver" + user = "root" + password = "Abc!@#135_seatunnel" + query = "SELECT id, ts_col FROM ts_source" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 1 + } + ] + field_rules = [ + { + # ts_col is MySQL TIMESTAMP → must still map to TIMESTAMP_TZ in non-UTC session. + # If the fix regresses, the field_type check will fail because the value will + # be read as a plain TIMESTAMP (LocalDateTime) shifted by the Seoul offset. + field_name = ts_col + field_type = timestamp_tz + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_to_assert.conf new file mode 100644 index 000000000000..9cdec934bbfd --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_timestamp_to_assert.conf @@ -0,0 +1,59 @@ +# +# 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. +# + +# Verifies that MySQL TIMESTAMP (LTZ) is read as SeaTunnel TIMESTAMP_TZ (OFFSET_DATE_TIME_TYPE). +# Covers the fix for https://github.com/apache/seatunnel/issues/10685 + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:mysql://mysql_ts_e2e:3306/ts_test?useSSL=false&serverTimezone=UTC" + driver = "com.mysql.cj.jdbc.Driver" + user = "root" + password = "Abc!@#135_seatunnel" + query = "SELECT id, ts_col FROM ts_source" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 1 + } + ] + field_rules = [ + { + # ts_col is MySQL TIMESTAMP → must map to SeaTunnel TIMESTAMP_TZ (LTZ) + field_name = ts_col + field_type = timestamp_tz + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresTimestampIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresTimestampIT.java new file mode 100644 index 000000000000..b9f14c4d1dac --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresTimestampIT.java @@ -0,0 +1,188 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.given; + +/** + * E2E test verifying that PostgreSQL NTZ/LTZ timestamp types are correctly distinguished by the + * JDBC connector after the fix for https://github.com/apache/seatunnel/issues/10685. + * + *

    + *
  • PostgreSQL {@code TIMESTAMP WITHOUT TIME ZONE} (NTZ) → SeaTunnel internal {@code TIMESTAMP} + *
  • PostgreSQL {@code TIMESTAMP WITH TIME ZONE} (LTZ) → SeaTunnel internal {@code TIMESTAMP_TZ} + *
+ * + *

The Assert sink's {@code field_type} check is used to validate the internal type mapping. + */ +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = + "Spark engine does not support TIMESTAMP_TZ (OffsetDateTime) natively; " + + "TIMESTAMP_TZ is serialized as a custom Decimal struct in Spark translation layer, " + + "which is incompatible with standard Sink connectors. " + + "Tested on Zeta and Flink engines only.") +@Slf4j +public class JdbcPostgresTimestampIT extends TestSuiteBase implements TestResource { + + private static final String PG_IMAGE = "postgres:14-alpine"; + private static final String PG_HOST = "pg_ts_e2e"; + private static final String PG_DATABASE = "ts_test"; + private static final String PG_USER = "postgres"; + private static final String PG_PASSWORD = "postgres"; + + private static final String PG_DRIVER_URL = + "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.3.3/postgresql-42.3.3.jar"; + + private PostgreSQLContainer pgContainer; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult result = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib" + + " && cd /tmp/seatunnel/plugins/Jdbc/lib" + + " && wget -q " + + PG_DRIVER_URL); + Assertions.assertEquals( + 0, + result.getExitCode(), + "Failed to download PostgreSQL driver: " + result.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + pgContainer = + new PostgreSQLContainer<>(DockerImageName.parse(PG_IMAGE)) + .withDatabaseName(PG_DATABASE) + .withUsername(PG_USER) + .withPassword(PG_PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(PG_HOST) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(PG_IMAGE))); + + Startables.deepStart(Stream.of(pgContainer)).join(); + + given().ignoreExceptions() + .await() + .atMost(60, TimeUnit.SECONDS) + .untilAsserted(() -> initPgData()); + log.info("PostgreSQL container started and test data initialised."); + } + + @AfterAll + @Override + public void tearDown() { + if (pgContainer != null) { + pgContainer.close(); + } + } + + /** + * Verifies that PostgreSQL {@code TIMESTAMP WITHOUT TIME ZONE} (NTZ) columns are read as + * SeaTunnel {@code TIMESTAMP} (i.e. {@code LOCAL_DATE_TIME_TYPE}). + * + *

The Assert sink's {@code field_type = timestamp} assertion will fail if the connector + * incorrectly maps plain {@code TIMESTAMP} to {@code TIMESTAMP_TZ}. + */ + @TestTemplate + public void testPgTimestampIsNtz(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult result = container.executeJob("/jdbc_pg_timestamp_to_assert.conf"); + Assertions.assertEquals( + 0, + result.getExitCode(), + "PostgreSQL TIMESTAMP (NTZ) assertion failed:\n" + result.getStderr()); + } + + /** + * Verifies that PostgreSQL {@code TIMESTAMP WITH TIME ZONE} (LTZ) columns are read as SeaTunnel + * {@code TIMESTAMP_TZ} (i.e. {@code OFFSET_DATE_TIME_TYPE}). + * + *

The Assert sink's {@code field_type = timestamp_tz} assertion will fail if the connector + * incorrectly maps {@code TIMESTAMPTZ} to plain {@code TIMESTAMP}. + */ + @TestTemplate + public void testPgTimestamptzIsLtz(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult result = container.executeJob("/jdbc_pg_timestamptz_to_assert.conf"); + Assertions.assertEquals( + 0, + result.getExitCode(), + "PostgreSQL TIMESTAMPTZ (LTZ) assertion failed:\n" + result.getStderr()); + } + + // ------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------- + + private void initPgData() throws Exception { + String jdbcUrl = + String.format( + "jdbc:postgresql://%s:%d/%s", + pgContainer.getHost(), pgContainer.getFirstMappedPort(), PG_DATABASE); + try (Connection conn = DriverManager.getConnection(jdbcUrl, PG_USER, PG_PASSWORD); + Statement stmt = conn.createStatement()) { + stmt.execute( + "CREATE TABLE IF NOT EXISTS ts_source (" + + " id INT PRIMARY KEY," + + " ts_col TIMESTAMP WITHOUT TIME ZONE," + + " tstz_col TIMESTAMP WITH TIME ZONE" + + ")"); + // ts_col: wall-clock value stored as-is (NTZ, no timezone conversion) + // tstz_col: value with explicit UTC offset stored in UTC internally (LTZ) + stmt.execute( + "INSERT INTO ts_source (id, ts_col, tstz_col) VALUES" + + " (1, '2026-01-01 00:00:00', '2026-01-01 00:00:00+00')"); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamp_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamp_to_assert.conf new file mode 100644 index 000000000000..f3de0481c7a0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamp_to_assert.conf @@ -0,0 +1,60 @@ +# +# 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. +# + +# Verifies that PostgreSQL TIMESTAMP WITHOUT TIME ZONE (NTZ) is read as SeaTunnel TIMESTAMP. +# Covers the fix for https://github.com/apache/seatunnel/issues/10685 + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:postgresql://pg_ts_e2e:5432/ts_test?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "postgres" + password = "postgres" + query = "SELECT id, ts_col FROM ts_source" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 1 + } + ] + field_rules = [ + { + # ts_col is TIMESTAMP WITHOUT TIME ZONE → must map to SeaTunnel TIMESTAMP (NTZ) + field_name = ts_col + field_type = timestamp + field_value = [ + { + rule_type = NOT_NULL + equals_to = "2026-01-01T00:00:00" + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamptz_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamptz_to_assert.conf new file mode 100644 index 000000000000..802c798cc7aa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_pg_timestamptz_to_assert.conf @@ -0,0 +1,59 @@ +# +# 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. +# + +# Verifies that PostgreSQL TIMESTAMP WITH TIME ZONE (LTZ) is read as SeaTunnel TIMESTAMP_TZ. +# Covers the fix for https://github.com/apache/seatunnel/issues/10685 + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:postgresql://pg_ts_e2e:5432/ts_test?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "postgres" + password = "postgres" + query = "SELECT id, tstz_col FROM ts_source" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 1 + } + ] + field_rules = [ + { + # tstz_col is TIMESTAMP WITH TIME ZONE → must map to SeaTunnel TIMESTAMP_TZ (LTZ) + field_name = tstz_col + field_type = timestamp_tz + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksSchemaChangeIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksSchemaChangeIT.java index f5b7522499b9..8d5d8a927f69 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksSchemaChangeIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksSchemaChangeIT.java @@ -54,6 +54,7 @@ import java.sql.Statement; import java.sql.Timestamp; import java.time.LocalDateTime; +import java.time.OffsetDateTime; import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.List; @@ -443,17 +444,21 @@ private List> query(String sql, Connection connection) { while (resultSet.next()) { ArrayList objects = new ArrayList<>(); for (int i = 1; i <= columnCount; i++) { - if (resultSet.getObject(i) instanceof Timestamp) { - Timestamp timestamp = resultSet.getTimestamp(i); - objects.add(timestamp.toLocalDateTime().format(DATE_TIME_FORMATTER)); - break; + Object obj = resultSet.getObject(i); + if (obj instanceof Timestamp) { + objects.add( + ((Timestamp) obj).toLocalDateTime().format(DATE_TIME_FORMATTER)); + } else if (obj instanceof LocalDateTime) { + objects.add(((LocalDateTime) obj).format(DATE_TIME_FORMATTER)); + } else if (obj instanceof OffsetDateTime) { + // TIMESTAMP_TZ (LTZ) → normalize to wall-clock string for comparison + objects.add( + ((OffsetDateTime) obj) + .toLocalDateTime() + .format(DATE_TIME_FORMATTER)); + } else { + objects.add(obj); } - if (resultSet.getObject(i) instanceof LocalDateTime) { - LocalDateTime localDateTime = resultSet.getObject(i, LocalDateTime.class); - objects.add(localDateTime.format(DATE_TIME_FORMATTER)); - break; - } - objects.add(resultSet.getObject(i)); } log.debug(String.format("Print query, sql: %s, data: %s", sql, objects)); result.add(objects); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonSerializationSchema.java index 749295a50349..5de7c116e4c3 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonSerializationSchema.java @@ -67,6 +67,14 @@ public JsonSerializationSchema(SeaTunnelRowType rowType, String nullValue) { this.charset = StandardCharsets.UTF_8; } + public JsonSerializationSchema(SeaTunnelRowType rowType, boolean serializeTimestampTzAsLocal) { + this.rowType = rowType; + this.runtimeConverter = + new RowToJsonConverters(serializeTimestampTzAsLocal) + .createConverter(checkNotNull(rowType)); + this.charset = StandardCharsets.UTF_8; + } + { mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java index 13a30442d172..5aaf0a49950d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java @@ -53,6 +53,16 @@ public class RowToJsonConverters implements Serializable { private String nullValue; + private final boolean serializeTimestampTzAsLocal; + + public RowToJsonConverters() { + this.serializeTimestampTzAsLocal = false; + } + + public RowToJsonConverters(boolean serializeTimestampTzAsLocal) { + this.serializeTimestampTzAsLocal = serializeTimestampTzAsLocal; + } + public RowToJsonConverter createConverter(SeaTunnelDataType type) { return wrapIntoNullableConverter(createNotNullConverter(type)); } @@ -186,6 +196,17 @@ public JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value) { } }; case TIMESTAMP_TZ: + if (serializeTimestampTzAsLocal) { + return new RowToJsonConverter() { + @Override + public JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value) { + return mapper.getNodeFactory() + .textNode( + ISO_LOCAL_DATE_TIME.format( + ((OffsetDateTime) value).toLocalDateTime())); + } + }; + } return new RowToJsonConverter() { @Override public JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value) { diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextSerializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextSerializationSchema.java index 08f7bd7eab67..61a137ef261a 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextSerializationSchema.java @@ -175,6 +175,12 @@ private String convert(Object field, SeaTunnelDataType fieldType, int level) return TimeUtils.toString((LocalTime) field, timeFormatter); case TIMESTAMP: return DateTimeUtils.toString((LocalDateTime) field, dateTimeFormatter); + case TIMESTAMP_TZ: + return DateTimeUtils.toString( + ((java.time.OffsetDateTime) field) + .withOffsetSameInstant(java.time.ZoneOffset.UTC) + .toLocalDateTime(), + dateTimeFormatter); case NULL: return ""; case BYTES: