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

Piotr Nowojski commented on FLINK-10379:
----------------------------------------

Thanks [~hequn8128] :)

Probably we should first also think about how to solve this. Do you have some 
specific solution in mind?

 I wasn't taking part of the discussion/effort that resulted in removing 
{{Table#join(String)}} method, but there were probably some reasons behind 
that. I heard that part of it was to switch to using implicit conversions of 
table functions to {{Table}} instances in Scala. Obviously that doesn't work, 
but maybe we could provide something to explicitly apply argument to 
{{TableFunction}} instance and convert it to {{Table}}? Something along the 
lines:

 
{code:java}
// Register the function.
TableFunction<Row> split = tableEnv.registerFunction("split", new Split("#"));

myTable.join(split.apply("a").as("word, length"));
{code}
(But I'm pretty unfamiliar with Table API, so treat this with grain of salt)

What do you think?

 

> Can not use Table Functions in Java Table API
> ---------------------------------------------
>
>                 Key: FLINK-10379
>                 URL: https://issues.apache.org/jira/browse/FLINK-10379
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API &amp; SQL
>    Affects Versions: 1.6.1
>            Reporter: Piotr Nowojski
>            Assignee: Hequn Cheng
>            Priority: Critical
>
> As stated in the 
> [documentation|[https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/udfs.html#table-functions],]
>  this is how table functions should be used in Java Table API:
> {code:java}
> // Register the function.
> tableEnv.registerFunction("split", new Split("#"));
> myTable.join("split(a) as (word, length)");
> {code}
> However {{Table.join(String)}} was removed sometime ago and now it is 
> impossible to use Table Functions in Java Table API.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to