[ https://issues.apache.org/jira/browse/FLINK-4469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15711944#comment-15711944 ]
ASF GitHub Bot commented on FLINK-4469: --------------------------------------- Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/2653#discussion_r90052039 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TableFunctionCallGen.scala --- @@ -0,0 +1,78 @@ +/* + * 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.api.table.codegen.calls + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.codegen.CodeGenUtils._ +import org.apache.flink.api.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression} +import org.apache.flink.api.table.functions.TableFunction +import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils._ + +/** + * Generates a call to user-defined [[TableFunction]]. + * + * @param tableFunction user-defined [[TableFunction]] that might be overloaded + * @param signature actual signature with which the function is called + * @param returnType actual return type required by the surrounding + */ +class TableFunctionCallGen( + tableFunction: TableFunction[_], + signature: Seq[TypeInformation[_]], + returnType: TypeInformation[_]) + extends CallGenerator { + + override def generate( + codeGenerator: CodeGenerator, + operands: Seq[GeneratedExpression]) + : GeneratedExpression = { + // determine function signature + val matchingSignature = getSignature(tableFunction.getClass, signature) + .getOrElse(throw new CodeGenException("No matching signature found.")) + + // convert parameters for function (output boxing) + val parameters = matchingSignature + .zip(operands) + .map { case (paramClass, operandExpr) => + if (paramClass.isPrimitive) { + operandExpr + } else { + val boxedTypeTerm = boxedTypeTermForTypeInfo(operandExpr.resultType) + val boxedExpr = codeGenerator.generateOutputFieldBoxing(operandExpr) + val exprOrNull: String = if (codeGenerator.nullCheck) { + s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}" + } else { + boxedExpr.resultTerm + } + boxedExpr.copy(resultTerm = exprOrNull) + } + } + + // generate function call + val functionReference = codeGenerator.addReusableInstance(tableFunction) + val functionCallCode = + s""" + |${parameters.map(_.code).mkString("\n")} + |$functionReference.clear(); + |$functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")}); + |""".stripMargin + + // has no result + GeneratedExpression(functionReference, "false", functionCallCode, returnType) --- End diff -- Btw. I created the constant `GeneratedExpression.NEVER_NULL` instead of `"false"` to make it more readable. > Add support for user defined table function in Table API & SQL > -------------------------------------------------------------- > > Key: FLINK-4469 > URL: https://issues.apache.org/jira/browse/FLINK-4469 > Project: Flink > Issue Type: New Feature > Components: Table API & SQL > Reporter: Jark Wu > Assignee: Jark Wu > > Normal user-defined functions, such as concat(), take in a single input row > and output a single output row. In contrast, table-generating functions > transform a single input row to multiple output rows. It is very useful in > some cases, such as look up in HBase by rowkey and return one or more rows. > Adding a user defined table function should: > 1. inherit from UDTF class with specific generic type T > 2. define one or more evel function. > NOTE: > 1. the eval method must be public and non-static. > 2. the generic type T is the row type returned by table function. Because of > Java type erasure, we can’t extract T from the Iterable. > 3. use {{collect(T)}} to emit table row > 4. eval method can be overload. Blink will choose the best match eval method > to call according to parameter types and number. > {code} > public class Word { > public String word; > public Integer length; > } > public class SplitStringUDTF extends UDTF<Word> { > public Iterable<Word> eval(String str) { > if (str != null) { > for (String s : str.split(",")) { > collect(new Word(s, s.length())); > } > } > } > } > // in SQL > tableEnv.registerFunction("split", new SplitStringUDTF()) > tableEnv.sql("SELECT a, b, t.* FROM MyTable, LATERAL TABLE(split(c)) AS > t(w,l)") > // in Java Table API > tableEnv.registerFunction("split", new SplitStringUDTF()) > // rename split table columns to “w” and “l” > table.crossApply("split(c) as (w, l)") > .select("a, b, w, l") > // without renaming, we will use the origin field names in the POJO/case/... > table.crossApply("split(c)") > .select("a, b, word, length") > // in Scala Table API > val split = new SplitStringUDTF() > table.crossApply(split('c) as ('w, 'l)) > .select('a, 'b, 'w, 'l) > // outerApply for outer join to a UDTF > table.outerApply(split('c)) > .select('a, 'b, 'word, 'length) > {code} > See [1] for more information about UDTF design. > [1] > https://docs.google.com/document/d/15iVc1781dxYWm3loVQlESYvMAxEzbbuVFPZWBYuY1Ek/edit# -- This message was sent by Atlassian JIRA (v6.3.4#6332)