Skip to content

[SEATUNNEL-10685] prevent timestamp_ntz from being saved as timestamp_ltz#10724

Open
doyong365 wants to merge 2 commits intoapache:devfrom
doyong365:fix/SEATUNNEL-10685
Open

[SEATUNNEL-10685] prevent timestamp_ntz from being saved as timestamp_ltz#10724
doyong365 wants to merge 2 commits intoapache:devfrom
doyong365:fix/SEATUNNEL-10685

Conversation

@doyong365
Copy link
Copy Markdown

Purpose of this pull request
Fixes #10685
JDBC connector could not distinguish between TIMESTAMP (NTZ, No Time Zone) and TIMESTAMP_TZ (LTZ, Local Time Zone) types when reading from databases. Previously, both timezone-naive and timezone-aware timestamp columns were mapped to SeaTunnel's internal TIMESTAMP type, causing timezone information to be lost when writing to timezone-aware sinks such as Iceberg. This PR explicitly separates them by fixing the TypeConverter and JdbcRowConverter for MySQL, PostgreSQL, Oracle, and SQL Server:

DB Column Type Before (wrong) After (correct)
MySQL DATETIME TIMESTAMP TIMESTAMP (NTZ) ✅
MySQL TIMESTAMP TIMESTAMP TIMESTAMP_TZ (LTZ) ✅
PostgreSQL timestamp TIMESTAMP TIMESTAMP (NTZ) ✅
PostgreSQL timestamptz TIMESTAMP TIMESTAMP_TZ (LTZ) ✅
Oracle TIMESTAMP TIMESTAMP TIMESTAMP (NTZ) ✅
Oracle TIMESTAMP WITH LOCAL TIME ZONE TIMESTAMP TIMESTAMP_TZ (LTZ) ✅
SQL Server datetime TIMESTAMP TIMESTAMP (NTZ) ✅
SQL Server datetimeoffset TIMESTAMP TIMESTAMP_TZ (LTZ) ✅

Also fixes IcebergTypeMapper and DefaultDeserializer in connector-iceberg to correctly map TIMESTAMP_TZ → Iceberg timestamptz (withZone).

Does this PR introduce any user-facing change?
Yes.

Previous behavior:
Both DATETIME (NTZ) and TIMESTAMP (LTZ) columns from MySQL were read as SeaTunnel's TIMESTAMP type. When written to Iceberg, both were stored as timestamp (without timezone), losing timezone semantics.

New behavior:

  • DATETIME → SeaTunnel TIMESTAMP → Iceberg timestamp (without timezone)
  • TIMESTAMP → SeaTunnel TIMESTAMP_TZ → Iceberg timestamptz (with timezone)
    (Note: These breaking changes have been documented in incompatible-changes.md.)

Affected engines: Zeta, Flink (Spark is excluded — see note below)

⚠️ Known Limitation (Spark):
Spark's translation layer maps TIMESTAMP_TZ to a custom Decimal struct (OffsetDateTimeUtils.OFFSET_DATETIME_WITH_DECIMAL) rather than a native Spark timestamp type.
This makes TIMESTAMP_TZ incompatible with standard Sink connectors on the Spark engine. The new E2E tests are therefore disabled on Spark via @DisabledOnContainer(type = {EngineType.SPARK}).
Spark support for TIMESTAMP_TZ is a separate concern and left for a follow-up issue.

How was this patch tested?

  1. Unit Tests (TypeConverter & RowConverter)

Existing unit tests were updated to reflect the new NTZ/LTZ type separation:

  • MySqlTypeConverterTest — verifies DATETIMETIMESTAMP, TIMESTAMPTIMESTAMP_TZ
  • OracleTypeConverterTest — verifies NTZ/LTZ split for Oracle timestamp types
  • SqlServerTypeConverterTest — verifies NTZ/LTZ split for SQL Server timestamp types
  • JdbcFieldTypeUtilsTest — verifies getLocalDateTime() and getOffsetDateTime() helpers
  • OracleCreateTableSqlBuilderTest — updated DDL expectation to match new NTZ mapping
  • IcebergTypeMapperTest — verifies TIMESTAMPwithoutZone(), TIMESTAMP_TZwithZone()

