[
https://issues.apache.org/jira/browse/FLINK-7548?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16207628#comment-16207628
]
Fabian Hueske commented on FLINK-7548:
--------------------------------------
Hi [~jark], [~wheat9], [~twalthr], [~xccui], [~ykt836],I'd really like your
feedback on this effort.
As recently discussed on the dev mailing list, the plan is to cut the 1.4.0
release branch in two weeks and I think this change should go in.
In the meantime, I thought about the interface and I'll do some refinements:
- Time indicator attributes will be of type Types.SQL_TIMESTAMP in the
TableSource (instead of TimeIndicatorType). The reason is that 1) they are
exposed as TIMESTAMP in the schema 2) this hides the internal representation
from the user.
- I add a DefinedProctimeAttribute interface that returns the name of a
processing time stamp, i.e., the field must exist in the TableSchema and will
be used as processing time indicator
- The DefinedRowtimeAttributes will be changed to return a list of
RowtimeAttributeDescriptors. The field names of the descriptors are matched to
the TableSchema and these fields must be of type Timestamp.
- The rowtime extraction expression will be a Flink Table API Expression
instead of a Calcite RexNode. Until now we have hidden Calcite interfaces from
the users.
- The BatchTableSourceScan will be adapted to also extract rowtime attributes
and assign the current timestamp to processing time attributes. This way, we
have a unified handling if a TableSource implements BatchTableSource and
StreamTableSource.
Please let me know what you think.
Thanks, Fabian
> 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
> Assignee: Fabian Hueske
> Priority: Blocker
> Fix For: 1.4.0
>
>
> 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)