Details

    • Type: Sub-task
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: None
    • Labels:
      None

      Description

      It's the second approach of FLINK-5826.
      We would like to make table functions (UDTF) of Flink support variable arguments.

        Issue Links

          Activity

          Hide
          twalthr Timo Walther added a comment -

          Fixed in 1.3.0: 04aee61d86f9ba30715c133380560739282feb81

          Show
          twalthr Timo Walther added a comment - Fixed in 1.3.0: 04aee61d86f9ba30715c133380560739282feb81
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3407

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3407
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3407

          Thanks @clarkyzl. Looks good. Will merge...

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3407 Thanks @clarkyzl. Looks good. Will merge...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3407

          @clarkyzl I merged the varargs scalar functions. Can you rebase this and the Hive UDFs?

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3407 @clarkyzl I merged the varargs scalar functions. Can you rebase this and the Hive UDFs?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user clarkyzl commented on the issue:

          https://github.com/apache/flink/pull/3407

          Hi @KurtYoung . Thanks for your reviewing. I've pushed another commit. This patch is based on https://github.com/apache/flink/pull/3389. If we can get https://github.com/apache/flink/pull/3389 merged first, this patch will look simpler. @twalthr , could you please take a look at https://github.com/apache/flink/pull/3389 ? Thank you all so much.

          Show
          githubbot ASF GitHub Bot added a comment - Github user clarkyzl commented on the issue: https://github.com/apache/flink/pull/3407 Hi @KurtYoung . Thanks for your reviewing. I've pushed another commit. This patch is based on https://github.com/apache/flink/pull/3389 . If we can get https://github.com/apache/flink/pull/3389 merged first, this patch will look simpler. @twalthr , could you please take a look at https://github.com/apache/flink/pull/3389 ? Thank you all so much.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user KurtYoung commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3407#discussion_r104598155

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala —
          @@ -45,13 +45,23 @@ class TableFunctionCallGen(
          operands: Seq[GeneratedExpression])
          : GeneratedExpression = {
          // determine function signature

          • val matchingSignature = getSignature(tableFunction, signature)
            + val matchingMethod = getEvalMethod(tableFunction, signature)
            .getOrElse(throw new CodeGenException("No matching signature found."))
            + val matchingSignature = matchingMethod.getParameterTypes
            +
            + // zip for variable signatures
            + var paramToOperands = matchingSignature.zip(operands)
            + var i = paramToOperands.length
            + while (i < operands.length
              • End diff –

          I think this while loop can be replaced with `zipAll`

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3407#discussion_r104598155 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala — @@ -45,13 +45,23 @@ class TableFunctionCallGen( operands: Seq [GeneratedExpression] ) : GeneratedExpression = { // determine function signature val matchingSignature = getSignature(tableFunction, signature) + val matchingMethod = getEvalMethod(tableFunction, signature) .getOrElse(throw new CodeGenException("No matching signature found.")) + val matchingSignature = matchingMethod.getParameterTypes + + // zip for variable signatures + var paramToOperands = matchingSignature.zip(operands) + var i = paramToOperands.length + while (i < operands.length End diff – I think this while loop can be replaced with `zipAll`
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user clarkyzl opened a pull request:

          https://github.com/apache/flink/pull/3407

          FLINK-5882 [table] TableFunction (UDTF) should support variable types and variable arguments

          Type: New Feature
          Priority: Major
          Components: table, udtf, TableFunction
          Problem Definition: TableFunction (UDTF) should support variable types and variable arguments
          Design:
          1. This pull request is based on FLINK-5881
          2. The only modification is on TableFunctionCallGen, and this part is similar with ScalarFunctionCallGen. It will make the the code generator accept the variable arguments without losing any part of it.
          3. add some tests, both stream and dataset.

          Impact Analysis: The only modification locates in TableFunctionCallGen. The orther part is based on FLINK-5881.
          Test: Unit tests done.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/clarkyzl/flink flink-5882

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3407.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3407


          commit 9af14d2be30edc5a3226ee60f218757ae029de21
          Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com>
          Date: 2017-02-22T10:53:34Z

          FLINK-5881 [table] ScalarFunction(UDF) should support variable types and variable arguments

          commit e5b85bec9a7becdfacef766660599cd1a4ea4c5d
          Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com>
          Date: 2017-02-23T11:29:37Z

          FLINK-5881 [table] Modification as reviews

          commit df4eab12d7657e971a2e6e72e0d1bd80a98ea8b0
          Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com>
          Date: 2017-02-24T08:17:50Z

          FLINK-5882 [table] TableFunction (UDTF) should support variable types and variable arguments


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user clarkyzl opened a pull request: https://github.com/apache/flink/pull/3407 FLINK-5882 [table] TableFunction (UDTF) should support variable types and variable arguments Type: New Feature Priority: Major Components: table, udtf, TableFunction Problem Definition: TableFunction (UDTF) should support variable types and variable arguments Design: 1. This pull request is based on FLINK-5881 2. The only modification is on TableFunctionCallGen, and this part is similar with ScalarFunctionCallGen. It will make the the code generator accept the variable arguments without losing any part of it. 3. add some tests, both stream and dataset. Impact Analysis: The only modification locates in TableFunctionCallGen. The orther part is based on FLINK-5881 . Test: Unit tests done. You can merge this pull request into a Git repository by running: $ git pull https://github.com/clarkyzl/flink flink-5882 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3407.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3407 commit 9af14d2be30edc5a3226ee60f218757ae029de21 Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com> Date: 2017-02-22T10:53:34Z FLINK-5881 [table] ScalarFunction(UDF) should support variable types and variable arguments commit e5b85bec9a7becdfacef766660599cd1a4ea4c5d Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com> Date: 2017-02-23T11:29:37Z FLINK-5881 [table] Modification as reviews commit df4eab12d7657e971a2e6e72e0d1bd80a98ea8b0 Author: Zhuoluo Yang <zhuoluo.yzl@alibaba-inc.com> Date: 2017-02-24T08:17:50Z FLINK-5882 [table] TableFunction (UDTF) should support variable types and variable arguments

            People

            • Assignee:
              clarkyzl Zhuoluo Yang
              Reporter:
              clarkyzl Zhuoluo Yang
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development