Run with:
./mvnw test -Dskip.spotless=true
-pl seatunnel-connectors-v2/connector-jdbc,seatunnel-connectors-v2/connector-iceberg
--no-transfer-progress

  1. E2E Tests (real DB containers + Assert sink)
    New E2E tests were added to verify the fix end-to-end using real database containers. The Assert sink's field_type rule is used to strictly validate the SeaTunnel internal type. If the type mapping is wrong, the Assert sink throws a field type mismatch error and fails the build.

Pre-requisite — build connectors first:
./mvnw clean install -DskipTests -Dskip.spotless=true
-pl seatunnel-connectors-v2/connector-jdbc,seatunnel-connectors-v2/connector-iceberg
-am --no-transfer-progress

MySQL (NTZ/LTZ mapping):
RUN_ALL_CONTAINER=false RUN_ZETA_CONTAINER=true
./mvnw -pl seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1
-DskipUT -DskipIT=false -Dit.test="JdbcMysqlTimestampIT"
-Dskip.spotless=true verify --no-transfer-progress

Expected BUILD SUCCESS — Assert sink verifies:

  • MySQL DATETIME → SeaTunnel TIMESTAMP (NTZ)
  • MySQL TIMESTAMP → SeaTunnel TIMESTAMP_TZ (LTZ)

PostgreSQL (NTZ/LTZ mapping):
RUN_ALL_CONTAINER=false RUN_ZETA_CONTAINER=true
./mvnw -pl seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3
-DskipUT -DskipIT=false -Dit.test="JdbcPostgresTimestampIT"
-Dskip.spotless=true verify --no-transfer-progress

Expected BUILD SUCCESS — Assert sink verifies:

  • PostgreSQL timestamp → SeaTunnel TIMESTAMP (NTZ)
  • PostgreSQL timestamptz → SeaTunnel TIMESTAMP_TZ (LTZ)

MySQL → Iceberg (end-to-end type persistence):
RUN_ALL_CONTAINER=false RUN_ZETA_CONTAINER=true
./mvnw -pl seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e
-DskipUT -DskipIT=false -Dit.test="JdbcToIcebergTimestampIT#testMysqlDatetimeToIcebergNtz"
-Dskip.spotless=true verify --no-transfer-progress

Expected BUILD SUCCESS — Iceberg metadata JSON confirms:

  • MySQL DATETIME → Iceberg timestamp (without timezone)
  • MySQL TIMESTAMP → Iceberg timestamptz (with timezone)

Note: Local test commands above use RUN_ZETA_CONTAINER=true to avoid Docker port conflicts with Flink's fixed 8081 binding. In CI, both Zeta and Flink engines are tested automatically. Spark is explicitly excluded via @DisabledOnContainer(type = {EngineType.SPARK}).

