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

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

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

    https://github.com/apache/flink/pull/3623#discussion_r109933183
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
 ---
    @@ -209,22 +216,23 @@ object UserDefinedFunctionUtils {
         *
         * @param name function name
         * @param tableFunction table function
    -    * @param resultType the type information of returned table
    +    * @param implicitResultType the implicit type information of returned 
table
         * @param typeFactory type factory
         * @return the TableSqlFunction
         */
       def createTableSqlFunctions(
           name: String,
           tableFunction: TableFunction[_],
    -      resultType: TypeInformation[_],
    +      implicitResultType: TypeInformation[_],
           typeFactory: FlinkTypeFactory)
         : Seq[SqlFunction] = {
    -    val (fieldNames, fieldIndexes, _) = 
UserDefinedFunctionUtils.getFieldInfo(resultType)
         val evalMethods = checkAndExtractEvalMethods(tableFunction)
     
         evalMethods.map { method =>
    -      val function = new FlinkTableFunctionImpl(resultType, fieldIndexes, 
fieldNames, method)
    -      TableSqlFunction(name, tableFunction, resultType, typeFactory, 
function)
    +      // we don't know the exact result type yet.
    +      val function = new DeferredTypeFlinkTableFunction(
    +        tableFunction, method, implicitResultType)
    --- End diff --
    
    parameters fit in line above


> Support dynamic schema in Table Function
> ----------------------------------------
>
>                 Key: FLINK-6196
>                 URL: https://issues.apache.org/jira/browse/FLINK-6196
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table API & SQL
>            Reporter: Zhuoluo Yang
>            Assignee: Zhuoluo Yang
>
> In many of our use cases. We have to decide the schema of a UDTF at the run 
> time. For example. udtf('c1, c2, c3') will generate three columns for a 
> lateral view. 
> Most systems such as calcite and hive support this feature. However, the 
> current implementation of flink didn't implement the feature correctly.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to