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

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

asfgit closed pull request #6744: [FLINK-10379][docs,table] Fix Table Function 
docs
URL: https://github.com/apache/flink/pull/6744
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md
index a9fd94f10d5..fb23511d78e 100644
--- a/docs/dev/table/sql.md
+++ b/docs/dev/table/sql.md
@@ -483,11 +483,11 @@ FROM Orders CROSS JOIN UNNEST(tags) AS t (tag)
     </tr>
     <tr>
        <td>
-        <strong>Join with User Defined Table Functions (UDTF)</strong><br>
+        <strong>Join with Table Function</strong><br>
         <span class="label label-primary">Batch</span> <span class="label 
label-primary">Streaming</span>
       </td>
        <td>
-        <p>UDTFs must be registered in the TableEnvironment. See the <a 
href="udfs.html">UDF documentation</a> for details on how to specify and 
register UDTFs. </p>
+        <p>To use User-Defined Table Function in SQL it must be registered in 
the TableEnvironment. See the <a href="udfs.html">UDF documentation</a> for 
details on how to specify and register UDTFs. </p>
         <p>Inner Join</p>
 {% highlight sql %}
 SELECT users, tag
diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md
index f50b0f54554..c5aa160cf1f 100644
--- a/docs/dev/table/tableApi.md
+++ b/docs/dev/table/tableApi.md
@@ -627,29 +627,29 @@ Table result = left.join(right)
       </td>
     </tr>
     <tr>
-       <td>
-        <strong>TableFunction Inner Join</strong><br>
+      <td>
+        <strong>Inner Join with Table Function</strong><br>
         <span class="label label-primary">Batch</span> <span class="label 
label-primary">Streaming</span>
       </td>
        <td>
         <p>Joins a table with a the results of a table function. Each row of 
the left (outer) table is joined with all rows produced by the corresponding 
call of the table function. A row of the left (outer) table is dropped, if its 
table function call returns an empty result.
         </p>
 {% highlight java %}
-// register function
+// register User-Defined Table Function
 TableFunction<String> split = new MySplitUDTF();
-tEnv.registerFunction("split", split);
+tableEnv.registerFunction("split", split);
 
 // join
 Table orders = tableEnv.scan("Orders");
 Table result = orders
-    .join(new Table(tEnv, "split(c)").as("s", "t", "v"))
+    .join(new Table(tableEnv, "split(c)").as("s", "t", "v"))
     .select("a, b, s, t, v");
 {% endhighlight %}
       </td>
     </tr>
     <tr>
-       <td>
-        <strong>TableFunction Left Outer Join</strong><br>
+      <td>
+        <strong>Left Outer Join with Table Function</strong><br>
         <span class="label label-primary">Batch</span> <span class="label 
label-primary">Streaming</span>
       </td>
       <td>
@@ -657,14 +657,14 @@ Table result = orders
         <p><b>Note:</b> Currently, the predicate of a table function left 
outer join can only be empty or literal <code>true</code>.</p>
         </p>
 {% highlight java %}
-// register function
+// register User-Defined Table Function
 TableFunction<String> split = new MySplitUDTF();
-tEnv.registerFunction("split", split);
+tableEnv.registerFunction("split", split);
 
 // join
 Table orders = tableEnv.scan("Orders");
 Table result = orders
-    .leftOuterJoin(new Table(tEnv, "split(c)").as("s", "t", "v"))
+    .leftOuterJoin(new Table(tableEnv, "split(c)").as("s", "t", "v"))
     .select("a, b, s, t, v");
 {% endhighlight %}
       </td>
@@ -729,7 +729,7 @@ val fullOuterResult = left.fullOuterJoin(right, 'a === 
'd).select('a, 'b, 'e)
       <td>
         <p><b>Note:</b> Time-windowed joins are a subset of regular joins that 
can be processed in a streaming fashion.</p>
 
-        <p>A time-windowed join requires at least one equi-join predicate and 
a join condition that bounds the time on both sides. Such a condition can be 
defined by two appropriate range predicates (<code>&lt;, &lt;=, &gt;=, 
&gt;</code>) or a single equality predicate that compares <a 
href="streaming.html#time-attributes">time attributes</a> of the same type 
(i.e., processing time or event time) of both input tables.</p> 
+        <p>A time-windowed join requires at least one equi-join predicate and 
a join condition that bounds the time on both sides. Such a condition can be 
defined by two appropriate range predicates (<code>&lt;, &lt;=, &gt;=, 
&gt;</code>) or a single equality predicate that compares <a 
href="streaming.html#time-attributes">time attributes</a> of the same type 
(i.e., processing time or event time) of both input tables.</p>
         <p>For example, the following predicates are valid window join 
conditions:</p>
 
         <ul>
@@ -748,15 +748,15 @@ val result = left.join(right)
       </td>
     </tr>
     <tr>
-       <td>
-        <strong>TableFunction Inner Join</strong><br>
+      <td>
+        <strong>Inner Join with Table Function</strong><br>
         <span class="label label-primary">Batch</span> <span class="label 