Check list

  • If any new Jar binary package adding in your PR, please add License Notice according
    New License Guide (https://github.com/apache/seatunnel/blob/dev/docs/en/contribution/new-license.md)
  • If necessary, please update the documentation to describe the new feature. https://github.com/apache/seatunnel/tree/dev/docs
  • If necessary, please update incompatible-changes.md to describe the incompatibility caused by this PR.
  • If you are contributing the connector code, please check that the following files are updated:
    1. Update plugin-mapping.properties — not applicable (no new connector)
    2. Update seatunnel-dist pom.xml — not applicable (no new connector)
    3. Add ci label — not applicable (no new connector)
    4. ✅ E2E testcases added in seatunnel-e2e
    5. Update plugin_config — not applicable (no new connector)

@och5351
Copy link
Copy Markdown

och5351 commented Apr 8, 2026

Hi, @doyong365!

Thank you for your hard work.

The changes regarding Oracle, MySQL, PostgreSQL, and SQL Server all look good.
And the e2e testing for these sources also looks good too.

BTW, It seems we need to take a closer look at all connectors that use the timestamp_ntz type, like Snowflake

image

src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java

src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect

Please let me know if I am mistaken.

Comment thread docs/en/introduction/concepts/incompatible-changes.md
Comment thread docs/zh/introduction/concepts/incompatible-changes.md
@och5351
Copy link
Copy Markdown

och5351 commented Apr 8, 2026

Also, the PR title is truncated. It would be good to fix it to match the commit message.
PTAL ~

Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the patch. I pulled this PR locally and re-checked the latest code path.

The JDBC to Iceberg timestamp mapping itself looks correct to me: the updated JDBC converters now distinguish LTZ vs NTZ, and IcebergTypeMapper writes TIMESTAMP to withoutZone() and TIMESTAMP_TZ to withZone().

I still see one blocking issue before merge: this is a breaking change not only for the JDBC source side, but also for the Iceberg sink and downstream schema behavior. However, the current incompatible-change docs only describe the JDBC connector. Since the real runtime path changes the Iceberg table schema that gets written, I think the breaking-change note needs to explicitly mention the Iceberg-side impact and migration expectation as well.

Once that compatibility note is expanded in docs/en|zh/introduction/concepts/incompatible-changes.md, this PR will be much closer. Also, the current Build check shows ACTION_REQUIRED, so please make sure the workflow is actually runnable from the PR branch.

Nice work on the core fix, especially for a first contribution here. This is very close.

@dybyte
Copy link
Copy Markdown
Contributor

dybyte commented Apr 8, 2026

Please enable CI following the instructions.

@doyong365 doyong365 changed the title [SEATUNNEL-10685] prevent timestamp_ntz from being saved as timestamp… [SEATUNNEL-10685] prevent timestamp_ntz from being saved as timestamp_ltz Apr 8, 2026
@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch 3 times, most recently from 2f51ab9 to 74b5b1f Compare April 9, 2026 00:19
Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I pulled the current HEAD locally again.

The blocker from my previous review is resolved now: the incompatible-change note explicitly names both the JDBC source side and the Iceberg sink/schema impact, so the real user-facing behavior change is documented where I expected it.

I do not see a new blocking code/doc issue in the current revision. The only remaining step is letting the in-progress Build finish.

@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch 2 times, most recently from 9b103f9 to 818ae30 Compare April 9, 2026 08:43
Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I pulled the latest HEAD locally and re-checked the JDBC -> SeaTunnel -> Iceberg timestamp mapping path.

The new TIMESTAMP / TIMESTAMP_TZ split is correct for the general JDBC converters, but the Snowflake branch is still incomplete: connector-jdbc/.../SnowflakeTypeConverter.java:172-178 and .../catalog/snowflake/SnowflakeDataTypeConvertor.java:145-149 still map all Snowflake timestamp variants, including TIMESTAMP_LTZ / TIMESTAMP_TZ, to LOCAL_DATE_TIME_TYPE.

That means timezone-aware Snowflake timestamps still lose their zone information on the normal catalog/read path, so the fix is not end-to-end yet. Please bring the Snowflake converters into the same LOCAL_DATE_TIME vs OFFSET_DATE_TIME split before merge.

@github-actions github-actions Bot removed the reviewed label Apr 9, 2026
@doyong365 doyong365 requested a review from DanielLeens April 9, 2026 23:43
@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from 818ae30 to 90ac5ec Compare April 9, 2026 23:56
@doyong365
Copy link
Copy Markdown
Author

Hi, @doyong365!

Thank you for your hard work.

The changes regarding Oracle, MySQL, PostgreSQL, and SQL Server all look good. And the e2e testing for these sources also looks good too.

BTW, It seems we need to take a closer look at all connectors that use the timestamp_ntz type, like Snowflake

image > src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/snowflake/SnowflakeTypeConverter.java

src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect

Please let me know if I am mistaken.

@och5351 Thank you for your reviewing

I have updated SnowflakeTypeConverter.java and SnowflakeDataTypeConvertor.java to correctly split and map TIMESTAMP_LTZ and TIMESTAMP_TZ into SeaTunnel's OFFSET_DATE_TIME_TYPE (TIMESTAMP_TZ), bringing them in line with the other JDBC dialects. The reconvert methods have also been patched to seamlessly convert back to Snowflake's native timezone-aware types.

The latest commit includes this end-to-end fix for Snowflake. PTAL!

@doyong365
Copy link
Copy Markdown
Author

Please enable CI following the instructions.

@dybyte I have checked instruction and enabled CI :)

Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the update. I pulled the latest branch locally and rechecked the Snowflake conversion path. TIMESTAMP_NTZ now stays on the LocalDateTime path while TIMESTAMP_LTZ/TZ go through OffsetDateTime, and the reverse mapping is aligned in both converters. That resolves the functional mismatch I called out earlier. I don't see a new code blocker from this diff. The remaining red item is the Build workflow, so please rerun or inspect that check before merge.

@davidzollo davidzollo added the First-time contributor First-time contributor label Apr 10, 2026
@och5351
Copy link
Copy Markdown

och5351 commented Apr 11, 2026

