You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/12/03 23:36:00 UTC

[jira] [Commented] (FLINK-10974) Add FlatMap to TableAPI

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

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

sunjincheng121 commented on a change in pull request #7196: [FLINK-10974] [table] Add support for flatMap to table API
URL: https://github.com/apache/flink/pull/7196#discussion_r238478347
 
 

 ##########
 File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
 ##########
 @@ -999,6 +1000,195 @@ class Table(
     new OverWindowedTable(this, overWindows.toArray)
   }
 
+  /**
+    * Performs a map operation with an user-defined scalar function or built-in scalar function.
+    * The output will be flattened if the output type is a 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 Types.ROW(Types.STRING, Types.STRING);
+    *     }
+    *   }
+    *
+    *   ScalarFunction func = new MyMapFunction();
+    *   tableEnv.registerFunction("func", func);
+    *   table.map("func(c)");
+    * }}}
+    */
+  def map(mapFunction: Expression): Table = {
+    unwrap(mapFunction, tableEnv) match {
+      case _: TableFunctionCall | _: AggFunctionCall | _: Aggregation =>
+        throw new ValidationException("Only ScalarFunction can be used in map.")
+      case _ =>
+    }
+
+    val expandedFields = expandProjectList(Seq(Flattening(mapFunction)), logicalPlan, tableEnv)
+    new Table(tableEnv,
+      Project(expandedFields.map(UnresolvedAlias), logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Performs a map operation with an user-defined scalar function or built-in scalar function.
+    * The output will be flattened if the output type is a 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)).as('a, 'b)
+    * }}}
+    *
+    * Java Example:
+    * {{{
+    *   class MyMapFunction extends ScalarFunction {
+    *     public Row eval(String str) {
+    *       return Row.of(str, str);
+    *     }
+    *
+    *     public TypeInformation getResultType(Class[] signature) {
+    *       return Types.ROW(Types.STRING, Types.STRING);
+    *     }
+    *   }
+    *
+    *   ScalarFunction func = new MyMapFunction();
+    *   tableEnv.registerFunction("func", func);
+    *   table.map("func(c)").as("a, b");
+    * }}}
+    */
+  def map(mapFunction: String): Table = {
+    map(ExpressionParser.parseExpression(mapFunction))
+  }
+
+  /**
+    * Performs a flatMap operation with an user-defined table function.
+    *
+    * Scala Example:
+    * {{{
+    *   class MyFlatMapFunction extends TableFunction[Row] {
+    *     def eval(str : String) {
+    *       if (str.contains("#")) {
+    *         val splits = str.split("#")
+    *         collect(Row.of(splits(0), splits(1)))
+    *       }
+    *     }
+    *
+    *     def getResultType(signature: Array[Class[_]]): TypeInformation[_] =
+    *       Types.ROW(Types.STRING, Types.STRING)
+    *   }
+    *
+    *   val func = new MyFlatMapFunction()
+    *   table.flatMap(func('c)).as('a, 'b)
+    * }}}
+    *
+    * Java Example:
+    * {{{
+    *   class MyFlatMapFunction extends TableFunction<Row> {
+    *     public void eval(String str) {
+    *       if (str.contains("#")) {
+    *         String[] splits = str.split("#");
+    *         collect(Row.of(splits[0], splits[1]));
+    *       }
+    *     }
+    *
+    *     public TypeInformation getResultType(Class[] signature) {
+    *       return Types.ROW(Types.STRING, Types.STRING);
+    *     }
+    *   }
+    *
+    *   TableFunction func = new MyFlatMapFunction();
+    *   tableEnv.registerFunction("func", func);
+    *   table.flatMap("func(c)").as("a, b");
+    * }}}
+    */
+  def flatMap(flatMapFunction: Expression): Table = {
+    unwrap(flatMapFunction, tableEnv) match {
+      case _: TableFunctionCall =>
+      case _ => throw new ValidationException("Only TableFunction can be used in flatMap.")
+    }
+
+    val joinNode = join(new Table(tableEnv,
+      UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, flatMapFunction))).logicalPlan
+    val outputFields = joinNode.asInstanceOf[Join].right.output
+      .map(_.name)
+      .map(UnresolvedFieldReference)
+    new Table(tableEnv,
+      Project(outputFields.map(UnresolvedAlias), joinNode).validate(tableEnv))
+  }
+
+  /**
+    * Performs a flatMap operation with an user-defined table function.
+    *
+    * Scala Example:
+    * {{{
+    *   class MyFlatMapFunction extends TableFunction[Row] {
+    *     def eval(str : String) {
+    *       if (str.contains("#")) {
+    *         val splits = str.split("#")
+    *         collect(Row.of(splits(0), splits(1)))
+    *       }
+    *     }
+    *
+    *     def getResultType(signature: Array[Class[_]]): TypeInformation[_] =
+    *       Types.ROW(Types.STRING, Types.STRING)
+    *   }
+    *
+    *   val func = new MyFlatMapFunction()
 
 Review comment:
   `MyFlatMapFunction()` -> `MyFlatMapFunction`

----------------------------------------------------------------
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:
users@infra.apache.org


> Add FlatMap to TableAPI
> -----------------------
>
>                 Key: FLINK-10974
>                 URL: https://issues.apache.org/jira/browse/FLINK-10974
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API &amp; SQL
>            Reporter: sunjincheng
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>
> Add FlatMap 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
>    .flatMap(fun: TableFunction)  // output has columns 'a, 'b, 'c
>    .select('a, 'c)
> {code}



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