Details

    Description

      Add FlatMap operator to Table API as described in Google doc.

      The usageļ¼š

      val res = tab
         .flatMap(fun: TableFunction)  // output has columns 'a, 'b, 'c
         .select('a, 'c)
      

      Attachments

        Issue Links

          Activity

            githubbot ASF GitHub Bot added a comment -

            dianfu opened a new pull request #7196: FLINK-10974 [table] Add support for flatMap to table API
            URL: https://github.com/apache/flink/pull/7196

              1. What is the purpose of the change

            This pull request adds the flatMap operation to the table API.

              1. Brief change log
            • Move the implicit tableFunctionCall2Table from package.scala to expressionDsl.scala
            • Add implicit in expressionDsl.scala which can convert a TableFunction to an Expression
            • Add flatMap API in table.scala
              1. Verifying this change

            This change added tests and can be verified as follows:

            • Added FlatMapTest/FlatMapITCase/FlatMapValidationTest/FlatMapStringExpressionTest
              1. Does this pull request potentially affect one of the following parts:
            • Dependencies (does it add or upgrade a dependency): (no)
            • The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
            • The serializers: (no)
            • The runtime per-record code paths (performance sensitive): (no)
            • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
            • The S3 file system connector: (no)
              1. Documentation
            • Does this pull request introduce a new feature? (yes)
            • If yes, how is the feature documented? (docs)

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

            githubbot ASF GitHub Bot added a comment - dianfu opened a new pull request #7196: FLINK-10974 [table] Add support for flatMap to table API URL: https://github.com/apache/flink/pull/7196 What is the purpose of the change This pull request adds the flatMap operation to the table API. Brief change log Move the implicit tableFunctionCall2Table from package.scala to expressionDsl.scala Add implicit in expressionDsl.scala which can convert a TableFunction to an Expression Add flatMap API in table.scala Verifying this change This change added tests and can be verified as follows: Added FlatMapTest/FlatMapITCase/FlatMapValidationTest/FlatMapStringExpressionTest Does this pull request potentially affect one of the following parts: Dependencies (does it add or upgrade a dependency): (no) The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no) The serializers: (no) The runtime per-record code paths (performance sensitive): (no) Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no) The S3 file system connector: (no) Documentation Does this pull request introduce a new feature? (yes) If yes, how is the feature documented? (docs) ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r238476606

            ##########
            File path: docs/dev/table/tableApi.md
            ##########
            @@ -1682,6 +1682,60 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov

            {% top %}

            +### Map
            +
            +Map 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.
            +
            +<div class="codetabs" markdown="1">
            +<div data-lang="java" markdown="1">
            +{% highlight java %}
            +ScalarFunction func = new MyMapFunction();
            +tableEnv.registerFunction("func", func);
            +
            +Table table = input
            + .map(func("c")).as("a, b")
            +{% endhighlight %}
            +</div>
            +
            +<div data-lang="scala" markdown="1">
            +{% highlight scala %}
            +val func: ScalarFunction = new MyMapFunction()
            +
            +val table = input
            + .map(func('c)).as('a, 'b)
            +{% endhighlight %}
            +</div>
            +</div>
            +
            +{% top %}

            +
            +### FlatMap
            +
            +FlatMap performs a flatMap operation with an user-defined table function.
            +
            +<div class="codetabs" markdown="1">
            +<div data-lang="java" markdown="1">
            +

            {% highlight java %}

            +TableFunction func = new MyFlatMapFunction();
            +tableEnv.registerFunction("func", func);
            +
            +Table table = input
            + .flatMap(func("c")).as("a, b")
            +

            {% endhighlight %}

            +</div>
            +
            +<div data-lang="scala" markdown="1">
            +

            {% highlight scala %}

            +val func: TableFunction = new MyFlatMapFunction()

            Review comment:
            `new MyFlatMapFunction()` -> `new 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

            githubbot ASF GitHub Bot added a comment - 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_r238476606 ########## File path: docs/dev/table/tableApi.md ########## @@ -1682,6 +1682,60 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov {% top %} +### Map + +Map 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. + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> +{% highlight java %} +ScalarFunction func = new MyMapFunction(); +tableEnv.registerFunction("func", func); + +Table table = input + .map(func("c")).as("a, b") +{% endhighlight %} +</div> + +<div data-lang="scala" markdown="1"> +{% highlight scala %} +val func: ScalarFunction = new MyMapFunction() + +val table = input + .map(func('c)).as('a, 'b) +{% endhighlight %} +</div> +</div> + +{% top %} + +### FlatMap + +FlatMap performs a flatMap operation with an user-defined table function. + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> + {% highlight java %} +TableFunction func = new MyFlatMapFunction(); +tableEnv.registerFunction("func", func); + +Table table = input + .flatMap(func("c")).as("a, b") + {% endhighlight %} +</div> + +<div data-lang="scala" markdown="1"> + {% highlight scala %} +val func: TableFunction = new MyFlatMapFunction() Review comment: `new MyFlatMapFunction()` -> `new 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
            githubbot ASF GitHub Bot added a comment -

            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_r238479911

            ##########
            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 = {

            Review comment:
            `flatMapFunction`->`tableFunction`? duo to we not introduce the new flatmap function interface.

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

            githubbot ASF GitHub Bot added a comment - 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_r238479911 ########## 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 = { Review comment: `flatMapFunction`->`tableFunction`? duo to we not introduce the new flatmap function interface. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r238476414

            ##########
            File path: docs/dev/table/tableApi.md
            ##########
            @@ -1682,6 +1682,60 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov

            {% top %}

            +### Map
            +
            +Map 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.
            +
            +<div class="codetabs" markdown="1">
            +<div data-lang="java" markdown="1">
            +{% highlight java %}
            +ScalarFunction func = new MyMapFunction();
            +tableEnv.registerFunction("func", func);
            +
            +Table table = input
            + .map(func("c")).as("a, b")
            +{% endhighlight %}
            +</div>
            +
            +<div data-lang="scala" markdown="1">
            +{% highlight scala %}
            +val func: ScalarFunction = new MyMapFunction()
            +
            +val table = input
            + .map(func('c)).as('a, 'b)
            +{% endhighlight %}
            +</div>
            +</div>
            +
            +{% top %}

            +
            +### FlatMap
            +
            +FlatMap performs a flatMap operation with an user-defined table function.

            Review comment:
            `with an user-defined table function` -> `with a table function`? Because we can also add built-in table functions, What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r238476414 ########## File path: docs/dev/table/tableApi.md ########## @@ -1682,6 +1682,60 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov {% top %} +### Map + +Map 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. + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> +{% highlight java %} +ScalarFunction func = new MyMapFunction(); +tableEnv.registerFunction("func", func); + +Table table = input + .map(func("c")).as("a, b") +{% endhighlight %} +</div> + +<div data-lang="scala" markdown="1"> +{% highlight scala %} +val func: ScalarFunction = new MyMapFunction() + +val table = input + .map(func('c)).as('a, 'b) +{% endhighlight %} +</div> +</div> + +{% top %} + +### FlatMap + +FlatMap performs a flatMap operation with an user-defined table function. Review comment: `with an user-defined table function` -> `with a table function`? Because we can also add built-in table functions, What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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

            githubbot ASF GitHub Bot added a comment - 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
            githubbot ASF GitHub Bot added a comment -

            dianfu commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API
            URL: https://github.com/apache/flink/pull/7196#issuecomment-443962507

            @sunjincheng121 Thanks a lot for the review. Have updated the PR accordingly.

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

            githubbot ASF GitHub Bot added a comment - dianfu commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API URL: https://github.com/apache/flink/pull/7196#issuecomment-443962507 @sunjincheng121 Thanks a lot for the review. Have updated the PR accordingly. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239273030

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -328,7 +340,9 @@ case class TableFunctionCall(

            • @return this table function call
              */
              private[flink] def as(aliasList: Option[Seq[String]]): TableFunctionCall = {
            • this.aliases = aliasList
              + if (aliasList.isDefined) {

            Review comment:
            After add the `as(fields: String)`, I think can remove this method. And then change the `UserDefinedFunctionUtils#`createLogicalFunctionCall` usage as following comment in `createLogicalFunctionCall`.

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

            githubbot ASF GitHub Bot added a comment - 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_r239273030 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -328,7 +340,9 @@ case class TableFunctionCall( @return this table function call */ private [flink] def as(aliasList: Option[Seq [String] ]): TableFunctionCall = { this.aliases = aliasList + if (aliasList.isDefined) { Review comment: After add the `as(fields: String)`, I think can remove this method. And then change the `UserDefinedFunctionUtils#`createLogicalFunctionCall` usage as following comment in `createLogicalFunctionCall`. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239271764

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -21,7 +21,7 @@ import org.apache.calcite.rel.RelNode
            import org.apache.flink.api.common.typeinfo.TypeInformation
            import org.apache.flink.api.java.operators.join.JoinType
            import org.apache.flink.table.calcite.

            {FlinkRelBuilder, FlinkTypeFactory}

            -import org.apache.flink.table.expressions.

            {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}

            +import org.apache.flink.table.expressions.

            {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, TableFunctionCall, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}

            Review comment:
            more than 10 class import, i suggestion using `._`, What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r239271764 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -21,7 +21,7 @@ import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} -import org.apache.flink.table.expressions. {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, UnresolvedAlias, UnresolvedFieldReference, WindowProperty} +import org.apache.flink.table.expressions. {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, TableFunctionCall, UnresolvedAlias, UnresolvedFieldReference, WindowProperty} Review comment: more than 10 class import, i suggestion using `._`, What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239273278

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -320,6 +320,18 @@ case class TableFunctionCall(

            override private[flink] def children: Seq[Expression] = parameters

            + def as(fields: Symbol*): TableFunctionCall =

            { + this.aliases = Some(fields.map(_.name)) + this + }

            +
            + def as(fields: String): TableFunctionCall = {
            + val fieldExprs = ExpressionParser

            Review comment:
            For Robustness I suggest add empty check, something like `if(fields.isEmpty) return this`, What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r239273278 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -320,6 +320,18 @@ case class TableFunctionCall( override private [flink] def children: Seq [Expression] = parameters + def as(fields: Symbol*): TableFunctionCall = { + this.aliases = Some(fields.map(_.name)) + this + } + + def as(fields: String): TableFunctionCall = { + val fieldExprs = ExpressionParser Review comment: For Robustness I suggest add empty check, something like `if(fields.isEmpty) return this`, What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239279239

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -320,6 +320,18 @@ case class TableFunctionCall(

            override private[flink] def children: Seq[Expression] = parameters

            + def as(fields: Symbol*): TableFunctionCall = {

            Review comment:
            Add java doc:
            /**

            • Assigns an alias for this table function's returned fields
              *
            • @param fields alias for this table function's returned fields
            • @return this table function call
              */

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

            githubbot ASF GitHub Bot added a comment - 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_r239279239 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -320,6 +320,18 @@ case class TableFunctionCall( override private [flink] def children: Seq [Expression] = parameters + def as(fields: Symbol*): TableFunctionCall = { Review comment: Add java doc: /** Assigns an alias for this table function's returned fields * @param fields alias for this table function's returned fields @return this table function call */ ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239291310

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] = mutable.HashSet()
            + logicalPlan.output.map(_.name).foreach(usedFieldNames.add)

            Review comment:
            `logicalPlan.output.map(_.name).foreach(usedFieldNames.add)` -> `logicalPlan.output.foreach(a => usedFieldNames.add(a.name))`

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

            githubbot ASF GitHub Bot added a comment - 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_r239291310 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = mutable.HashSet() + logicalPlan.output.map(_.name).foreach(usedFieldNames.add) Review comment: `logicalPlan.output.map(_.name).foreach(usedFieldNames.add)` -> `logicalPlan.output.foreach(a => usedFieldNames.add(a.name))` ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239273803

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
            ##########
            @@ -795,7 +795,7 @@ object UserDefinedFunctionUtils

            { "define table function followed by some Alias.") }
            • val functionCall: LogicalTableFunctionCall = unwrap(ExpressionParser.parseExpression(udtf))
              + val functionCall: LogicalTableFunctionCall = unwrap(udtfExpr)

            Review comment:
            If we remove `as(Option(Seq[String]))`, we should change the usage looks like as follows:
            ```
            unwrap(udtfExpr).as(alias.getOrElse(Seq()).mkString(",")).toLogicalTableFunctionCall(child = null)
            ```

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

            githubbot ASF GitHub Bot added a comment - 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_r239273803 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala ########## @@ -795,7 +795,7 @@ object UserDefinedFunctionUtils { "define table function followed by some Alias.") } val functionCall: LogicalTableFunctionCall = unwrap(ExpressionParser.parseExpression(udtf)) + val functionCall: LogicalTableFunctionCall = unwrap(udtfExpr) Review comment: If we remove `as(Option(Seq [String] ))`, we should change the usage looks like as follows: ``` unwrap(udtfExpr).as(alias.getOrElse(Seq()).mkString(",")).toLogicalTableFunctionCall(child = null) ``` ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239285343

            ##########
            File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/FlatMapITCase.scala
            ##########
            @@ -0,0 +1,72 @@
            +/*
            + * Licensed to the Apache Software Foundation (ASF) under one
            + * or more contributor license agreements. See the NOTICE file
            + * distributed with this work for additional information
            + * regarding copyright ownership. The ASF licenses this file
            + * to you under the Apache License, Version 2.0 (the
            + * "License"); you may not use this file except in compliance
            + * with the License. You may obtain a copy of the License at
            + *
            + * http://www.apache.org/licenses/LICENSE-2.0
            + *
            + * Unless required by applicable law or agreed to in writing, software
            + * distributed under the License is distributed on an "AS IS" BASIS,
            + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
            + * See the License for the specific language governing permissions and
            + * limitations under the License.
            + */
            +
            +package org.apache.flink.table.runtime.batch.table
            +
            +import org.apache.flink.api.scala._
            +import org.apache.flink.table.api.TableEnvironment
            +import org.apache.flink.table.api.scala._
            +import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase
            +import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode
            +import org.apache.flink.table.utils.TableFunc2
            +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
            +import org.apache.flink.test.util.TestBaseUtils
            +import org.junit.Test
            +import org.junit.runner.RunWith
            +import org.junit.runners.Parameterized
            +
            +import scala.collection.JavaConverters._
            +import scala.collection.mutable
            +
            +@RunWith(classOf[Parameterized])
            +class FlatMapITCase(
            + mode: TestExecutionMode,
            + configMode: TableConfigMode)
            + extends TableProgramsClusterTestBase(mode, configMode) {
            +
            + private def testData(env: ExecutionEnvironment): DataSet[(Int, Long, String)] =

            { + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "Jack#22")) + data.+=((2, 2L, "John#333")) + data.+=((3, 2L, "Anna#4444")) + data.+=((4, 3L, "nosharp#55555")) + env.fromCollection(data) + }

            +
            + @Test
            + def testFlatMap(): Unit = {
            + val env = ExecutionEnvironment.getExecutionEnvironment
            + val tEnv = TableEnvironment.getTableEnvironment(env)
            +
            + val func2 = new TableFunc2
            + val results = testData(env).toTable(tEnv, 'a, 'b, 'c)
            + .flatMap(func2('c))

            Review comment:
            Suggestions to enhance our test case as follows: (I'm fine if you want add the test in validate test not in it case)
            val ds = testData(env).toTable(tEnv, 'a, 'b, 'c)
            .flatMap(func2('c)) // test non alias
            .select('f0, 'f1)
            .flatMap(func2("Sunny#Panpan") as('a, 'b)) // test alias
            .select('a, 'b)
            // test function output name same as left table field name
            .flatMap(func2("Dian#Flink").as('a, 'b))
            .select('a, 'b)

            What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r239285343 ########## File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/FlatMapITCase.scala ########## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.batch.table + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase +import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.table.utils.TableFunc2 +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.TestBaseUtils +import org.junit.Test +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +@RunWith(classOf [Parameterized] ) +class FlatMapITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsClusterTestBase(mode, configMode) { + + private def testData(env: ExecutionEnvironment): DataSet [(Int, Long, String)] = { + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "Jack#22")) + data.+=((2, 2L, "John#333")) + data.+=((3, 2L, "Anna#4444")) + data.+=((4, 3L, "nosharp#55555")) + env.fromCollection(data) + } + + @Test + def testFlatMap(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + val func2 = new TableFunc2 + val results = testData(env).toTable(tEnv, 'a, 'b, 'c) + .flatMap(func2('c)) Review comment: Suggestions to enhance our test case as follows: (I'm fine if you want add the test in validate test not in it case) val ds = testData(env).toTable(tEnv, 'a, 'b, 'c) .flatMap(func2('c)) // test non alias .select('f0, 'f1) .flatMap(func2("Sunny#Panpan") as('a, 'b)) // test alias .select('a, 'b) // test function output name same as left table field name .flatMap(func2("Dian#Flink").as('a, 'b)) .select('a, 'b) What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239271885

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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

            githubbot ASF GitHub Bot added a comment - 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_r239271885 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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
            githubbot ASF GitHub Bot added a comment -

            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_r239285940

            ##########
            File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/FlatMapValidationTest.scala
            ##########
            @@ -0,0 +1,68 @@
            +/*
            + * Licensed to the Apache Software Foundation (ASF) under one
            + * or more contributor license agreements. See the NOTICE file
            + * distributed with this work for additional information
            + * regarding copyright ownership. The ASF licenses this file
            + * to you under the Apache License, Version 2.0 (the
            + * "License"); you may not use this file except in compliance
            + * with the License. You may obtain a copy of the License at
            + *
            + * http://www.apache.org/licenses/LICENSE-2.0
            + *
            + * Unless required by applicable law or agreed to in writing, software
            + * distributed under the License is distributed on an "AS IS" BASIS,
            + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
            + * See the License for the specific language governing permissions and
            + * limitations under the License.
            + */
            +
            +package org.apache.flink.table.api.stream.table.validation
            +
            +import org.apache.flink.api.scala._
            +import org.apache.flink.table.api.ValidationException
            +import org.apache.flink.table.api.scala._
            +import org.apache.flink.table.expressions.utils.Func15
            +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg
            +import org.apache.flink.table.utils.

            {TableFunc0, TableTestBase}

            +import org.junit.Test
            +
            +class FlatMapValidationTest extends TableTestBase {
            +
            + @Test(expected = classOf[ValidationException])
            + def testInvalidMapFunctionTypeAggregation(): Unit = {
            + val util = streamTestUtil()
            + util.addTable[(Int)](
            + "MyTable", 'int)
            + .flatMap('int.sum) // do not support AggregateFunction as input

            Review comment:
            add `.flatMap('int) ` test case ?

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

            githubbot ASF GitHub Bot added a comment - 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_r239285940 ########## File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/FlatMapValidationTest.scala ########## @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api.stream.table.validation + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.utils.Func15 +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg +import org.apache.flink.table.utils. {TableFunc0, TableTestBase} +import org.junit.Test + +class FlatMapValidationTest extends TableTestBase { + + @Test(expected = classOf [ValidationException] ) + def testInvalidMapFunctionTypeAggregation(): Unit = { + val util = streamTestUtil() + util.addTable [(Int)] ( + "MyTable", 'int) + .flatMap('int.sum) // do not support AggregateFunction as input Review comment: add `.flatMap('int) ` test case ? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239281388

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -79,7 +80,8 @@ class Table(

            • @param udtfCall A String expression of the TableFunction call.
              */
              def this(tableEnv: TableEnvironment, udtfCall: String) {
            • this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, udtfCall))
              + this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(
              + tableEnv, ExpressionParser.parseExpression(udtfCall)))

            Review comment:
            Format code as follows: ?
            ```
            this(
            tableEnv,
            UserDefinedFunctionUtils.createLogicalFunctionCall(
            tableEnv,
            ExpressionParser.parseExpression(udtfCall)
            )
            )
            ```

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

            githubbot ASF GitHub Bot added a comment - 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_r239281388 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -79,7 +80,8 @@ class Table( @param udtfCall A String expression of the TableFunction call. */ def this(tableEnv: TableEnvironment, udtfCall: String) { this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, udtfCall)) + this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall( + tableEnv, ExpressionParser.parseExpression(udtfCall))) Review comment: Format code as follows: ? ``` this( tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall( tableEnv, ExpressionParser.parseExpression(udtfCall) ) ) ``` ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239295918

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] = mutable.HashSet()
            + logicalPlan.output.map(_.name).foreach(usedFieldNames.add)
            +
            + var i: Int = 0
            + def findNewName(n: String): String = {
            + val newName = n + "_" + i
            + i += 1
            + if (usedFieldNames.contains(newName))

            { + findNewName(n) + } else { + usedFieldNames.add(newName) + newName + }
            + }
            +
            + val newOutputFieldNames = originalOutputFieldNames.map(n =>
            + if (usedFieldNames.contains) {+ findNewName(n)+ }

            else

            { + n + }

            + )

            Review comment:
            The logic of gen the new output field name can be changed as follows:
            ```
            val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map( {
            case (name, index) => if(usedFieldNames.contains(name))

            { index + "_" + name }

            else

            { name }

            }).toArray.sorted
            ```
            What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r239295918 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = mutable.HashSet() + logicalPlan.output.map(_.name).foreach(usedFieldNames.add) + + var i: Int = 0 + def findNewName(n: String): String = { + val newName = n + "_" + i + i += 1 + if (usedFieldNames.contains(newName)) { + findNewName(n) + } else { + usedFieldNames.add(newName) + newName + } + } + + val newOutputFieldNames = originalOutputFieldNames.map(n => + if (usedFieldNames.contains ) {+ findNewName(n)+ } else { + n + } + ) Review comment: The logic of gen the new output field name can be changed as follows: ``` val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map( { case (name, index) => if(usedFieldNames.contains(name)) { index + "_" + name } else { name } }).toArray.sorted ``` What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -320,6 +320,18 @@ case class TableFunctionCall(

            override private[flink] def children: Seq[Expression] = parameters

            + def as(fields: Symbol*): TableFunctionCall =

            { + this.aliases = Some(fields.map(_.name)) + this + }

            +
            + def as(fields: String): TableFunctionCall = {
            + val fieldExprs = ExpressionParser

            Review comment:
            Make sense. Done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239308474 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -320,6 +320,18 @@ case class TableFunctionCall( override private [flink] def children: Seq [Expression] = parameters + def as(fields: Symbol*): TableFunctionCall = { + this.aliases = Some(fields.map(_.name)) + this + } + + def as(fields: String): TableFunctionCall = { + val fieldExprs = ExpressionParser Review comment: Make sense. Done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/FlatMapValidationTest.scala
            ##########
            @@ -0,0 +1,68 @@
            +/*
            + * Licensed to the Apache Software Foundation (ASF) under one
            + * or more contributor license agreements. See the NOTICE file
            + * distributed with this work for additional information
            + * regarding copyright ownership. The ASF licenses this file
            + * to you under the Apache License, Version 2.0 (the
            + * "License"); you may not use this file except in compliance
            + * with the License. You may obtain a copy of the License at
            + *
            + * http://www.apache.org/licenses/LICENSE-2.0
            + *
            + * Unless required by applicable law or agreed to in writing, software
            + * distributed under the License is distributed on an "AS IS" BASIS,
            + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
            + * See the License for the specific language governing permissions and
            + * limitations under the License.
            + */
            +
            +package org.apache.flink.table.api.stream.table.validation
            +
            +import org.apache.flink.api.scala._
            +import org.apache.flink.table.api.ValidationException
            +import org.apache.flink.table.api.scala._
            +import org.apache.flink.table.expressions.utils.Func15
            +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg
            +import org.apache.flink.table.utils.

            {TableFunc0, TableTestBase}

            +import org.junit.Test
            +
            +class FlatMapValidationTest extends TableTestBase {
            +
            + @Test(expected = classOf[ValidationException])
            + def testInvalidMapFunctionTypeAggregation(): Unit = {
            + val util = streamTestUtil()
            + util.addTable[(Int)](
            + "MyTable", 'int)
            + .flatMap('int.sum) // do not support AggregateFunction as input

            Review comment:
            Done

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239308968 ########## File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/FlatMapValidationTest.scala ########## @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api.stream.table.validation + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.utils.Func15 +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg +import org.apache.flink.table.utils. {TableFunc0, TableTestBase} +import org.junit.Test + +class FlatMapValidationTest extends TableTestBase { + + @Test(expected = classOf [ValidationException] ) + def testInvalidMapFunctionTypeAggregation(): Unit = { + val util = streamTestUtil() + util.addTable [(Int)] ( + "MyTable", 'int) + .flatMap('int.sum) // do not support AggregateFunction as input Review comment: Done ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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:
            Done

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239302128 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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: Done ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
            ##########
            @@ -795,7 +795,7 @@ object UserDefinedFunctionUtils

            { "define table function followed by some Alias.") }
            • val functionCall: LogicalTableFunctionCall = unwrap(ExpressionParser.parseExpression(udtf))
              + val functionCall: LogicalTableFunctionCall = unwrap(udtfExpr)

            Review comment:
            Done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239308521 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala ########## @@ -795,7 +795,7 @@ object UserDefinedFunctionUtils { "define table function followed by some Alias.") } val functionCall: LogicalTableFunctionCall = unwrap(ExpressionParser.parseExpression(udtf)) + val functionCall: LogicalTableFunctionCall = unwrap(udtfExpr) Review comment: Done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] = mutable.HashSet()
            + logicalPlan.output.map(_.name).foreach(usedFieldNames.add)

            Review comment:
            Good suggestion, done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239302508 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = mutable.HashSet() + logicalPlan.output.map(_.name).foreach(usedFieldNames.add) Review comment: Good suggestion, done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -320,6 +320,18 @@ case class TableFunctionCall(

            override private[flink] def children: Seq[Expression] = parameters

            + def as(fields: Symbol*): TableFunctionCall = {

            Review comment:
            Done

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239307765 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -320,6 +320,18 @@ case class TableFunctionCall( override private [flink] def children: Seq [Expression] = parameters + def as(fields: Symbol*): TableFunctionCall = { Review comment: Done ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1001,131 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] = mutable.HashSet()
            + logicalPlan.output.map(_.name).foreach(usedFieldNames.add)
            +
            + var i: Int = 0
            + def findNewName(n: String): String = {
            + val newName = n + "_" + i
            + i += 1
            + if (usedFieldNames.contains(newName))

            { + findNewName(n) + } else { + usedFieldNames.add(newName) + newName + }
            + }
            +
            + val newOutputFieldNames = originalOutputFieldNames.map(n =>
            + if (usedFieldNames.contains) {+ findNewName(n)+ }

            else

            { + n + }

            + )

            Review comment:
            Good suggestion. Done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239307495 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1001,131 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = mutable.HashSet() + logicalPlan.output.map(_.name).foreach(usedFieldNames.add) + + var i: Int = 0 + def findNewName(n: String): String = { + val newName = n + "_" + i + i += 1 + if (usedFieldNames.contains(newName)) { + findNewName(n) + } else { + usedFieldNames.add(newName) + newName + } + } + + val newOutputFieldNames = originalOutputFieldNames.map(n => + if (usedFieldNames.contains ) {+ findNewName(n)+ } else { + n + } + ) Review comment: Good suggestion. Done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -79,7 +80,8 @@ class Table(

            • @param udtfCall A String expression of the TableFunction call.
              */
              def this(tableEnv: TableEnvironment, udtfCall: String) {
            • this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, udtfCall))
              + this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(
              + tableEnv, ExpressionParser.parseExpression(udtfCall)))

            Review comment:
            Done

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239302052 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -79,7 +80,8 @@ class Table( @param udtfCall A String expression of the TableFunction call. */ def this(tableEnv: TableEnvironment, udtfCall: String) { this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, udtfCall)) + this(tableEnv, UserDefinedFunctionUtils.createLogicalFunctionCall( + tableEnv, ExpressionParser.parseExpression(udtfCall))) Review comment: Done ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/FlatMapITCase.scala
            ##########
            @@ -0,0 +1,72 @@
            +/*
            + * Licensed to the Apache Software Foundation (ASF) under one
            + * or more contributor license agreements. See the NOTICE file
            + * distributed with this work for additional information
            + * regarding copyright ownership. The ASF licenses this file
            + * to you under the Apache License, Version 2.0 (the
            + * "License"); you may not use this file except in compliance
            + * with the License. You may obtain a copy of the License at
            + *
            + * http://www.apache.org/licenses/LICENSE-2.0
            + *
            + * Unless required by applicable law or agreed to in writing, software
            + * distributed under the License is distributed on an "AS IS" BASIS,
            + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
            + * See the License for the specific language governing permissions and
            + * limitations under the License.
            + */
            +
            +package org.apache.flink.table.runtime.batch.table
            +
            +import org.apache.flink.api.scala._
            +import org.apache.flink.table.api.TableEnvironment
            +import org.apache.flink.table.api.scala._
            +import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase
            +import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode
            +import org.apache.flink.table.utils.TableFunc2
            +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
            +import org.apache.flink.test.util.TestBaseUtils
            +import org.junit.Test
            +import org.junit.runner.RunWith
            +import org.junit.runners.Parameterized
            +
            +import scala.collection.JavaConverters._
            +import scala.collection.mutable
            +
            +@RunWith(classOf[Parameterized])
            +class FlatMapITCase(
            + mode: TestExecutionMode,
            + configMode: TableConfigMode)
            + extends TableProgramsClusterTestBase(mode, configMode) {
            +
            + private def testData(env: ExecutionEnvironment): DataSet[(Int, Long, String)] =

            { + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "Jack#22")) + data.+=((2, 2L, "John#333")) + data.+=((3, 2L, "Anna#4444")) + data.+=((4, 3L, "nosharp#55555")) + env.fromCollection(data) + }

            +
            + @Test
            + def testFlatMap(): Unit = {
            + val env = ExecutionEnvironment.getExecutionEnvironment
            + val tEnv = TableEnvironment.getTableEnvironment(env)
            +
            + val func2 = new TableFunc2
            + val results = testData(env).toTable(tEnv, 'a, 'b, 'c)
            + .flatMap(func2('c))

            Review comment:
            Agree. Done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239318443 ########## File path: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/FlatMapITCase.scala ########## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.batch.table + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase +import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.table.utils.TableFunc2 +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.TestBaseUtils +import org.junit.Test +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +@RunWith(classOf [Parameterized] ) +class FlatMapITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsClusterTestBase(mode, configMode) { + + private def testData(env: ExecutionEnvironment): DataSet [(Int, Long, String)] = { + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "Jack#22")) + data.+=((2, 2L, "John#333")) + data.+=((3, 2L, "Anna#4444")) + data.+=((4, 3L, "nosharp#55555")) + env.fromCollection(data) + } + + @Test + def testFlatMap(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + val func2 = new TableFunc2 + val results = testData(env).toTable(tEnv, 'a, 'b, 'c) + .flatMap(func2('c)) Review comment: Agree. Done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
            ##########
            @@ -328,7 +340,9 @@ case class TableFunctionCall(

            • @return this table function call
              */
              private[flink] def as(aliasList: Option[Seq[String]]): TableFunctionCall = {
            • this.aliases = aliasList
              + if (aliasList.isDefined) {

            Review comment:
            Agree. Done.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239308496 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala ########## @@ -328,7 +340,9 @@ case class TableFunctionCall( @return this table function call */ private [flink] def as(aliasList: Option[Seq [String] ]): TableFunctionCall = { this.aliases = aliasList + if (aliasList.isDefined) { Review comment: Agree. Done. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -21,7 +21,7 @@ import org.apache.calcite.rel.RelNode
            import org.apache.flink.api.common.typeinfo.TypeInformation
            import org.apache.flink.api.java.operators.join.JoinType
            import org.apache.flink.table.calcite.

            {FlinkRelBuilder, FlinkTypeFactory}

            -import org.apache.flink.table.expressions.

            {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}

            +import org.apache.flink.table.expressions.

            {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, TableFunctionCall, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}

            Review comment:
            Done

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239301814 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -21,7 +21,7 @@ import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} -import org.apache.flink.table.expressions. {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, UnresolvedAlias, UnresolvedFieldReference, WindowProperty} +import org.apache.flink.table.expressions. {Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, TableFunctionCall, UnresolvedAlias, UnresolvedFieldReference, WindowProperty} Review comment: Done ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239654833

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1006,121 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }
            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] =
            + logicalPlan.output.foldLeft(mutable.HashSet[String]()) { (s, o) => s += o.name }
            +
            + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) =>
            + val newName = if (usedFieldNames.contains(name)) { + index + "_" + usedFieldNames.mkString("_") + "_" + name + } else { + name + }
            + usedFieldNames.add(newName)
            + newName
            + }
            +
            + val newCall = originalCall.copy(fieldNames = newOutputFieldNames.toArray)
            + val joinNode = join(new Table(tableEnv, newCall)).logicalPlan
            + val outputFields = newCall.output.zip(originalOutputFieldNames)
            + .map(a => UnresolvedAlias(Alias(a._1, a._2)))
            + new Table(tableEnv, Project(outputFields, joinNode).validate(tableEnv))
            + }
            +
            + /**
            + * Performs a flatMap operation with a 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);

            Review comment:
            How a bout using RowTypeInfo.java, like: new RowTypeInfo( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)?

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

            githubbot ASF GitHub Bot added a comment - 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_r239654833 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1006,121 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = + logicalPlan.output.foldLeft(mutable.HashSet [String] ()) { (s, o) => s += o.name } + + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) => + val newName = if (usedFieldNames.contains(name)) { + index + "_" + usedFieldNames.mkString("_") + "_" + name + } else { + name + } + usedFieldNames.add(newName) + newName + } + + val newCall = originalCall.copy(fieldNames = newOutputFieldNames.toArray) + val joinNode = join(new Table(tableEnv, newCall)).logicalPlan + val outputFields = newCall.output.zip(originalOutputFieldNames) + .map(a => UnresolvedAlias(Alias(a._1, a._2))) + new Table(tableEnv, Project(outputFields, joinNode).validate(tableEnv)) + } + + /** + * Performs a flatMap operation with a 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); Review comment: How a bout using RowTypeInfo.java, like: new RowTypeInfo( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            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_r239653799

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1006,121 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] =
            + logicalPlan.output.foldLeft(mutable.HashSet[String]())

            { (s, o) => s += o.name }

            +
            + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) =>
            + val newName = if (usedFieldNames.contains(name)) {
            + index + "" + usedFieldNames.mkString("") + "_" + name

            Review comment:
            May be `index + "" + usedFieldNames.mkString("") + "_" + name` ->
            `index + "" + usedFieldNames.mkString("")` is enoughļ¼Œ due to `usedFieldNames` contains the `name`, but apeend a name also fine to me, What do you think?

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

            githubbot ASF GitHub Bot added a comment - 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_r239653799 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1006,121 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = + logicalPlan.output.foldLeft(mutable.HashSet [String] ()) { (s, o) => s += o.name } + + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) => + val newName = if (usedFieldNames.contains(name)) { + index + " " + usedFieldNames.mkString(" ") + "_" + name Review comment: May be `index + " " + usedFieldNames.mkString(" ") + "_" + name` -> `index + " " + usedFieldNames.mkString(" ")` is enoughļ¼Œ due to `usedFieldNames` contains the `name`, but apeend a name also fine to me, What do you think? ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

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

            ##########
            File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
            ##########
            @@ -999,6 +1006,121 @@ class Table(
            new OverWindowedTable(this, overWindows.toArray)
            }

            + /**
            + * 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(tableFunction: Expression): Table = {
            + unwrap(tableFunction, tableEnv) match

            { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + }

            +
            + // rename output fields names to avoid ambiguous name
            + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction)
            + val originalOutputFieldNames = originalCall.output.map(_.name)
            + val usedFieldNames: mutable.HashSet[String] =
            + logicalPlan.output.foldLeft(mutable.HashSet[String]())

            { (s, o) => s += o.name }

            +
            + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) =>
            + val newName = if (usedFieldNames.contains(name)) {
            + index + "" + usedFieldNames.mkString("") + "_" + name

            Review comment:
            OK. Make sense.

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

            githubbot ASF GitHub Bot added a comment - dianfu 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_r239688529 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala ########## @@ -999,6 +1006,121 @@ class Table( new OverWindowedTable(this, overWindows.toArray) } + /** + * 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(tableFunction: Expression): Table = { + unwrap(tableFunction, tableEnv) match { + case _: TableFunctionCall => + case _ => throw new ValidationException("Only TableFunction can be used in flatMap.") + } + + // rename output fields names to avoid ambiguous name + val originalCall = UserDefinedFunctionUtils.createLogicalFunctionCall(tableEnv, tableFunction) + val originalOutputFieldNames = originalCall.output.map(_.name) + val usedFieldNames: mutable.HashSet [String] = + logicalPlan.output.foldLeft(mutable.HashSet [String] ()) { (s, o) => s += o.name } + + val newOutputFieldNames = originalOutputFieldNames.zipWithIndex.map { case (name, index) => + val newName = if (usedFieldNames.contains(name)) { + index + " " + usedFieldNames.mkString(" ") + "_" + name Review comment: OK. Make sense. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            dianfu commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API
            URL: https://github.com/apache/flink/pull/7196#issuecomment-445621944

            @sunjincheng121 Thanks a lot for the review. Have updated the PR accordingly.

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

            githubbot ASF GitHub Bot added a comment - dianfu commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API URL: https://github.com/apache/flink/pull/7196#issuecomment-445621944 @sunjincheng121 Thanks a lot for the review. Have updated the PR accordingly. ---------------------------------------------------------------- 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
            githubbot ASF GitHub Bot added a comment -

            sunjincheng121 commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API
            URL: https://github.com/apache/flink/pull/7196#issuecomment-446002606

            Thanks for the update! @dianfu
            LGTM. +1 to merged

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

            githubbot ASF GitHub Bot added a comment - sunjincheng121 commented on issue #7196: FLINK-10974 [table] Add support for flatMap to table API URL: https://github.com/apache/flink/pull/7196#issuecomment-446002606 Thanks for the update! @dianfu LGTM. +1 to merged ---------------------------------------------------------------- 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
            sunjincheng121 sunjincheng added a comment -

            Fixed in master: 58e69a0f6951d99d38ba5462afbb94d5bef478fc

            sunjincheng121 sunjincheng added a comment - Fixed in master: 58e69a0f6951d99d38ba5462afbb94d5bef478fc

            People

              dian.fu Dian Fu
              sunjincheng121 sunjincheng
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 10m
                  10m