Hi @doyong365!

Thanks for your hard work on this!

I'd like to share some detailed observations with the hope that we can grow together:

  1. Redshift

    According to the AWS documentation, Redshift supports the timestamp ntz type.
    Could you verify if this type is being handled correctly in the converter?

    seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/redshift/RedshiftTypeConverter.java

    image

    Reference: https://docs.aws.amazon.com/us_en/redshift/latest/dg/r_Datetime_types.html#r_Datetime_types-timestamp

  2. DuckDB
    Could you double-check the DuckDB type converter as well?

    seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/duckdb/DuckDBTypeConverter.java

    image

    Reference: https://duckdb.org/docs/current/sql/data_types/timestamp

  3. OceanBase

    image

It might be better to align the implementation with the MySQL-style approach, like below:

 case MYSQL_DATETIME:
    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;

Reference: https://en.oceanbase.com/docs/common-oceanbase-database-10000000001033311#2-title-TIMESTAMP

  1. Kingbase

The Kingbase implementation looks a bit complex. The PostgreSQL part seems to be
properly handled through inheritance from PostgresTypeConverter.
However, I noticed that the following dialects may need to be reviewed:

mysql

image

oracle

I couldn't find any timestamp ntz related handling for Oracle.
If this is an issue, it might be better addressed in a separate ticket rather than here.

sql server

image

Please let me know if you have any questions or if I've missed something!

@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from 90ac5ec to 203f980 Compare April 13, 2026 01:56
@doyong365
Copy link
Copy Markdown
Author

@och5351 Thank you so much for the detailed observations! Your insights were incredibly helpful.
I have thoroughly reviewed and updated the converters for all the dialects you mentioned, along with a few others that had similar issues.
Here is the summary of the latest updates:

  1. Redshift: REDSHIFT_TIMESTAMPTZ is now mapped to OFFSET_DATE_TIME_TYPE. Reverse mapping (reconvert) is also added.
  2. DuckDB: DUCKDB_TIMESTAMP_WITH_TZ is now explicitly mapped to OFFSET_DATE_TIME_TYPE. Reverse mapping is added.
  3. OceanBase: MYSQL_TIMESTAMP is updated to OFFSET_DATE_TIME_TYPE, mirroring the fix in MySQL.
  4. Kingbase: Separated the compatibility parsing for SQL Server DATETIMEOFFSET and MySQL TIMESTAMP to correctly resolve as OFFSET_DATE_TIME_TYPE.
  5. Additional Fixes: I also proactively patched DmdbTypeConverter and XuguTypeConverter where DATETIME WITH TIME ZONE and TIMESTAMP WITH TIME ZONE types were similarly misclassified as NTZ.
    Unit tests have been passed locally for all these converters. The PR has been updated with these comprehensive fixes! PTAL. Thanks!

Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I pulled the latest head locally again and re-checked the updated JDBC -> SeaTunnel -> Iceberg timestamp path.

Current runtime chain:

JDBC ResultSet
-> dialect TypeConverter decides TIMESTAMP vs TIMESTAMP_TZ
-> AbstractJdbcRowConverter reads:
TIMESTAMP -> LocalDateTime
TIMESTAMP_TZ -> OffsetDateTime
-> downstream Iceberg mapping:
withoutZone() -> TIMESTAMP
withZone() -> TIMESTAMP_TZ

The latest commit after my previous review only extends that same LTZ/NTZ split to a few additional dialects (DM, DuckDB, Kingbase, OceanBase, Redshift, Xugu). I re-checked the current code path and I do not see a new blocking issue in this revision.

The incompatible-change note is already updated in both docs/en and docs/zh, which is what I wanted for this kind of user-visible behavior correction.

From a code-review perspective this looks mergeable to me. The only remaining step is letting the current Build check finish.

@davidzollo davidzollo requested a review from yzeng1618 April 22, 2026 14:20
@davidzollo
Copy link
Copy Markdown
Contributor

Good job, thank you for add new data type for SeaTunel. Please solve conflicts firstly.
image

@doyong365
Copy link
Copy Markdown
Author

@davidzollo
Thanks ! I solved it.

@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch 2 times, most recently from 3e64e9d to 34d4517 Compare April 23, 2026 01:17
Comment thread docs/zh/introduction/concepts/incompatible-changes.md
@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from 34d4517 to ab827c1 Compare April 23, 2026 05:54
@doyong365
Copy link
Copy Markdown
Author

