[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-11-08 Thread Nikolay Vasilishin (JIRA)

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

Nikolay Vasilishin commented on FLINK-4565:
---

Hi, guys, I faced some problems.
Now I have IN operator for literals, subqueries are not supported yet.
You can find my code [on my 
github|https://github.com/NickolayVasilishin/flink/tree/FLINK-4565].
So, the problems are:
#   I’m using HashSet to check entry. The code generates in 
[ScalarOperators.scala|https://github.com/apache/flink/compare/master...NickolayVasilishin:FLINK-4565#diff-423fbbd7967ec8e9feee7c1b7062b884R106].
 But creating the object of HashSet and adding elements to it is placing into 
the body of  public void flatMap(..) method, which invokes for every row, as I 
understand. The comment above the 
[CodeGenerator#generateResultExpression|https://github.com/NickolayVasilishin/flink/blob/FLINK-4565/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala#L305]
 says that reusable code will be reused internally, but how to check if it 
works properly?
#   The problem in 
[ExpressionParser.scala|https://github.com/NickolayVasilishin/flink/blob/FLINK-4565/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala].
 Since I’ve implemented matching pattern for IN operator, it conflicts with 
initCap() function ([in this 
test|https://github.com/NickolayVasilishin/flink/blob/FLINK-4565/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala#L156].
 During the expression parsing it goes through 
[ExpressionParser#functionIdent|https://github.com/NickolayVasilishin/flink/blob/FLINK-4565/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala#L79]
 method (where ‘not’-checks occur on operators such as AS, COUNT, IF and “my” 
IN), where it gets into my [suffixIn 
method|https://github.com/NickolayVasilishin/flink/blob/FLINK-4565/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala#L194]
 and fails with an exception: 
{noformat}
Could not parse expression at column 6: `(' expected but `i' found f0.initCap().
{noformat}
I expected that expression will go to the next check if current fails. 
Also my check cannot be the last check in this chain.
So what are ways to solve this problem? Maybe there is a solution to make 
matcher not so greedy? The easiest way I think is to rename IN operator to ISIN 
operator like it is implemented in Spark.


Appreciate any help and thanks in advance.


> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Nikolay Vasilishin
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-10-24 Thread Nikolay Vasilishin (JIRA)

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

Nikolay Vasilishin commented on FLINK-4565:
---

Hi [~jark].
Yes, i'm working on it right now.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Nikolay Vasilishin
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-10-24 Thread Jark Wu (JIRA)

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

Jark Wu commented on FLINK-4565:


Hi [~nvasilishin], do you still plan to implement this issue ? Otherwise, I can 
have a try on this .

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Nikolay Vasilishin
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-19 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

Considering my current schedule and the importance of this issue, I'd rather 
let someone else take of it. The work I've already done is not enough to be of 
any use for the person that will take care of this issue so I will just turn 
the issue unassigned and let someone else work on it. I will keep watching the 
issue, just in case.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-19 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

I looked into the code for this issue. It would be very tricky to support IN 
for the Table API as we currently separate expressions ({{RexNodes}}) from 
operators ({{RelNodes}}).

In the end we would need to call something similar to 
{{org.apache.calcite.sql2rel.SqlToRelConverter.Blackboard#convertExpression}}

{code}
final RexSubQuery in = RexSubQuery.in(root.rel, builder.build());
return op.isNotIn()
? rexBuilder.makeCall(SqlStdOperatorTable.NOT, in)
: in;
{code}

{{RexSubQuery}} is a rex node, however, it needs access to 
{{Table}}/{{LogicalNode}} to get the {{RelNode}}.

The following steps need to be implemented:
- For Java API: Create a {{UnresolvedTableReference}} expression that takes the 
name of the table.
- For Java API: Resolve the name of the table in 
{{org.apache.flink.api.table.plan.logical.LogicalNode#resolveExpressions}} to 
{{TableReference}} using the table environment that is available in this 
method. {{TableReference}} then has a {{Table}} field.
- Create an expression {{In}} that takes a {{TableReference}} and does the 
above code snippet in {{toRexNode}}.

[~chobeat] I hope this helps otherwise it also fine if you let someone else 
implement this.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

I'm very sorry. Maybe I underestimated this issue after thinking about it. It 
is the first combination of expression and table which makes this issue tricky. 
Yes the validation is very complicated especially because we have 2 validations 
one for SQL and one for the Table API (with heavy Scala magic). If you are 
still wanna do that, I can also help you in a private chat. Otherwise e.g. 
FLINK-4599 would be easier. I will add the "starter" label to easier tasks.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

After a couple of days of work I believe this task is a bit too complex for me. 
I'm struggling with the complexity of constructing the RelNode and doing the 
validation. Missing an example similar enough to take inspiration, it's hard to 
work on this by myself. I can imagine that this would take no more than a 
couple of hours to someone with experience with Flink and Calcite and despite 
all the stuff I've learnt in these days, I'm seeing no progress. My biggest 
problem is that I cannot really wrap my head around many of the exception it 
throws when performing construction and validation.

If there's someone patient enough to go through what I've done right now, I can 
open a WIP PR or link to my repo, pointing to all the stuff that gets me 
confused. Otherwise I can leave this issue to someone more skilled than me and 
retry with something easier and more straightforward.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

Yes, this would fail then. But I think there is no other good solution if you 
specify expressions as String as we do in the Java API.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

I'm no expert in the Table API, but if a user doesn't explicitely register a 
table in the table register, will this fail? 

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

I think it is ok, if you just parse the table name and introduce a 
`UnresolvedTable` expression which can later be resolved/looked up in the table 
registry.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-08 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

I defined this parser combinator rule ```lazy val in:PackratParser[Expression] 
= term ~ "IN" ~ table ^^{
case l ~ _ ~ r => In(l,r)
  }```

`In` takes an expression (containing the field of the left table to be matched 
against the right table) and a Table. I need to define  a parser for a Table 
but I never worked with parser-combinators so I have no idea how to do that. 
Any suggestion?

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

That's exactly why I'm asking it: I had no idea what to use as a right operand. 

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Fabian Hueske (JIRA)

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

Fabian Hueske commented on FLINK-4565:
--

I thought rather of something like this: {{ds1.where('a in d2)}}. This would be 
the first time that a {{Table}} would be included in an expression. So not sure 
how easy it is to put it together and whether there are side effects to 
consider.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

The resulting API should look like `ds1.join(ds2).where('a in 'b)` or 
`ds1.in(ds2).something('a,'b)`?

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

It seems my answer was to late. Yes, I would also go for the second approach.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

Calcite translates the IN operator in 
{{org.apache.calcite.sql2rel.SqlToRelConverter#convertExpression}}. Calcite 
translates this into an Aggregate and Join. After fixing some issue in 
"DataSetAggregate" we can execute: {{"SELECT WordCount.word FROM WordCount 
WHERE WordCount.word IN (SELECT WordCount1.word AS w FROM WordCount1)"}}. The 
plan looks like:

{code}
== Physical Execution Plan ==
Stage 4 : Data Source
content : collect elements with CollectionInputFormat
Partitioning : RANDOM_PARTITIONED

Stage 3 : Map
content : from: (word, frequency)
ship_strategy : Forward
exchange_mode : BATCH
driver_strategy : Map
Partitioning : RANDOM_PARTITIONED

Stage 8 : Map
content : from: (word, frequency)
ship_strategy : Forward
exchange_mode : BATCH
driver_strategy : Map
Partitioning : RANDOM_PARTITIONED

Stage 7 : Map
content : prepare select: (word)
ship_strategy : Forward
exchange_mode : PIPELINED
driver_strategy : Map
Partitioning : RANDOM_PARTITIONED

Stage 6 : GroupCombine
content : groupBy: (word), select:(word)
ship_strategy : Forward
exchange_mode : PIPELINED
driver_strategy : Sorted Combine
Partitioning : RANDOM_PARTITIONED

Stage 5 : GroupReduce
content : groupBy: (word), 
select:(word)
ship_strategy : Hash Partition 
on [0]
exchange_mode : PIPELINED
driver_strategy : Sorted Group 
Reduce
Partitioning : 
RANDOM_PARTITIONED

Stage 2 : Join
content : where: 
(=(word, w)), join: (word, frequency, w)
ship_strategy : Hash 
Partition on [0]
exchange_mode : 
PIPELINED
driver_strategy : 
Hybrid Hash (build: from: (word, frequency) (id: 3))
Partitioning : 
RANDOM_PARTITIONED

Stage 1 : FlatMap
content : 
select: (word)
ship_strategy : 
Forward
exchange_mode : 
PIPELINED
driver_strategy 
: FlatMap
Partitioning : 
RANDOM_PARTITIONED

Stage 0 : Data 
Sink
content 
: org.apache.flink.api.java.io.DiscardingOutputFormat

ship_strategy : Forward

exchange_mode : PIPELINED

Partitioning : RANDOM_PARTITIONED
{code}

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

I imagined it could happen that way and I was going to open calcite source to  
look for that. You saved me a lot of time. I will go for the second approach, 
considering that it looks a lot like what happens for all the others operators 
I've seen.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>Assignee: Simone Robutti
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Fabian Hueske (JIRA)

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

Fabian Hueske commented on FLINK-4565:
--

SQL {{IN}} does not require a special implementation because it can be 
translated into an inner join (as long as the subquery is not correlated with 
the outer query). 
We use Calcite to convert the subquery into a join 
({{RelDecorrelator.decorrelateQuery(relNode)}}) in 
{{org.apache.flink.api.table.BatchTableEnvironment}} line 246.
After the conversion, IN is represented and executed as join.

I think the integration with the Table API can happen in two ways:

1. Generate a RelNode plan with a subquery and let Calcite do the decorrelation 
before the query is optimized (same approach as SQL)
2. Generate a RelNode plan with a Join from the beginning.

I would try to go for the second approach first.


> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-07 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

Actually the confusing part is that the translation to the execution plan 
should already exist but I can't find where it happens. 



> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Jark Wu (JIRA)

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

Jark Wu commented on FLINK-4565:


I think what is confusing [~chobeat] is what's "IN" 's counterpart in DataSet 
transformation, right ?  Maybe we can use broadcast variables [1] to support 
this feature. In addition, [FLINK-4541] may be could implemented together. 

[1] 
https://ci.apache.org/projects/flink/flink-docs-master/dev/batch/index.html#broadcast-variables


> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

I haven't assigned the issue to myself because I'm still studying to clear the 
remaining grey areas. Tomorrow I will, if no one else did. 

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Konstantin Boudnik (JIRA)

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

Konstantin Boudnik commented on FLINK-4565:
---

If someone starts working on an issue, shouldn't it be assigned to that person, 
so no two people work on the same task simultaneously? What is the accepted 
drill here, guys? Appreciate the info!

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

tableEnv.sql(...) returns a Table containing a LogicalRelNode (which is a 
LogicalNode and just wraps RelNodes; Calcite already did the validation). The 
Table API returns Tables containing specific LogicalNodes depending on the 
operation (Table API does the validation). All LogicalNodes have a "construct" 
method translating the Flink logical operators to Calcites RelNode 
representation for optimization. The magic happens in 
Batch/StreamTableEnvironment.translate() where the RelNodes are optimized using 
a specific set of rules and converted to "DataSet/DataStreamRel"s also by using 
specific rules.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

Ok, the testing part seems easy and straightforward. I'm still missing a part 
of this whole process. For what I saw, the tableEnv.sql(...) is parsed by 
Calcite and it returns a SqlRelNode, while the Table API translates the calls 
and expressions to a LogicalNode. Both are then translated to an execution plan 
to be executed in the runtime. I can't really understand how and where this 
happens. The `.plan` and `.runtime` package contains some of the things I was 
expecting but I can't really follow the logic. Also, if "IN" is supported, 
there should be a translation somewhere for this operator and I can't find it.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Timo Walther (JIRA)

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

Timo Walther commented on FLINK-4565:
-

Hi [~chobeat], yes I think this issue is good for a beginner. Flink SQL already 
supports things like "SELECT x FROM myTable WHERE x IN (SELECT y FROM 
myOtherTable)". Nonetheless, we should add tests for it. The question is if we 
could make this functionality also available to the Table API. The Table API 
has Expressions (expressionDsl.scala for Scala and ExpressionParser for Java) 
and LogicalNodes (operators.scala). This type would combine both. 
{{expr.in(table)}}

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-4565) Support for SQL IN operator

2016-09-06 Thread Simone Robutti (JIRA)

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

Simone Robutti commented on FLINK-4565:
---

I would like to take care of this issue to get my hands dirty with some work on 
the Table API. I've spent the last 2 days working with them and breaking them a 
bit to understand all the stuff they do. If you believe this task is ok for a 
newbie, please give me a few pointers to better understand the issue so I can 
start working on it
.

> Support for SQL IN operator
> ---
>
> Key: FLINK-4565
> URL: https://issues.apache.org/jira/browse/FLINK-4565
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Timo Walther
>
> It seems that Flink SQL supports the uncorrelated sub-query IN operator. But 
> it should also be available in the Table API and tested.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


<    1   2