flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Xingcan Cui (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-7548) Support watermark generation for TableSource
Date Fri, 22 Sep 2017 07:05:00 GMT

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

Xingcan Cui commented on FLINK-7548:
------------------------------------

Hi [~fhueske] and [~jark], sorry for the late reply.

Honestly, I don't have much experience in real application, but I think the proposal is quite
reasonable. In brief, any (existing or computed) field with the type {{LONG}} or {{TIMESTAMP}}
can be taken as a rowtime field, right? Here are my thoughts:

1. Considering that the data type should be preserved, it may bring extra logic if we do that
internally. To keep the consistency, I wonder if it's possible to encapsulate the time into
a new {{Rowtime<T>}} type. It exposes two methods, {{getTime(): Long}} for logical level
use and {{getValue(): T}} for physical level use.

2. Besides, I think the watermark generation should not be bound with rowtime extraction.
Compared with implementing them in a single scan operator (not sure if I understood correctly),
I prefer to generate watermarks in extra operators. That should be more flexible.

3. I am thinking of a new record number bounded out-of-order generation strategy. Do you think
it will be useful in real applications?

4. I still feel that the machine time is not compatible with the rowtime watermark generation.
Shall we consider getting rid of it?

> Support watermark generation for TableSource
> --------------------------------------------
>
>                 Key: FLINK-7548
>                 URL: https://issues.apache.org/jira/browse/FLINK-7548
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API & SQL
>            Reporter: Jark Wu
>
> As discussed in FLINK-7446, currently the TableSource only support to define rowtime
field, but not support to extract watermarks from the rowtime field. We can provide a new
interface called {{DefinedWatermark}}, which has two methods {{getRowtimeAttribute}} (can
only be an existing field) and {{getWatermarkGenerator}}. The {{DefinedRowtimeAttribute}}
will be marked deprecated.
> How to support periodic and punctuated watermarks and support some built-in strategies
needs further discussion.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message