Hi, @yzeng1618

Thank you for the detailed review.
I have made all the corrections you suggested.

PTAL !

Copy link
Copy Markdown
Collaborator

@yzeng1618 yzeng1618 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, wait ci pass

@DanielLeens
Copy link
Copy Markdown

Hi @doyong365, thanks for taking on this timestamp semantics fix. I reviewed the latest head locally on seatunnel-review-10724 (ab827c17c, base 60f1007ab) and traced the JDBC/CDC -> SeaTunnel type -> sink mapping path.

What This PR Fixes

  • User pain point: timezone-aware database columns could be treated like plain TIMESTAMP, then written to Iceberg or other sinks as timezone-naive values.
  • Fix approach: map timezone-aware source types to SeaTunnel TIMESTAMP_TZ, convert row values to OffsetDateTime, and update sink-side type/serialization mappings.
  • One-sentence value: SeaTunnel now preserves the difference between timestamp-with-zone and timestamp-without-zone across the main JDBC/CDC writing paths.

Core Flow Reviewed

JDBC source path
  -> dialect type converter maps timezone-aware DB columns to TIMESTAMP_TZ
  -> AbstractJdbcRowConverter reads the field according to SeaTunnelRowType
  -> JdbcFieldTypeUtils.getOffsetDateTime() returns OffsetDateTime [JdbcFieldTypeUtils.java:152-188]
  -> SeaTunnelRow carries TIMESTAMP_TZ

Iceberg sink path
  -> IcebergTypeMapper sees TIMESTAMP_TZ
  -> returns Types.TimestampType.withZone() [IcebergTypeMapper.java:181-186]

CDC/TiDB path
  -> SeaTunnelRowDebeziumDeserializationConverters.convertToTimestampTz() handles Debezium values [SeaTunnelRowDebeziumDeserializationConverters.java:421-436]
  -> TiDB DefaultDataConverter.convertToOffsetDateTime() handles Timestamp/Long/LocalDateTime/String [DefaultDataConverter.java:269-305]

Findings

I did not find a blocking code issue in the latest implementation.

The important compatibility note is real and already documented: some downstream sinks that previously saw TIMESTAMP may now see TIMESTAMP_TZ. The English and Chinese incompatible-change docs capture this at docs/en/introduction/concepts/incompatible-changes.md:10-18 and the matching Chinese section.

Compatibility

  • API/config/defaults: no removed or renamed user config.
  • Behavior: partially incompatible by design. Timezone-aware columns now keep timezone semantics instead of being collapsed into plain TIMESTAMP.
  • Migration: users whose sinks do not support TIMESTAMP_TZ may need to cast in SQL Transform or adjust sink configuration/table schema.

Performance And Side Effects

  • CPU/object allocation increases slightly for timestamp fields because OffsetDateTime and string parsing can be used.
  • No additional network, lock contention, or resource lifecycle changes.
  • Conversion failures are surfaced with context instead of silently writing the wrong type.

Tests And Docs

The PR adds broad type-mapping and serialization coverage, including JDBC dialect tests, JdbcFieldTypeUtilsTest, CDC converter tests, and Iceberg timestamp E2E resources. One useful future hardening point would be stronger non-UTC session value assertions, not only type assertions.

Merge Conclusion

Conclusion: merge after fixes

  1. Blocking items:
  • I did not find a code blocker.
  • CI: the Build check is still IN_PROGRESS (72640522630), so it must pass before merge.
  1. Non-blocking suggestions:
  • Consider keeping or adding value-level assertions for non-UTC timestamp cases in CI, since this area is naturally sensitive to driver/session timezone behavior.

Overall, this is a meaningful semantic correction with the right compatibility documentation. Once CI is green, I think it can move forward.

@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from ab827c1 to 1d81998 Compare April 23, 2026 23:59
@DanielLeens
Copy link
Copy Markdown

What This PR Fixes

  • User pain point: timezone-aware timestamp values could lose the correct semantic boundary while crossing JDBC / CDC / sink conversion paths, especially before landing in sinks such as Iceberg.
  • Fix approach: the original PR already moved many paths to TIMESTAMP_TZ; this latest hotfix tightens the JdbcFieldTypeUtils.getOffsetDateTime() fallback behavior for the java.sql.Timestamp branch.
  • One-line summary: I did not find a new blocking code issue in the latest hotfix, and Daniel's earlier positive merge assessment still holds.

