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

2017-07-18 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user asfgit closed the pull request at:

https://github.com/apache/flink/pull/4324


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-18 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4324
  
Merging.

I'll open a JIRA to discuss how to pass error messages to users.

Thanks @hongyuhong and @wuchong!


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-17 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4324
  
Hi @hongyuhong, thanks for your feedback!
I agree, passing good error messages to users is important. However, this 
is very difficult in the optimizer because the optimizer is exploring different 
paths. If we throw an exception in the optimizer, we would kill the exploration 
and miss a valid plan. That's why I'm very hesitant to throw exceptions in 
rules. However, if the optimizer does not find a valid execution plan, it quits 
with an error message that tells the user that no valid plan could be found but 
does not point to a particular problem (such as an invalid join condition). 

A good solution for this is still needed. Maybe we can collect potential 
problems during optimization and provide these when the translation fails.




> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-16 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on the issue:

https://github.com/apache/flink/pull/4324
  
Hi @fhueske, sorry for the late feedback. The refactoring looks pretty 
great to me, i think it's good to be merged. The only thing i worry about is 
that there is no enough message for the user to figure out the exact error, do 
we have plan to improve this?

Thanks very much.
Yuhong


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-16 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4324
  
Hi @hongyuhong, it would be great if you could have a look as well since 
this PR is mostly your work.  
I'll appreciate any feedback.

Thank you, Fabian


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-14 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4324
  
Thanks for the review @wuchong. 

The `DataStream` join semantics can be implemented with this join by first 
window grouping both inputs with a UDAGG that collects all records in a list, 
and then joins the lists on the same time with `left.t BETWEEN right.t AND 
right.t` (we do not support a single `==` predicate on time so we need `>=` and 
`<=`, but this could be added, IMO). And then a UDF that crosses the lists. 

I think this is how you would need to do it in SQL (you could use UNNEST to 
do the Cartesian product). The Table API could have a syntactic shortcut for 
that but the internal logical representation would again be the same. 

Not sure if it adds much value. I think the windowed join semantics of this 
PR are nicer because they do not have cut-off points at the window boundaries 
(two records could be only 3 msecs apart but would not join because they are in 
different tumbling windows).


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-14 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on the issue:

https://github.com/apache/flink/pull/4324
  
The code looks good to me. Good refactoring!

+1 to merge.

BTW, I have a question. This time-windowed join is different with 
`DataStream.join(...).window(...)`.  The `DataStream.join.window` is joining 
two streams on a same window (such as 1 hour tumbling window).  Do we have plan 
to support it? 


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong closed the pull request at:

https://github.com/apache/flink/pull/3715


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong closed the pull request at:

https://github.com/apache/flink/pull/4266


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4266
  
Hi @hongyuhong and @wuchong, I opened a new PR which extends this PR. 
Please have a look and give feedback.

@hongyuhong can you close the PRs #3715 and this one? 

Thank you, Fabian


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

GitHub user fhueske opened a pull request:

https://github.com/apache/flink/pull/4324

[FLINK-6232] [table] Add processing time window inner join to SQL.

This is continuation and extension of PR #3715 and #4266 by @hongyuhong.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/fhueske/flink table-join

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/4324.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4324


commit 10b219678f13c0c21889f97f267dcf4c517045e5
Author: hongyuhong 
Date:   2017-07-06T03:24:04Z

[FLINK-6232] [table] Add support for processing time inner windowed stream 
join.

commit 3d671a2d1867aea2f3d4eee30b2772045917d6d4
Author: Fabian Hueske 
Date:   2017-07-12T22:49:30Z

[FLINK-6232] [table] Add SQL documentation for time window join.

- Add support for window join predicates in WHERE clause.
- Refactoring of WindowJoinUtil.
- Minor refactorings of join classes.




> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127193974
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

Yes, I think you are right. These checks should also check for updates and 
not retraction mode. 

Maybe it makes sense to integrate the whole append-only/updates check into 
the decoration rules. Same for the inference of unique keys (the other method 
in `UpdateCheckUtils`).


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: hongyuhong
>
> The goal of this 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127192628
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

