[ 
https://issues.apache.org/jira/browse/FLINK-39492?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18082225#comment-18082225
 ] 

Thorne commented on FLINK-39492:
--------------------------------

Starrocks version 3.3.5 supports millisecond precision for datetime types. I 
will fix it shortly

> [Bug][StarRocks Connector] TIMESTAMP/TIMESTAMP_LTZ precision (microseconds) 
> lost when sinking to StarRocks
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-39492
>                 URL: https://issues.apache.org/jira/browse/FLINK-39492
>             Project: Flink
>          Issue Type: Bug
>          Components: Flink CDC
>    Affects Versions: cdc-3.6.0
>            Reporter: 王玉恒
>            Priority: Major
>
> Describe the bug
> When using Flink CDC Pipeline mode to sync MySQL TIMESTAMP(6) to StarRocks, 
> the microsecond precision is lost. StarRocks receives only second-level 
> precision
> (e.g. 2026-03-27 15:20:29 instead of 2026-03-27 15:20:29.921550).
> Root Cause
> In StarRocksUtils.java, the DATETIME_FORMATTER is hardcoded as "yyyy-MM-dd 
> HH:mm:ss" without fractional seconds:
> java
> private static final DateTimeFormatter DATETIME_FORMATTER =
> DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");
> Both TIMESTAMP_WITHOUT_TIME_ZONE and TIMESTAMP_WITH_LOCAL_TIME_ZONE use this 
> formatter, ignoring the precision parameter.
> In contrast, TIME type correctly handles precision via timeFormatter(int 
> precision) with appendFraction(ChronoField.NANO_OF_SECOND, ...).
> Expected behavior
> The DATETIME formatter should respect the precision of TIMESTAMP types, 
> similar to how TIME is handled. For example:
>  * TIMESTAMP(0) → "2026-03-27 15:20:29"
>  * TIMESTAMP(6) → "2026-03-27 15:20:29.921550"
> Suggested fix
> Add a datetimeFormatter(int precision) method similar to timeFormatter:
> java
> private static DateTimeFormatter datetimeFormatter(int precision) {
> if (precision <= 0) {
> return DATETIME_FORMATTER;
> }
> return new DateTimeFormatterBuilder()
> .appendPattern("yyyy-MM-dd HH:mm:ss")
> .appendFraction(ChronoField.NANO_OF_SECOND, precision, precision, true)
> .toFormatter();
> }
> Environment
>  * Flink CDC: 3.6.0
>  * Flink: 1.20.1
>  * StarRocks: 3.4.8
>  * MySQL source field: TIMESTAMP(6)
>  * Pipeline mode with schema.change.behavior: evolve
> Source code reference
> [https://github.com/apache/flink-cdc/blob/master/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/]
> apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to