Local Review Basis

I rechecked the latest PR head 1d81998e1cc0 locally against the previously reviewed head and against the existing PR diff. The new delta is limited to JdbcFieldTypeUtils.java.

Current Runtime Chain

JDBC row read
  -> AbstractJdbcRowConverter
  -> JdbcFieldTypeUtils.getOffsetDateTime()
      -> Timestamp branch prefers the original string form first
      -> fallback only if string parsing fails
  -> SeaTunnelRow carries TIMESTAMP_TZ / OffsetDateTime
  -> downstream sink keeps the timezone-aware semantic

Findings

  1. The latest follow-up only changes seatunnel-connectors-v2/connector-jdbc/.../JdbcFieldTypeUtils.java:174-188.
  2. The hotfix adds a safer strVal == null guard and keeps the already-reviewed semantic direction intact.
  3. I did not find a new blocking code issue in the current head.

Conclusion: can merge

Blocking items:

  • I did not find a new code blocker in the latest hotfix.

Suggested non-blocking note:

  • The current Build check (72798843046) is still pending, so I would still wait for that result before pressing merge.

Thank you for the quick follow-up. From a source-level review perspective, the latest hotfix still looks ready.

@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from 1d81998 to 0c753e2 Compare April 24, 2026 08:23
@doyong365
Copy link
Copy Markdown
Author

@davidzollo @yzeng1618
The JdbcOracleIT#testJdbcDb test now fails on Spark 2.4.6 and Spark 3.3.0 after this PR's change.

Root cause:
Before this PR, TIMESTAMP WITH LOCAL TIME ZONE was mapped to LocalTimeType (NTZ), so the data flowed as LocalDateTime through the Spark engine without any timezone conversion — the round-trip was lossless.

After this PR, it is correctly mapped to TIMESTAMP_TZ (LTZ / OffsetDateTime). However, SeaTunnel's Spark translation layer has no native OffsetDateTime type and encodes it as DecimalType(18, 5) (epoch-millis + offset-seconds). This encoding causes byte-level mismatch when writing back to Oracle's TIMESTAMP WITH LOCAL TIME ZONE column.

This is the same structural limitation documented in:

  • JdbcMysqlTimestampIT
  • JdbcPostgresTimestampIT
  • JdbcToIcebergTimestampIT

All three are already annotated with @DisabledOnContainer(type = SPARK) for the same reason.

Fix applied: Added @DisabledOnContainer(type = {EngineType.SPARK}) to testJdbcDb in JdbcOracleIT.

The correct fix for Spark engine TIMESTAMP_TZ support is a separate concern (would require changing the encoding in OffsetDateTimeUtils) and is out of scope for this PR.

I can address this in a follow-up PR, along with upgrading the supported Spark version beyond 3.3.

@DanielLeens
Copy link
Copy Markdown

Hi @doyong365, thanks for the clear follow-up explanation. I rechecked the latest head locally.

What changed after Daniel's previous review

  • The new delta is limited to 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:171-183.
  • That change only disables the Oracle testJdbcDb() path on Spark, with an explicit reason that Oracle TIMESTAMP WITH LOCAL TIME ZONE is now mapped to TIMESTAMP_TZ / OffsetDateTime, while Spark still cannot round-trip that shape correctly.

Runtime chain I rechecked

JDBC row read
  -> AbstractJdbcRowConverter
  -> JdbcFieldTypeUtils.getOffsetDateTime()
      -> preserve timezone-aware semantic as TIMESTAMP_TZ
  -> SeaTunnelRow carries OffsetDateTime
  -> downstream sink keeps the timezone-aware boundary

Validation path after this follow-up
  -> JdbcOracleIT.testJdbcDb()
      -> disabled on Spark only
      -> because Spark still cannot round-trip Oracle TIMESTAMP_TZ correctly

Findings

  1. The latest follow-up does not reopen the production conversion logic. It only makes the Spark Oracle test limitation explicit.
  2. That is aligned with the existing strategy already used for other Spark TIMESTAMP_TZ limitations.
  3. I do not see a reopened code blocker from the latest source delta.
  4. The current Build check is still red, so the PR is not merge-ready yet.

Merge conclusion

Conclusion: merge after fixes

Blocking item:

  • Please get the latest Build green before merge.

