[
https://issues.apache.org/jira/browse/FLINK-16938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17335657#comment-17335657
]
Jark Wu commented on FLINK-16938:
---------------------------------
[~liuyufei], currently, we treat timestamp of {{StreamRecord}} as in UTC time
zone (this is the semntic in DataStream), so the timestamp is convered into
{{TIMESTAMP WITHOUT TIME ZONE}}. In your case, you should make sure the
assigned timestamp of your DataStream is shift to UTC time zone.
If you want to keep the long value unchanged when converting into Table, I
guess what you are looking for is something like {{.rowtime_ltz()}} method
which converts DataStream timestamp into {{TIMESTAMP WITH LOCAL TIME ZONE}}
type.
> SqlTimestamp has lag when convert long to Timestamp
> ---------------------------------------------------
>
> Key: FLINK-16938
> URL: https://issues.apache.org/jira/browse/FLINK-16938
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / API
> Affects Versions: 1.10.0
> Reporter: YufeiLiu
> Priority: Major
> Labels: stale-major
>
> When I set rowtime attribute by using expression 'column.rowtime' , and
> result type is sql.Timestamp, the result will have lag which is equals with
> default timezone offset.
> {code:java}
> tEnv.fromDataStream(stream, "user_action_time.rowtime, user_name, data");
> {code}
> I look into the conversion logic, the field was go through 'long ->
> SqlTimestamp -> Timestamp' conversion.
> {code:java}
> long from = System.currentTimeMillis();
> long to = SqlTimestamp
> .fromEpochMillis(from)
> .toTimestamp()
> .getTime();
> {code}
> The result is {{from!=to}}. In {{SqlTimestamp.toTimestamp()}} using
> {{Timestamp.valueOf(LocalDateTime dateTime)}} which is contain timezone
> infomation, will casue time lag.
> From Timestamp to Timestamp not have this issue, but convert Datastream to
> Table is use StreamRecord.timestamp as rowtime field.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)