Thank you for the explanation, that makes sense to me.  But I find 
`DataStreamOverAggregate` and `DataStreamGroupWindowAggregate` use 
`DataStreamRetractionRules.isAccRetract`, is that a misusage.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.
> 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127178517
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

the ` UpdateCheckUtils.isAppendOnly` recursively checks if any downstream 
operator produces updates. As soon as any downstream operator produces updates, 
the given operator has to be able to handle them. 

Updates can be encodes as retraction or be implicit per key-wise updates if 
the update producing and receiving operators use the same keys. Retraction 
updates are encoded as two messages. Non-retraction updates are encoded as 
single message and require a key to which they relate (`CRow.change == true` -> 
insert or update per key, `CRow.change == false`  -> delete on key). Right now, 
only UpsertTableSinks use non-retraction updates, but other operators such as 
unbounded joins will use it as well.

 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127176219
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+cRowWrapper.setChange(true)
+
+// initialize row state
+val rowListTypeInfo1: 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127173868
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

The following SQL `select a, sum(b), a+1 from t1 group by a` will optimized 
into the following nodes:

```
DataStreamCalc (AccRetract,  producesUpdates=false)
DataStreamGroupAggregate (AccRetract, producesUpdates=true)
   DataStreamScan (Acc, producesUpdates=fale)
```
The DataStreamCalc is append only, but is in AccRetract mode which means 
the output contains retraction. 

I think we want to check whether the input contains retraction, right? 





> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127173843
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+cRowWrapper.setChange(true)
+
+// initialize row state
+val rowListTypeInfo1: 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127166606
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

`isAccRetract` only checks how updates are encoded but not whether there 
are updates.
The current approach is correct, IMO.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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}
> 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127165003
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
--- End diff --

We should use `DataStreamRetractionRules.isAccRetract(input)` to check 
whether the input will produces updates.  


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4266
  
Thanks for the review @wuchong.
I'll address your comments in my upcoming PR.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126684002
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.types.Row
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
--- End diff --