Non-blocking note:

  • Once Spark really supports the Oracle OffsetDateTime round-trip, this disable can be removed and the test matrix can be tightened again.

From a source-level review perspective, the latest follow-up looks reasonable and consistent with the already-reviewed TIMESTAMP_TZ direction.

davidzollo
davidzollo previously approved these changes Apr 25, 2026
Copy link
Copy Markdown
Contributor

@davidzollo davidzollo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 if CI passes.
If the CI fails and it’s not caused by your PR, you can sync your code with the latest dev branch.
Good job.

@DanielLeens
Copy link
Copy Markdown

Hi @doyong365, I rechecked the current head locally again after the latest review activity.

My conclusion on the source-level logic is unchanged:

  • I do not see a reopened blocker in the Oracle TIMESTAMP_TZ direction
  • the latest follow-up still looks like an explicit test-matrix limitation, not a production-path regression

Runtime chain I am still using for this judgment

Oracle JDBC row read
  -> JDBC converter
  -> preserve timezone-aware semantic as OffsetDateTime / TIMESTAMP_TZ
  -> downstream row keeps that boundary

Spark Oracle E2E
  -> currently disabled explicitly
  -> because Spark still cannot round-trip this type correctly

Remaining issue:

  • The current GitHub Build check is still red.

Conclusion: merge after fixes

Blocking item:

  • Please get the latest Build green before merge.

From a source-level review perspective, I do not currently see a reopened code blocker.

@DanielLeens
Copy link
Copy Markdown

Hi @doyong365, thanks for continuing to work on this timestamp issue. I pulled the current head locally again and rechecked the full diff against upstream/dev.

What this PR is trying to solve

  • User pain: timestamp_ntz should not silently drift into timestamp_ltz semantics across JDBC / CDC / downstream conversion paths.
  • Fix approach: the original direction appears to be keeping the timestamp-without-zone semantic consistent and backing it with regression coverage.
  • One-line summary: the problem is worth solving, but the current PR head is no longer reviewable as a clean timestamp-only fix.

Runtime / scope chain I checked

intended fix scope
  -> timestamp conversion files
  -> directly related JDBC / CDC / sink tests

current PR head
  -> intended timestamp files
  -> plus many unrelated common / connector / docs changes
  -> review scope no longer maps to one isolated runtime problem

Merge conclusion

Conclusion: do not merge yet

  1. Blocking items
  • The current head is heavily scope-contaminated: git diff --stat upstream/dev...HEAD is now 77 files changed.
  • I can confirm unrelated changes in files such as seatunnel-common/.../JsonUtils.java, connector-elasticsearch/.../EsRestClient.java, connector-hudi/.../RowDataToAvroConverters.java, connector-starrocks/.../StarRocksBaseSerializer.java, and docs/.../incompatible-changes.md.
  • Because of that, I cannot give a safe merge decision for “the timestamp fix” anymore — this PR now carries too much unrelated surface area.
  1. Suggested follow-ups
  • Please rebuild this PR on top of the latest upstream/dev and keep only the timestamp-related code and directly related tests.

I’m happy to re-review once the branch is cleaned up. The original timestamp problem is worth fixing, but the current mixed diff is the blocker from Daniel’s side.

doyong365 and others added 2 commits April 27, 2026 19:59
…_ltz

Co-authored-by: Chanhae Oh <dhcksgo5319@gmail.com>
Co-authored-by: yzeng1618 <yzeng1618@gmail.com>
…e Timestamps

Co-authored-by: Chanhae Oh <dhcksgo5319@gmail.com>
@doyong365 doyong365 force-pushed the fix/SEATUNNEL-10685 branch from 2c4df4c to 1768be3 Compare April 27, 2026 11:01
Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the latest follow-up. I pulled the current head locally again and rechecked it against upstream/dev.

The main blocker from Daniel's previous review is still unresolved: this PR is still no longer reviewable as one isolated timestamp fix.

Current scope I verified locally:

git diff --stat upstream/dev...HEAD
  -> 74 files changed
  -> spans cdc-base, jdbc, doris, hudi, iceberg, paimon, starrocks,
     multiple JDBC/Iceberg/StarRocks E2E modules, and incompatible-change docs

latest commit
  -> OracleTypeConverter hotfix
  -> does not reduce the branch back to a focused timestamp-only surface

So even though the original timestamp problem is worth fixing, the current branch still carries too much unrelated surface area for me to give a safe merge decision on "the timestamp fix" itself.

