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

Xingcan Cui commented on FLINK-6233:
------------------------------------

Hi [~fhueske],

thanks for sharing. I totally agree to your first two ideas on the timestamp. 
Regarding to the last one on watermark, I have some questions.
# What happens if we just provide the needed timestamps and do nothing to the 
original watermarks? Some results triggered by the watermarks (which means they 
are generated AFTER the corresponding watermarks) would be treated as delayed 
and thus may not be correctly processed by the downstream functions, right?
# When said "holding back", did you mean the original/old watermarks should be 
swallowed by some special operators so that the downstream components will 
never receive them?
# If the downstream operators are time-based, users have to assign new 
timestamp fields from the projected fields and thus new watermarks should also 
be automatically generated and propagated, right? 
# If the timestamp field can be reassigned, is it necessary to restrict that 
the new field must be one of the old timestamp fields (from the left or right 
streams), so that the old watermarks may be reused somehow?

Thanks, Xingcan

> Support rowtime inner equi-join between two streams in the SQL API
> ------------------------------------------------------------------
>
>                 Key: FLINK-6233
>                 URL: https://issues.apache.org/jira/browse/FLINK-6233
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: hongyuhong
>            Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime < s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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

Reply via email to