You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "hongyuhong (JIRA)" <ji...@apache.org> on 2017/04/07 03:29:41 UTC

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

hongyuhong commented on FLINK-6233:
-----------------------------------

Hi [~fhueske], i would deal with the out-of-order records like follow:
{code}
A:  ---A(2)---A(1)-------watermark(3)-------A(4)----A(6)---A(5)---watermark(10)-----|--------------------------------|
B:  ---------B(2)--------watermark(3)----B(4)------B(5)-----------watermark(10)-----|--------------------------------|
{code}

When watermark come, it will sort the records between  watermark, then output the join result as the timestamp order, and
also expire watermark - winsize records. And when processElement, records before currrent watermark will be discarded.
watermark(3) output:
A1B2, A2B2
watermark(10) output:
A1B4, A2B4
A4B2, A4B4
A1B5, A2B5, A4B5
A5B2, A5B5
A6B2, A6B4, A6B5

What do you think?

> 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: hongyuhong
>
> 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 &lt; 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.3.15#6346)