label-primary">Streaming</span>
       </td>
        <td>
         <p>Joins a table with a the results of a table function. Each row of 
the left (outer) table is joined with all rows produced by the corresponding 
call of the table function. A row of the left (outer) table is dropped, if its 
table function call returns an empty result.
         </p>
         {% highlight scala %}
-// instantiate function
+// instantiate User-Defined Table Function
 val split: TableFunction[_] = new MySplitUDTF()
 
 // join
@@ -767,15 +767,15 @@ val result: Table = table
         </td>
     </tr>
     <tr>
-       <td>
-        <strong>TableFunction Left Outer Join</strong><br>
+      <td>
+        <strong>Left Outer Join with Table Function</strong><br>
         <span class="label label-primary">Batch</span> <span class="label 
label-primary">Streaming</span></td>
        <td>
         <p>Joins a table with a the results of a table function. Each row of 
the left (outer) table is joined with all rows produced by the corresponding 
call of the table function. If a table function call returns an empty result, 
the corresponding outer row is preserved and the result padded with null values.
         <p><b>Note:</b> Currently, the predicate of a table function left 
outer join can only be empty or literal <code>true</code>.</p>
         </p>
 {% highlight scala %}
-// instantiate function
+// instantiate User-Defined Table Function
 val split: TableFunction[_] = new MySplitUDTF()
 
 // join
diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md
index 7dce8009777..20bf49d5999 100644
--- a/docs/dev/table/udfs.md
+++ b/docs/dev/table/udfs.md
@@ -141,7 +141,7 @@ Similar to a user-defined scalar function, a user-defined 
table function takes z
 
 In order to define a table function one has to extend the base class 
`TableFunction` in `org.apache.flink.table.functions` and implement (one or 
more) evaluation methods. The behavior of a table function is determined by its 
evaluation methods. An evaluation method must be declared `public` and named 
`eval`. The `TableFunction` can be overloaded by implementing multiple methods 
named `eval`. The parameter types of the evaluation methods determine all valid 
parameters of the table function. Evaluation methods can also support variable 
arguments, such as `eval(String... strs)`. The type of the returned table is 
determined by the generic type of `TableFunction`. Evaluation methods emit 
output rows using the protected `collect(T)` method.
 
-In the Table API, a table function is used with `.join(Expression)` or 
`.leftOuterJoin(Expression)` for Scala users and `.join(String)` or 
`.leftOuterJoin(String)` for Java users. The `join` operator (cross) joins each 
row from the outer table (table on the left of the operator) with all rows 
produced by the table-valued function (which is on the right side of the 
operator). The `leftOuterJoin` operator joins each row from the outer table 
(table on the left of the operator) with all rows produced by the table-valued 
function (which is on the right side of the operator) and preserves outer rows 
for which the table function returns an empty table. In SQL use `LATERAL 
TABLE(<TableFunction>)` with CROSS JOIN and LEFT JOIN with an ON TRUE join 
condition (see examples below).
+In the Table API, a table function is used with `.join(Table)` or 
`.leftOuterJoin(Table)`. The `join` operator (cross) joins each row from the 
outer table (table on the left of the operator) with all rows produced by the 
table-valued function (which is on the right side of the operator). The 
`leftOuterJoin` operator joins each row from the outer table (table on the left 
of the operator) with all rows produced by the table-valued function (which is 
on the right side of the operator) and preserves outer rows for which the table 
function returns an empty table. In SQL use `LATERAL TABLE(<TableFunction>)` 
with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples 
below).
 
 The following example shows how to define table-valued function, register it 
in the TableEnvironment, and call it in a query. Note that you can configure 
your table function via a constructor before it is registered: 
 
@@ -171,8 +171,10 @@ Table myTable = ...         // table schema: [a: String]
 tableEnv.registerFunction("split", new Split("#"));
 
 // Use the table function in the Java Table API. "as" specifies the field 
names of the table.
-myTable.join("split(a) as (word, length)").select("a, word, length");
-myTable.leftOuterJoin("split(a) as (word, length)").select("a, word, length");
+myTable.join(new Table(tableEnv, "split(a) as (word, length)"))
+    .select("a, word, length");
+myTable.leftOuterJoin(new Table(tableEnv, "split(a) as (word, length)"))
+    .select("a, word, length");
 
 // Use the table function in SQL with LATERAL and TABLE keywords.
 // CROSS JOIN a table function (equivalent to "join" in Table API).
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
index d80ec4710d2..489b0e66a18 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
@@ -69,7 +69,7 @@ import org.apache.flink.util.Collector
   *
   *   // for Java users
   *   tEnv.registerFunction("split", new Split())   // register table function 
first
-  *   table.join("split(a) as (s)").select("a, s")
+  *   table.join(new Table(tEnv, "split(a) as (s)")).select("a, s")
   *
   *   // for SQL users
   *   tEnv.registerFunction("split", new Split())   // register table function 
first


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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
>              Labels: pull-request-available
>
> 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