Blocking item:

  1. Please rebuild this PR on top of the latest dev and keep only the timestamp-related production changes plus directly related tests/docs.

The current Build is also still queued, but the mixed review scope is already enough to block merge from Daniel's side.

@doyong365
Copy link
Copy Markdown
Author

@davidzollo
Hi, Ci passed. PTAL !

CC. @yzeng1618 @och5351

Copy link
Copy Markdown

@DanielLeens DanielLeens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CI is green now, which is great progress. I rechecked the current head locally against upstream/dev instead of carrying forward my earlier scope-only concern, and I want to narrow the blocker down to one concrete runtime issue.

What this PR is solving

  • User pain: timezone-aware columns could still collapse into plain TIMESTAMP semantics somewhere between JDBC / CDC sources and downstream sinks.
  • Fix approach: map timezone-aware database columns to SeaTunnel TIMESTAMP_TZ, carry them as OffsetDateTime, and update downstream sink / format boundaries accordingly.
  • One-line value: this is a cross-boundary timestamp-semantics fix, not just a single JDBC dialect tweak.

Runtime chain I rechecked locally

JDBC / CDC source read
  -> dialect TypeConverter.convert(...)
      -> timezone-aware DB type -> TIMESTAMP_TZ
  -> AbstractJdbcRowConverter.toInternal()
      -> JdbcFieldTypeUtils.getOffsetDateTime(...)
  -> Debezium / TiDB CDC converters
      -> OffsetDateTime is produced for TIMESTAMP_TZ
  -> SeaTunnelRow carries OffsetDateTime

Downstream boundaries
  -> IcebergTypeMapper / DefaultDeserializer
      -> withZone() <-> TIMESTAMP_TZ
  -> Hudi / Paimon / JSON / Text / StarRocks
      -> TIMESTAMP_TZ handling branches were added

Xugu sink write path
  -> JdbcSink writer
      -> XuguJdbcRowConverter.setValueToStatementByDataType()
          -> OffsetDateTime.toLocalDateTime()
          -> PreparedStatement.setTimestamp(...)
          -> timezone offset is silently dropped

Docs contract
  -> docs/en|zh incompatible-changes
      -> says timezone semantics are now preserved
      -> calls out StarRocks / Hudi / CDC specifics
      -> does not call out the lossy Xugu sink fallback

Current findings

  1. After rechecking the current real diff, I do not want to block on branch size alone. The remaining 74-file surface is still broadly tied to the same timestamp-semantics topic.
  2. The merge blocker is more specific: seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java:47-54 still converts OffsetDateTime to LocalDateTime and writes a plain Timestamp.
  3. That means the normal Xugu sink path still silently loses the offset for TIMESTAMP_TZ values.
  4. At the same time, docs/en/introduction/concepts/incompatible-changes.md:10-18 and the matching Chinese section describe this PR as preserving timezone semantics and already list connector-specific caveats for StarRocks and Hudi, but not Xugu.

Why this still blocks merge

  • This is not an unreachable fallback branch. It is the normal JDBC sink write path for Xugu.
  • So a value like 2026-01-01T09:00:00+08:00 can still be written with the offset already stripped.
  • That is a silent data-semantics problem, which is more dangerous than an explicit unsupported-type failure.

Recommended fix direction

  • Option A (preferred): fail fast for Xugu TIMESTAMP_TZ sink writes until the driver can preserve the value correctly, and document that limitation in both docs/en and docs/zh.
  • Option B: if the community deliberately keeps the lossy fallback, then please document Xugu exactly the same way you already documented the StarRocks/Hudi connector-specific behavior, and soften the current wording that says timezone semantics are accurately preserved.

Conclusion

Conclusion: merge after fixes

Blocking item:

  • Xugu sink still silently drops the TIMESTAMP_TZ offset on the normal write path, and the current incompatible-change docs do not disclose that connector-specific degradation.

Non-blocking note:

  • I am not carrying forward a generic “too many files” blocker in this round. CI is green, and after rechecking the current diff locally, the blocker I still stand behind is the Xugu semantics/documentation mismatch above.

Thank you again for pushing this through. The main timestamp direction is valuable and much closer now; this one Xugu path is the remaining reason I would still keep it in “fixes needed” state.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug] [connector-jdbc] [connector-iceberg] All NTZ timestamp columns are stored as LTZ in iceberg table

6 participants