Looks like the `data` is never used, can we remove it?


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126683783
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.types.Row
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /** test process time inner join **/
+  @Test
+  def testProcessTimeInnerJoin(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+val tEnv = TableEnvironment.getTableEnvironment(env)
+env.setStateBackend(getStateBackend)
+StreamITCase.testResults = mutable.MutableList()
+env.setParallelism(1)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '5' second and 
t2.proctime + interval '5' second"
+
+val data1 = new mutable.MutableList[(Int, Long, String)]
+data1.+=((1, 1L, "Hi1"))
+data1.+=((1, 2L, "Hi2"))
+data1.+=((1, 5L, "Hi3"))
+data1.+=((2, 7L, "Hi5"))
+data1.+=((1, 9L, "Hi6"))
+data1.+=((1, 8L, "Hi8"))
+
+val data2 = new mutable.MutableList[(Int, Long, String)]
+data2.+=((1, 1L, "HiHi"))
+data2.+=((2, 2L, "HeHe"))
+
+val t1 = env.fromCollection(data1).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data2).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val result = tEnv.sql(sqlQuery).toAppendStream[Row]
+result.addSink(new StreamITCase.StringSink[Row])
+env.execute()
+  }
+
+  /** test process time inner join with other condition **/
+  @Test
+  def testProcessTimeInnerJoinWithOtherCondition(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+val tEnv = TableEnvironment.getTableEnvironment(env)
+env.setStateBackend(getStateBackend)
+StreamITCase.testResults = mutable.MutableList()
--- End diff --

You can simply do `StreamITCase.clear` instead of this.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + INTERVAL '1' HOUR;
> {code}
> The 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126680468
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  inputSchema join rowtype schema
+* @param  rexBuilder  util to build rexNode
+* @param  config  table environment config
+* @return isRowTime, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  inputSchema: RowSchema,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (Boolean, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) = cnfCondition match {
+  case c: RexCall if cnfCondition.getKind == SqlKind.AND =>
+c.getOperands.asScala
+  .map(analyzeCondtionTermType(_, leftLogicalFieldCnt, 
inputSchema.logicalType))
+  .reduceLeft((l, r) => {
+(l._1 ++ r._1, l._2 ++ r._2)
+  })
+  case _ =>
+throw new TableException("A time-based stream join requires 
exactly " +
+  "two join predicates that bound the time in both directions.")
+}
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
--- End diff --

minor typo: conditon -> condition


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127129047
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  inputSchema join rowtype schema
+* @param  rexBuilder  util to build rexNode
+* @param  config  table environment config
+* @return isRowTime, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  inputSchema: RowSchema,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (Boolean, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) = cnfCondition match {
+  case c: RexCall if cnfCondition.getKind == SqlKind.AND =>
+c.getOperands.asScala
+  .map(analyzeCondtionTermType(_, leftLogicalFieldCnt, 
inputSchema.logicalType))
+  .reduceLeft((l, r) => {
+(l._1 ++ r._1, l._2 ++ r._2)
+  })
+  case _ =>
+throw new TableException("A time-based stream join requires 
exactly " +
+  "two join predicates that bound the time in both directions.")
+}
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+remainTerms match {
+  case Seq() => None
+  case _ =>

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126680044
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
--- End diff --

minor typo: condtion -> condition


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127142150
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+cRowWrapper.setChange(true)
+
+// initialize row state
+val rowListTypeInfo1: 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126671328
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, 
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamWindowJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+isRowTime: Boolean,
+leftLowerBound: Long,
+leftUpperBound: Long,
+remainCondition: Option[RexNode],
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+with CommonJoin
+with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamWindowJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  isRowTime,
+  leftLowerBound,
+  leftUpperBound,
+  remainCondition,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+joinToString(
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+joinExplainTerms(
+  super.explainTerms(pw),
+  schema.logicalType,
+  joinCondition,
+  joinType,
+  getExpressionString)
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
+val isRightAppendOnly = UpdateCheckUtils.isAppendOnly(right)
+if (!isLeftAppendOnly || !isRightAppendOnly) {
+  throw new TableException(
+"Windowed stream join does not support updates.")
+}
+
+val leftDataStream = 
left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+val rightDataStream = 
right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+
+// get the equality keys and other condition
+val joinInfo = JoinInfo.of(leftNode, rightNode, joinCondition)
+val leftKeys = joinInfo.leftKeys.toIntArray
+val rightKeys = joinInfo.rightKeys.toIntArray
+
+// generate join function
+val joinFunction 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r127153991
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+cRowWrapper.setChange(true)
+
+// initialize row state
+val rowListTypeInfo1: 

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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126683700
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.types.Row
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /** test process time inner join **/
+  @Test
+  def testProcessTimeInnerJoin(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+val tEnv = TableEnvironment.getTableEnvironment(env)
+env.setStateBackend(getStateBackend)
+StreamITCase.testResults = mutable.MutableList()
--- End diff --

You can simply do `StreamITCase.clear` instead of this.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/4266#discussion_r126671128
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
--- End diff --

remove unused import


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on the issue:

https://github.com/apache/flink/pull/4266
  
Thanks for the update @hongyuhong!
I will take this PR from here. The logic looks very good but I would like 
to refactor some parts (mainly  the `WindowJoinUtil`). 

I will open a new PR with your work and my commit on top, probably later 
today. 
It would be great if you could review and check my PR.

@wuchong your review is of course also highly welcome :-)

Thank you, Fabian


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-12 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r127113760
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Hi @fhueske , I think a good reason to put the check in `translateToPlan` 
is for throwing a proper exception message, such as `Left join is not 
supported...`. If we put the check in rule's matches or throw the exception in 
rule's matches, it will throw a obscure message with the logical plan.

IMO, a meaningful message is important for users. 


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * 

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

2017-07-12 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r126987096
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

`DataStreamWindowJoin` should support outer joins, but at the moment it 
does not. 
Until it supports outer joins, I would not translate to 
`DataStreamWindowJoin`.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-05 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125807639
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Should we check the type here?  I think the DataStreamWindowJoin can 
support inner and outer join, so i check the jointype in translateToPlan for 
convenient expansion.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-05 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on the issue:

https://github.com/apache/flink/pull/3715
  
Hi @fhueske, thanks for the reviewing, i have update the pr here for 
reviewing, and also create a new pr https://github.com/apache/flink/issues/4266 
to merge the commit, once you finish reviewing, i will close this pr.

Thanks very much.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-05 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on the issue:

https://github.com/apache/flink/pull/4266
  
Hi @wuchong, thanks for the reminding. There still have some modify in the 
new commit, so i want to reserve the older commit for easy reviewing, after the 
reviewing finish, i will close the pr https://github.com/apache/flink/pull/3715.

Thanks very much.



> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-05 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user wuchong commented on the issue:

https://github.com/apache/flink/pull/4266
  
Hi @hongyuhong , is this the same PR with #3715 ?  In order to 
rebase/remove merge commit, please do not create a new PR, otherwise committers 
may review an out-date PR or lose the review context. 

You can force update your repo branch via `git push  
flink-6232 --force` and close this PR.

Thanks,
Jark  


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-05 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

GitHub user hongyuhong opened a pull request:

https://github.com/apache/flink/pull/4266

[FLINK-6232][Table] support proctime inner windowed stream join

Thanks for contributing to Apache Flink. Before you open your pull request, 
please take the following check list into consideration.
If your changes take all of the items into account, feel free to open your 
pull request. For more information and/or questions please refer to the [How To 
Contribute guide](http://flink.apache.org/how-to-contribute.html).
In addition to going through the list, please provide a meaningful 
description of your changes.

- [ ] General
  - The pull request references the related JIRA issue ("[FLINK-XXX] Jira 
title text")
  - The pull request addresses only one issue
  - Each commit in the PR has a meaningful commit message (including the 
JIRA id)

- [ ] Documentation
  - Documentation has been added for new functionality
  - Old documentation affected by the pull request has been updated
  - JavaDoc for public methods has been added

- [ ] Tests & Build
  - Functionality added by the pull request is covered by tests
  - `mvn clean verify` has been executed successfully locally or a Travis 
build has passed


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/hongyuhong/flink flink-6232-re

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/4266.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4266


commit e2baf558e100eb7bb0d6b9927cb166b892f78d8f
Author: hongyuhong 
Date:   2017-07-06T03:24:04Z

[FLINK-6232][Table] support proctime inner windowed stream join




> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125440019
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Also check for `join.getJoinType == JoinType.INNER`


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125443593
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125464505
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -67,27 +66,47 @@ class JoinTest extends TableTestBase {
 streamUtil.verifySql(sqlQuery, expected)
   }
 
+  @Test(expected = classOf[TableException])
--- End diff --

Please add a brief comment why the query is expected to fail.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125427060
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -87,44 +98,38 @@ class DataStreamRowStreamJoin(
 
 val config = tableEnv.getConfig
 
+val leftIsAccRetract = DataStreamRetractionRules.isAccRetract(left)
+val rightIsAccRetract = DataStreamRetractionRules.isAccRetract(right)
+if (leftIsAccRetract || rightIsAccRetract) {
+  throw new TableException(
+"Retraction on stream window join is not supported yet.")
--- End diff --

```
Windowed stream join does not support updates.
```


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125456634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -162,24 +135,26 @@ object JoinUtil {
   (l._1 ++ r._1, l._2 ++ r._2)
 }
   case c: RexCall =>
-val timeIndicators = analyzeSingleConditionTerm(c, leftFieldCount, 
inputType)
+val timeIndicators = extractTimeIndicatorAccesses(c, 
leftFieldCount, inputType)
 timeIndicators match {
   case Seq() =>
 (Seq(), Seq(c))
   case Seq(v1, v2) =>
 if (v1._1 != v2._1) {
-  throw new TableException("The time indicators for each input 
should be the same.")
+  throw new TableException(
+"Both time attributes in a join condition must be of the 
same type.")
 }
 if (v1._2 == v2._2) {
-  throw new TableException("Both input's time indicators is 
needed.")
+  throw new TableException("Time-based join conditions " +
+"must reference the time attribute of both input tables.")
 }
 (Seq((v1._1, v1._2, c)), Seq())
--- End diff --

If we check the `RexCall` type here, we can analyze both sides of the 
condition independently and also handle cases where both attributes are on the 
same side of the condition.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448784
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125482464
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -87,44 +98,38 @@ class DataStreamRowStreamJoin(
 
 val config = tableEnv.getConfig
 
+val leftIsAccRetract = DataStreamRetractionRules.isAccRetract(left)
--- End diff --

Retractions are just one (of two) ways to encode updates in a stream. This 
join implementation does not support any kind of updates (which is fine). 
However, we have to check if the input streams produce updates or not. The 
`StreamTableEnvironment` has a private class `AppendOnlyValidator`. We should 
move `AppendOnlyValidator` and `UniqueKeyExtractor` into a class `UpdateUtils` 
together with the methods `isAppendOnly()` and `getUniqueKeyFields()`. 
`isAppendOnly()` checks if an operator produces updates or not and should be 
used here.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125452889
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
+expr match {
+  case c: RexCall =>
+

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125447106
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125478938
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -160,37 +181,52 @@ class JoinHarnessTest extends HarnessTestBase{
 
 testHarness.setProcessingTime(1)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(1: JInt, "aaa"), true), 1))
+  CRow(Row.of(1: JInt, "aaa1"), true), 1))
 testHarness.setProcessingTime(2)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(2: JInt, "bbb"), true), 2))
+  CRow(Row.of(2: JInt, "aaa2"), true), 2))
 testHarness.setProcessingTime(3)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(1: JInt, "aaa2"), true), 3))
+  CRow(Row.of(1: JInt, "aaa3"), true), 3))
+assert(testHarness.numKeyedStateEntries() == 4)
+assert(testHarness.numProcessingTimeTimers() == 2)
 
