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

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

dianfu commented on a change in pull request #7167: [FLINK-10973] [table] Add 
support for map to table API
URL: https://github.com/apache/flink/pull/7167#discussion_r236103842
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
 ##########
 @@ -999,6 +999,93 @@ class Table(
     new OverWindowedTable(this, overWindows.toArray)
   }
 
+  /**
+    * Performs a map operation with an user-defined function. The output will 
be flattened
+    * if the output type is composite type.
+    *
+    * Scala Example:
+    * {{{
+    *   class MyMapFunction extends ScalarFunction {
+    *     def eval(str: String): Row = {
+    *       Row.of(str, str)
+    *     }
+    *
+    *     def getResultType(signature: Array[Class[_]]): TypeInformation[_] =
+    *       Types.ROW(Types.STRING, Types.STRING)
+    *   }
+    *
+    *   val func = new MyMapFunction()
+    *   table.map(func('c))
+    * }}}
+    *
+    * Java Example:
+    * {{{
+    *   class MyMapFunction extends ScalarFunction {
+    *     public Row eval(String str) {
+    *       return Row.of(str, str);
+    *     }
+    *
+    *     public TypeInformation getResultType(Class[] signature) {
+    *       return new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO);
+    *     }
+    *   }
+    *
+    *   ScalarFunction func = new MyMapFunction();
+    *   tableEnv.registerFunction("func", func);
+    *   table.map("func(c)");
+    * }}}
+    */
+  def map(mapFunction: Expression): Table = {
 
 Review comment:
   `Expression` makes sure that it will not break backward compatible if we 
want to support `Expression*` in the future. 

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


> Add Map operator to Table API
> -----------------------------
>
>                 Key: FLINK-10973
>                 URL: https://issues.apache.org/jira/browse/FLINK-10973
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: sunjincheng
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>
> Add Map operator to Table API as described in [Google 
> doc|https://docs.google.com/document/d/1tnpxg31EQz2-MEzSotwFzqatsB4rNLz0I-l_vPa5H4Q/edit#heading=h.q23rny2iglsr]
> The usageļ¼š
> {code:java}
> val res = tab
>    .map(fun: ScalarFunction)  // output has columns 'a, 'b, 'c
>    .select('a, 'c)
> {code}



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

Reply via email to