[ 
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)

Reply via email to