Class TableFunction<T>

java.lang.Object
org.apache.flink.table.functions.UserDefinedFunction
org.apache.flink.table.functions.TableFunction<T>
Type Parameters:
T - The type of the output row. Either an explicit composite type or an atomic type that is implicitly wrapped into a row consisting of one field.
All Implemented Interfaces:
Serializable, FunctionDefinition
Direct Known Subclasses:
LookupFunction, PythonTableFunction, TemporalTableFunction

@PublicEvolving public abstract class TableFunction<T> extends UserDefinedFunction
Base class for a user-defined table function. A user-defined table function maps zero, one, or multiple scalar values to zero, one, or multiple rows (or structured types). If the output record consists of only one field, the wrapper can be omitted, and a scalar value can be emitted that will be implicitly wrapped into a row by the runtime.

The behavior of a TableFunction can be defined by implementing a custom evaluation method. An evaluation method must be declared publicly, not static, and named eval. Evaluation methods can also be overloaded by implementing multiple methods named eval .

By default, input and output data types are automatically extracted using reflection. This includes the generic argument T of the class for determining an output data type. Input arguments are derived from one or more eval() methods. If the reflective information is not sufficient, it can be supported and enriched with DataTypeHint and FunctionHint annotations.

The following examples show how to specify a table function:


 // Function that accepts an arbitrary number of INT arguments and emits them as implicit ROW < INT >
 class FlattenFunction extends TableFunction<Integer> {
   public void eval(Integer... args) {
     for (Integer i : args) {
       collect(i);
     }
   }
 }

 // Function that accepts either INT or STRING and emits them as an implicit ROW < STRING >
 class DuplicatorFunction extends TableFunction<String> {
   public void eval(Integer i) {
     eval(String.valueOf(i));
   }
   public void eval(String s) {
     collect(s);
     collect(s);
   }
 }

 // Function that produces an explicit ROW < i INT, s STRING > from arguments, the function hint helps in
 // declaring the row's fields
 @FunctionHint(output = @DataTypeHint("ROW< i INT, s STRING >"))
 class DuplicatorFunction extends TableFunction<Row> {
   public void eval(Integer i, String s) {
     collect(Row.of(i, s));
     collect(Row.of(i, s));
   }
 }

 // Function that accepts either INT or DECIMAL(10, 4) and emits them as an implicit ROW < INT > or
 // ROW<DECIMAL(10, 4)> using function hints for declaring the output type
 class DuplicatorFunction extends TableFunction<Object> {
   @FunctionHint(output = @DataTypeHint("INT"))
   public void eval(Integer i) {
     collect(i);
     collect(i);
   }
   @FunctionHint(output = @DataTypeHint("DECIMAL(10, 4)"))
   public void eval(@DataTypeHint("DECIMAL(10, 4)") BigDecimal d) {
     collect(d);
     collect(d);
   }
 }
 

For storing a user-defined function in a catalog, the class must have a default constructor and must be instantiable during runtime. Anonymous functions in Table API can only be persisted if the function object is not stateful (i.e. containing only transient and static fields).

In the API, a table function can be used as follows:


 public class Split extends TableFunction<String> {

   // implement an "eval" method with as many parameters as you want
   public void eval(String str) {
     for (String s : str.split(" ")) {
       collect(s);   // use collect(...) to emit an output row
     }
   }

   // you can overload the eval method here ...
 }

 TableEnvironment tEnv = ...
 Table table = ...    // schema: ROW< a VARCHAR >

 // for Scala users
 table.joinLateral(call(classOf[Split], $"a") as ("s")).select($"a", $"s")

 // for Java users
 table.joinLateral(call(Split.class, $("a")).as("s")).select($("a"), $("s"));

 // for SQL users
 tEnv.createTemporarySystemFunction("split", Split.class); // register table function first
 tEnv.sqlQuery("SELECT a, s FROM MyTable, LATERAL TABLE(split(a)) as T(s)");
 