+// Do not store b elemets
--- End diff --

elemets -> elements


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125446548
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -238,13 +247,13 @@ class ProcTimeInnerJoin(
 
 }
 
-// loop the the other stream elments
+// loop the the other stream elements
--- End diff --

"the the"


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125465060
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -101,22 +120,22 @@ class JoinTest extends TableTestBase {
 verifyTimeBoundary(
--- End diff --

I'd add another case where both bounds are negative.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125447810
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457556
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
--- End diff --

Unsupport -> Unsupported


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457951
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
   }
 
-val timeOffset =
-  // only preceding offset need to store records
-  if (tmpTimeOffset < 0)
-// determain the boudary value
+val (leftLiteral, rightLiteral) =
+  reduceTimeExpression(
+timeCall.operands.get(0),
+timeCall.operands.get(1),
+rexBuilder,
+config)
+val tmpTimeOffset: Long =
+  if (isLeftExprBelongLeftTable) rightLiteral - leftLiteral else 
leftLiteral - rightLiteral
+
+val boundary =
+  tmpTimeOffset.signum * (
 if (timeTerm.getKind == SqlKind.LESS_THAN || timeTerm.getKind == 
SqlKind.GREATER_THAN) {
-  -tmpTimeOffset - 1
+  tmpTimeOffset.abs - 1
 } else {
-  -tmpTimeOffset
-}
-  else 0
+  tmpTimeOffset.abs
+})
 
-(timeOffset, isLeftTableTimeOffset)
+(boundary, isLeftLowerBound)
   }
 
   /**
 * Calcute the time boundary. Replace the rowtime/proctime with zero 
literal.
 * For example:
-*  a.proctime - inteval '1' second > b.proctime - interval '1' second 
- interval '2' second
-*  |---left--|   
|---right---\
-* then the boundary of a is right - left:
-*  ((0 - 1000) - 2000) - (0 - 1000) = -2000(-preceding, +following)
+* b.proctime - interval '1' second - interval '2' second will be 
translate to
--- End diff --

will be translate -> will be translated


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: hongyuhong
>
> The goal of this issue 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125423693
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
--- End diff --

This can be done more concisely as:

```
(timeType, leftLowerBoundary, leftUpperBoundary, remainCondition) =
  WindowJoinUtil.analyzeTimeBoundary(...)
```


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125458518
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
+expr match {
+  case c: RexCall =>
+

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448467
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125417060
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -44,15 +46,19 @@ class DataStreamRowStreamJoin(
 leftSchema: RowSchema,
 rightSchema: RowSchema,
 schema: RowSchema,
+timeType: RelDataType,
--- End diff --

Can be a boolean flag `isRowTime`


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457882
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
   }
 
-val timeOffset =
-  // only preceding offset need to store records
-  if (tmpTimeOffset < 0)
-// determain the boudary value
+val (leftLiteral, rightLiteral) =
+  reduceTimeExpression(
+timeCall.operands.get(0),
+timeCall.operands.get(1),
+rexBuilder,
+config)
+val tmpTimeOffset: Long =
+  if (isLeftExprBelongLeftTable) rightLiteral - leftLiteral else 
leftLiteral - rightLiteral
+
+val boundary =
+  tmpTimeOffset.signum * (
 if (timeTerm.getKind == SqlKind.LESS_THAN || timeTerm.getKind == 
SqlKind.GREATER_THAN) {
-  -tmpTimeOffset - 1
+  tmpTimeOffset.abs - 1
 } else {
-  -tmpTimeOffset
-}
-  else 0
+  tmpTimeOffset.abs
+})
 
-(timeOffset, isLeftTableTimeOffset)
+(boundary, isLeftLowerBound)
   }
 
   /**
 * Calcute the time boundary. Replace the rowtime/proctime with zero 
literal.
--- End diff --

```
Calculates the time boundary by replacing the time attribute by a zero 
literal and reducing the expression.
```


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + INTERVAL '1' 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125446795
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -238,13 +247,13 @@ class ProcTimeInnerJoin(
 
 }
 
-// loop the the other stream elments
+// loop the the other stream elements
 val oppositeKeyIter = oppoRowMapState.keys().iterator()
 while (oppositeKeyIter.hasNext) {
   val eleTime = oppositeKeyIter.next()
-  if (eleTime < oppoExpiredTime) {
+  if (eleTime < oppoLowerTime) {
 listToRemove.add(eleTime)
-  } else {
+  } else if (eleTime <= oppoUpperTime){
--- End diff --

+space between `) {`


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125456339
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -162,24 +135,26 @@ object JoinUtil {
   (l._1 ++ r._1, l._2 ++ r._2)
 }
   case c: RexCall =>
-val timeIndicators = analyzeSingleConditionTerm(c, leftFieldCount, 
inputType)
+val timeIndicators = extractTimeIndicatorAccesses(c, 
leftFieldCount, inputType)
 timeIndicators match {
   case Seq() =>
 (Seq(), Seq(c))
   case Seq(v1, v2) =>
 if (v1._1 != v2._1) {
-  throw new TableException("The time indicators for each input 
should be the same.")
+  throw new TableException(
+"Both time attributes in a join condition must be of the 
same type.")
 }
 if (v1._2 == v2._2) {
-  throw new TableException("Both input's time indicators is 
needed.")
+  throw new TableException("Time-based join conditions " +
+"must reference the time attribute of both input tables.")
 }
 (Seq((v1._1, v1._2, c)), Seq())
--- End diff --

I think the way the `boolean` field is derived is a bit dangerous. We 
assume here that the `RexCall` is a comparison where the first operand is the 
left term and the second operand is the right term but it could also be any 
other call (with an arbitrary number of operands) that returns a boolean. 
Although we later check that the `RexCall` is a comparison, I think we should 
do this here.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125421732
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ package org.apache.flink.table.plan.rules.datastream
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
--- End diff --

I'm not sure if we can store these values as local variables. It kind 
assumes that the rule is called in a certain way (first check `matches()` and 
immediately after that `convert()`. The rule might also be called concurrently 
from different threads. I'd rather run the same analysis again in `convert()`.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125463323
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
--- End diff --

Can we use `RowSchema.mapRexNode()` to compute the 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125454546
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -191,8 +166,10 @@ object JoinUtil {
 
   /**
* analysis if condition term has time indicator
--- End diff --

```
Extracts all time indicator attributes that are accessed in an expression.
```


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125459202
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -300,22 +271,27 @@ object JoinUtil {
   }
 }
 
-val literalRex = replaceTimeFieldWithLiteral(rexNode)
+val literalLeftRex = replaceTimeFieldWithLiteral(leftRexNode)
+val literalRightRex = replaceTimeFieldWithLiteral(rightRexNode)
 
 val exprReducer = new ExpressionReducer(config)
 val originList = new util.ArrayList[RexNode]()
-originList.add(literalRex)
+originList.add(literalLeftRex)
+originList.add(literalRightRex)
 val reduceList = new util.ArrayList[RexNode]()
 exprReducer.reduce(rexBuilder, originList, reduceList)
 
-reduceList.get(0) match {
+val literals = reduceList.map(f => f match {
   case call: RexCall =>
--- End diff --

Which case is this? 
Isn't a blind cast of the first operand to `RexLiteral` dangerous and could 
result in a `ClassCastException`?


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448648
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 

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

2017-07-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125461422
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
--- End diff --

```
Computes the absolute bound on the left operand of a comparison expression 
and 
whether the bound is an upper or lower bound.
```


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on the issue:

https://github.com/apache/flink/pull/3715
  
Hi @fhueske, sorry for late reply, i have update the pr.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125205215
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType

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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203460
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+ 

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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203271
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203253
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125202985
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType

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

2017-07-02 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125202807
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123031759
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123209008
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
--- End diff --

add a new line before the method


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + INTERVAL '1' HOUR;
> {code}
> The 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123016738
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123232793
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226130
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123228716
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123227488
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123037106
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123030850
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123020909
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123026855
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123009399
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
--- End diff --

The check is only approximate, i.e., the stream join operator might not be 
able to execute the query even if this check is passed.

For example it only checks if there is at least one time indicator in the 
condition. However, we would need to check that there are exactly two 
conjunctive terms that have time indicator attributes on both sides and define 
bounds to both sides. Basically the complete analysis that we later do in the 
join. I think we can do this analysis already in the rule and pass the result 
of the analysis to the join.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123004606
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
--- End diff --

can be simplified to `c.getOperands.exists(isExistTumble(_))`


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123236585
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
+import org.apache.flink.table.codegen.GeneratedFunction
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+
+class JoinHarnessTest extends HarnessTestBase{
+
+  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
+INT_TYPE_INFO,
+STRING_TYPE_INFO),
+Array("a", "b"))
+
+
+  val funcCode: String =
+"""
+  |public class TestJoinFunction
+  |  extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
+  |  transient org.apache.flink.types.Row out =
+  |new org.apache.flink.types.Row(4);
+  |  public TestJoinFunction() throws Exception {}
+  |
+  |  @Override
+  |  public void open(org.apache.flink.configuration.Configuration 
parameters)
+  |  throws Exception {}
+  |
+  |  @Override
+  |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
+  |   throws Exception {
+  |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
+  |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
+  |
+  |   out.setField(0, in1.getField(0));
+  |   out.setField(1, in1.getField(1));
+  |   out.setField(2, in2.getField(0));
+  |   out.setField(3, in2.getField(1));
+  |
+  |   c.collect(out);
+  |
+  |  }
+  |
+  |  @Override
+  |  public void close() throws Exception {}
+  |}
+""".stripMargin
+
+  @Test
+  def testProcTimeJoin() {
+
+val joinProcessFunc = new ProcTimeInnerJoin(10, 20, rT, rT, 
"TestJoinFunction", funcCode)
+
+val operator: KeyedCoProcessOperator[Integer, CRow, CRow, CRow] =
+  new KeyedCoProcessOperator[Integer, CRow, CRow, 
CRow](joinProcessFunc)
+val testHarness: TwoInputStreamOperatorTestHarness[CRow, CRow, CRow] =
+  new KeyedTwoInputStreamOperatorTestHarness[Integer, CRow, CRow, 
CRow](
+   operator,
+   new TupleRowKeySelector[Integer](0),
+   new TupleRowKeySelector[Integer](0),
+   BasicTypeInfo.INT_TYPE_INFO,
+   1,1,0)
+
+testHarness.open()
+
+testHarness.setProcessingTime(1)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa"), true), 1))
+testHarness.setProcessingTime(2)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(2: JInt, "bbb"), true), 2))
+

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123035300
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123233427
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123231931
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException0(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException1(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
+  "and t1.proctime > t2.proctime - interval '5' second"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should use same time indicator
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException2(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'rowtime.rowtime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123229245
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123238837
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
+import org.apache.flink.table.codegen.GeneratedFunction
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+
+class JoinHarnessTest extends HarnessTestBase{
+
+  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
+INT_TYPE_INFO,
+STRING_TYPE_INFO),
+Array("a", "b"))
+
+
+  val funcCode: String =
+"""
+  |public class TestJoinFunction
+  |  extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
+  |  transient org.apache.flink.types.Row out =
+  |new org.apache.flink.types.Row(4);
+  |  public TestJoinFunction() throws Exception {}
+  |
+  |  @Override
+  |  public void open(org.apache.flink.configuration.Configuration 
parameters)
+  |  throws Exception {}
+  |
+  |  @Override
+  |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
+  |   throws Exception {
+  |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
+  |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
+  |
+  |   out.setField(0, in1.getField(0));
+  |   out.setField(1, in1.getField(1));
+  |   out.setField(2, in2.getField(0));
+  |   out.setField(3, in2.getField(1));
+  |
+  |   c.collect(out);
+  |
+  |  }
+  |
+  |  @Override
+  |  public void close() throws Exception {}
+  |}
+""".stripMargin
+
+  @Test
+  def testProcTimeJoin() {
+
+val joinProcessFunc = new ProcTimeInnerJoin(10, 20, rT, rT, 
"TestJoinFunction", funcCode)
+
+val operator: KeyedCoProcessOperator[Integer, CRow, CRow, CRow] =
+  new KeyedCoProcessOperator[Integer, CRow, CRow, 
CRow](joinProcessFunc)
+val testHarness: TwoInputStreamOperatorTestHarness[CRow, CRow, CRow] =
+  new KeyedTwoInputStreamOperatorTestHarness[Integer, CRow, CRow, 
CRow](
+   operator,
+   new TupleRowKeySelector[Integer](0),
+   new TupleRowKeySelector[Integer](0),
+   BasicTypeInfo.INT_TYPE_INFO,
+   1,1,0)
+
+testHarness.open()
+
+testHarness.setProcessingTime(1)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa"), true), 1))
+testHarness.setProcessingTime(2)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(2: JInt, "bbb"), true), 2))
+

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226695
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123229427
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123231612
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException0(): Unit = {
--- End diff --

We should not need an ITCase for these checks. The problem is that the 
validation is done during translation. If we would move the correctness checks 
into the optimizer, we don't need to translate the program.


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123020714
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123225232
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
--- End diff --

Does it make sense to split the implementation into two operators:
1. both streams need to be buffered (`l.ptime > r.ptime - 10.secs AND 
l.ptime < r.ptime + 5.secs`)
2. only one stream needs to be buffered (`l.ptime > r.ptime - 10.secs AND 
l.ptime < r.ptime - 5.secs`)


> Support proctime inner equi-join between two streams in the SQL API
> ---
>
> Key: FLINK-6232
> URL: https://issues.apache.org/jira/browse/FLINK-6232
> 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.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + 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.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> 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)


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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226087
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
+  /** state to record last timer of right stream, 0 means no timer **/
+  private var timerState2: ValueState[Long] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
+  s"Code:\n$genJoinFuncCode")
+val clazz = compile(
+  getRuntimeContext.getUserCodeClassLoader,
+  genJoinFuncName,
+  genJoinFuncCode)
+LOG.debug("Instantiating JoinFunction.")
+joinFunction = clazz.newInstance()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = 

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123021563
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

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

2017-06-21 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on FLINK-6232:
---

Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123205673
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+   

  1   2   3   >