See Also:
  • Constructor Details

    • TableFunction

      public TableFunction()
  • Method Details

    • setCollector

      public final void setCollector(org.apache.flink.util.Collector<T> collector)
      Internal use. Sets the current collector.
    • getResultType

      @Deprecated public org.apache.flink.api.common.typeinfo.TypeInformation<T> getResultType()
      Deprecated.
      This method uses the old type system and is based on the old reflective extraction logic. The method will be removed in future versions and is only called when using the deprecated TableEnvironment.registerFunction(...) method. The new reflective extraction logic (possibly enriched with DataTypeHint and FunctionHint) should be powerful enough to cover most use cases. For advanced users, it is possible to override UserDefinedFunction.getTypeInference(DataTypeFactory).
      Returns the result type of the evaluation method.
    • getParameterTypes

      @Deprecated public org.apache.flink.api.common.typeinfo.TypeInformation<?>[] getParameterTypes(Class<?>[] signature)
      Deprecated.
      This method uses the old type system and is based on the old reflective extraction logic. The method will be removed in future versions and is only called when using the deprecated TableEnvironment.registerFunction(...) method. The new reflective extraction logic (possibly enriched with DataTypeHint and FunctionHint) should be powerful enough to cover most use cases. For advanced users, it is possible to override UserDefinedFunction.getTypeInference(DataTypeFactory).
      Returns TypeInformation about the operands of the evaluation method with a given signature.
    • collect

      protected final void collect(T row)
      Emits an (implicit or explicit) output row.

      If null is emitted as an explicit row, it will be skipped by the runtime. For implicit rows, the row's field will be null.

      Parameters:
      row - the output row
    • getKind

      public final FunctionKind getKind()
      Description copied from interface: FunctionDefinition
      Returns the kind of function this definition describes.
    • getTypeInference

      public TypeInference getTypeInference(DataTypeFactory typeFactory)
      Description copied from class: UserDefinedFunction
      Returns the logic for performing type inference of a call to this function definition.

      The type inference process is responsible for inferring unknown types of input arguments, validating input arguments, and producing result types. The type inference process happens independent of a function body. The output of the type inference is used to search for a corresponding runtime implementation.

      Instances of type inference can be created by using TypeInference.newBuilder().

      See BuiltInFunctionDefinitions for concrete usage examples.

      The type inference for user-defined functions is automatically extracted using reflection. It does this by analyzing implementation methods such as eval() or accumulate() and the generic parameters of a function class if present. If the reflective information is not sufficient, it can be supported and enriched with DataTypeHint and FunctionHint annotations.

      Note: Overriding this method is only recommended for advanced users. If a custom type inference is specified, it is the responsibility of the implementer to make sure that the output of the type inference process matches with the implementation method:

      The implementation method must comply with each DataType.getConversionClass() returned by the type inference. For example, if DataTypes.TIMESTAMP(3).bridgedTo(java.sql.Timestamp.class) is an expected argument type, the method must accept a call eval(java.sql.Timestamp).

      Regular Java calling semantics (including type widening and autoboxing) are applied when calling an implementation method which means that the signature can be eval(java.lang.Object).

      The runtime will take care of converting the data to the data format specified by the DataType.getConversionClass() coming from the type inference logic.

      Specified by:
      getTypeInference in interface FunctionDefinition
      Specified by:
      getTypeInference in class UserDefinedFunction
    • finish

      public void finish() throws Exception
      This method is called at the end of data processing. After this method is called, no more records can be produced for the downstream operators.

      NOTE:This method does not need to close any resources. You should release external resources in the UserDefinedFunction.close() method. More details can see StreamOperator#finish().

      Important:Emit record in the UserDefinedFunction.close() method is impossible since flink-1.14, if you need to emit records at the end of data processing, do so in the finish() method.

      Throws:
      Exception