User-defined functions are an important feature, because they significantly extend the expressiveness of queries.

Register User-Defined Functions

In most cases, a user-defined function must be registered before it can be used in an query. It is not necessary to register functions for the Scala Table API.

Functions are registered at the TableEnvironment by calling a registerFunction() method. When a user-defined function is registered, it is inserted into the function catalog of the TableEnvironment such that the Table API or SQL parser can recognize and properly translate it.

Please find detailed examples of how to register and how to call each type of user-defined function (ScalarFunction, TableFunction, and AggregateFunction) in the following sub-sessions.

Scalar Functions

If a required scalar function is not contained in the built-in functions, it is possible to define custom, user-defined scalar functions for both the Table API and SQL. A user-defined scalar functions maps zero, one, or multiple scalar values to a new scalar value.

In order to define a scalar function one has to extend the base class ScalarFunction in org.apache.flink.table.functions and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named eval. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named eval. Evaluation methods can also support variable arguments, such as eval(String... strs).

The following example shows how to define your own hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered:

  1. public class HashCode extends ScalarFunction {
  2. private int factor = 12;
  3. public HashCode(int factor) {
  4. this.factor = factor;
  5. }
  6. public int eval(String s) {
  7. return s.hashCode() * factor;
  8. }
  9. }
  10. BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
  11. // register the function
  12. tableEnv.registerFunction("hashCode", new HashCode(10));
  13. // use the function in Java Table API
  14. myTable.select("string, string.hashCode(), hashCode(string)");
  15. // use the function in SQL API
  16. tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable");
  1. // must be defined in static/object context class HashCode(factor: Int) extends ScalarFunction {
  2. def eval(s: String): Int = {
  3. s.hashCode() * factor
  4. }
  5. }
  6. val tableEnv = TableEnvironment.getTableEnvironment(env)
  7. // use the function in Scala Table API val hashCode = new HashCode(10)
  8. myTable.select('string, hashCode('string))
  9. // register and use the function in SQL tableEnv.registerFunction("hashCode", new HashCode(10))
  10. tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable")

By default the result type of an evaluation method is determined by Flink’s type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases TypeInformation of the result type can be manually defined by overriding ScalarFunction#getResultType().

The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding ScalarFunction#getResultType() we define that the returned long value should be interpreted as a Types.TIMESTAMP by the code generation.

  1. public static class TimestampModifier extends ScalarFunction {
  2. public long eval(long t) {
  3. return t % 1000;
  4. }
  5. public TypeInformation<?> getResultType(signature: Class<?>[]) {
  6. return Types.TIMESTAMP;
  7. }
  8. }
  1. object TimestampModifier extends ScalarFunction {
  2. def eval(t: Long): Long = {
  3. t % 1000
  4. }
  5. override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
  6. Types.TIMESTAMP
  7. }
  8. }

Table Functions

Similar to a user-defined scalar function, a user-defined table function takes zero, one, or multiple scalar values as input parameters. However in contrast to a scalar function, it can return an arbitrary number of rows as output instead of a single value. The returned rows may consist of one or more columns.

In order to define a table function one has to extend the base class TableFunction in org.apache.flink.table.functions and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared public and named eval. The TableFunction can be overloaded by implementing multiple methods named eval. The parameter types of the evaluation methods determine all valid parameters of the table function. Evaluation methods can also support variable arguments, such as eval(String... strs). The type of the returned table is determined by the generic type of TableFunction. Evaluation methods emit output rows using the protected collect(T) method.

In the Table API, a table function is used with .join(Table) or .leftOuterJoin(Table). The join operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The leftOuterJoin operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use LATERAL TABLE(&lt;TableFunction&gt;) with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).

The following example shows how to define table-valued function, register it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:

  1. // The generic type "Tuple2<String, Integer>" determines the schema of the returned table as (String, Integer).
  2. public class Split extends TableFunction<Tuple2<String, Integer>> {
  3. private String separator = " ";
  4. public Split(String separator) {
  5. this.separator = separator;
  6. }
  7. public void eval(String str) {
  8. for (String s : str.split(separator)) {
  9. // use collect(...) to emit a row
  10. collect(new Tuple2<String, Integer>(s, s.length()));
  11. }
  12. }
  13. }
  14. BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
  15. Table myTable = ... // table schema: [a: String]
  16. // Register the function.
  17. tableEnv.registerFunction("split", new Split("#"));
  18. // Use the table function in the Java Table API. "as" specifies the field names of the table.
  19. myTable.join(new Table(tableEnv, "split(a) as (word, length)"))
  20. .select("a, word, length");
  21. myTable.leftOuterJoin(new Table(tableEnv, "split(a) as (word, length)"))
  22. .select("a, word, length");
  23. // Use the table function in SQL with LATERAL and TABLE keywords.
  24. // CROSS JOIN a table function (equivalent to "join" in Table API).
  25. tableEnv.sqlQuery("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)");
  26. // LEFT JOIN a table function (equivalent to "leftOuterJoin" in Table API).
  27. tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE");
  1. // The generic type "(String, Int)" determines the schema of the returned table as (String, Integer). class Split(separator: String) extends TableFunction[(String, Int)] {
  2. def eval(str: String): Unit = {
  3. // use collect(...) to emit a row.
  4. str.split(separator).foreach(x => collect((x, x.length)))
  5. }
  6. }
  7. val tableEnv = TableEnvironment.getTableEnvironment(env)
  8. val myTable = ... // table schema: [a: String]
  9. // Use the table function in the Scala Table API (Note: No registration required in Scala Table API). val split = new Split("#")
  10. // "as" specifies the field names of the generated table. myTable.join(split('a) as ('word, 'length)).select('a, 'word, 'length)
  11. myTable.leftOuterJoin(split('a) as ('word, 'length)).select('a, 'word, 'length)
  12. // Register the table function to use it in SQL queries. tableEnv.registerFunction("split", new Split("#"))
  13. // Use the table function in SQL with LATERAL and TABLE keywords.
  14. // CROSS JOIN a table function (equivalent to "join" in Table API) tableEnv.sqlQuery("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
  15. // LEFT JOIN a table function (equivalent to "leftOuterJoin" in Table API) tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")

IMPORTANT: Do not implement TableFunction as a Scala object. Scala object is a singleton and will cause concurrency issues.

Please note that POJO types do not have a deterministic field order. Therefore, you cannot rename the fields of POJO returned by a table function using AS.

By default the result type of a TableFunction is determined by Flink’s automatic type extraction facilities. This works well for basic types and simple POJOs but might be wrong for more complex, custom, or composite types. In such a case, the type of the result can be manually specified by overriding TableFunction#getResultType() which returns its TypeInformation.

The following example shows an example of a TableFunction that returns a Row type which requires explicit type information. We define that the returned table type should be RowTypeInfo(String, Integer) by overriding TableFunction#getResultType().

  1. public class CustomTypeSplit extends TableFunction<Row> {
  2. public void eval(String str) {
  3. for (String s : str.split(" ")) {
  4. Row row = new Row(2);
  5. row.setField(0, s);
  6. row.setField(1, s.length);
  7. collect(row);
  8. }
  9. }
  10. @Override
  11. public TypeInformation<Row> getResultType() {
  12. return Types.ROW(Types.STRING(), Types.INT());
  13. }
  14. }
  1. class CustomTypeSplit extends TableFunction[Row] {
  2. def eval(str: String): Unit = {
  3. str.split(" ").foreach({ s =>
  4. val row = new Row(2)
  5. row.setField(0, s)
  6. row.setField(1, s.length)
  7. collect(row)
  8. })
  9. }
  10. override def getResultType: TypeInformation[Row] = {
  11. Types.ROW(Types.STRING, Types.INT)
  12. }
  13. }

Aggregation Functions

User-Defined Aggregate Functions (UDAGGs) aggregate a table (one ore more rows with one or more attributes) to a scalar value.
The above figure shows an example of an aggregation. Assume you have a table that contains data about beverages. The table consists of three columns, id, name and price and 5 rows. Imagine you need to find the highest price of all beverages in the table, i.e., perform a max() aggregation. You would need to check each of the 5 rows and the result would be a single numeric value.

User-defined aggregation functions are implemented by extending the AggregateFunction class. An AggregateFunction works as follows. First, it needs an accumulator, which is the data structure that holds the intermediate result of the aggregation. An empty accumulator is created by calling the createAccumulator() method of the AggregateFunction. Subsequently, the accumulate() method of the function is called for each input row to update the accumulator. Once all rows have been processed, the getValue() method of the function is called to compute and return the final result.

The following methods are mandatory for each AggregateFunction:

  • createAccumulator()
  • accumulate()
  • getValue()

Flink’s type extraction facilities can fail to identify complex data types, e.g., if they are not basic types or simple POJOs. So similar to ScalarFunction and TableFunction, AggregateFunction provides methods to specify the TypeInformation of the result type (through AggregateFunction#getResultType()) and the type of the accumulator (through AggregateFunction#getAccumulatorType()).

Besides the above methods, there are a few contracted methods that can be optionally implemented. While some of these methods allow the system more efficient query execution, others are mandatory for certain use cases. For instance, the merge() method is mandatory if the aggregation function should be applied in the context of a session group window (the accumulators of two session windows need to be joined when a row is observed that “connects” them).

The following methods of AggregateFunction are required depending on the use case:

  • retract() is required for aggregations on bounded OVER windows.
  • merge() is required for many batch aggregations and session window aggregations.
  • resetAccumulator() is required for many batch aggregations.

All methods of AggregateFunction must be declared as public, not static and named exactly as the names mentioned above. The methods createAccumulator, getValue, getResultType, and getAccumulatorType are defined in the AggregateFunction abstract class, while others are contracted methods. In order to define a aggregate function, one has to extend the base class org.apache.flink.table.functions.AggregateFunction and implement one (or more) accumulate methods. The method accumulate can be overloaded with different parameter types and supports variable arguments.

Detailed documentation for all methods of AggregateFunction is given below.

  1. /**
  2. * Base class for aggregation functions.
  3. *
  4. * @param <T> the type of the aggregation result
  5. * @param <ACC> the type of the aggregation accumulator. The accumulator is used to keep the
  6. * aggregated values which are needed to compute an aggregation result.
  7. * AggregateFunction represents its state using accumulator, thereby the state of the
  8. * AggregateFunction must be put into the accumulator.
  9. */
  10. public abstract class AggregateFunction<T, ACC> extends UserDefinedFunction {
  11. /**
  12. * Creates and init the Accumulator for this [[AggregateFunction]].
  13. *
  14. * @return the accumulator with the initial value
  15. */
  16. public ACC createAccumulator(); // MANDATORY
  17. /** Processes the input values and update the provided accumulator instance. The method
  18. * accumulate can be overloaded with different custom types and arguments. An AggregateFunction
  19. * requires at least one accumulate() method.
  20. *
  21. * @param accumulator the accumulator which contains the current aggregated results
  22. * @param [user defined inputs] the input value (usually obtained from a new arrived data).
  23. */
  24. public void accumulate(ACC accumulator, [user defined inputs]); // MANDATORY
  25. /**
  26. * Retracts the input values from the accumulator instance. The current design assumes the
  27. * inputs are the values that have been previously accumulated. The method retract can be
  28. * overloaded with different custom types and arguments. This function must be implemented for
  29. * datastream bounded over aggregate.
  30. *
  31. * @param accumulator the accumulator which contains the current aggregated results
  32. * @param [user defined inputs] the input value (usually obtained from a new arrived data).
  33. */
  34. public void retract(ACC accumulator, [user defined inputs]); // OPTIONAL
  35. /**
  36. * Merges a group of accumulator instances into one accumulator instance. This function must be
  37. * implemented for datastream session window grouping aggregate and dataset grouping aggregate.
  38. *
  39. * @param accumulator the accumulator which will keep the merged aggregate results. It should
  40. * be noted that the accumulator may contain the previous aggregated
  41. * results. Therefore user should not replace or clean this instance in the
  42. * custom merge method.
  43. * @param its an [[java.lang.Iterable]] pointed to a group of accumulators that will be
  44. * merged.
  45. */
  46. public void merge(ACC accumulator, java.lang.Iterable<ACC> its); // OPTIONAL
  47. /**
  48. * Called every time when an aggregation result should be materialized.
  49. * The returned value could be either an early and incomplete result
  50. * (periodically emitted as data arrive) or the final result of the
  51. * aggregation.
  52. *
  53. * @param accumulator the accumulator which contains the current
  54. * aggregated results
  55. * @return the aggregation result
  56. */
  57. public T getValue(ACC accumulator); // MANDATORY
  58. /**
  59. * Resets the accumulator for this [[AggregateFunction]]. This function must be implemented for
  60. * dataset grouping aggregate.
  61. *
  62. * @param accumulator the accumulator which needs to be reset
  63. */
  64. public void resetAccumulator(ACC accumulator); // OPTIONAL
  65. /**
  66. * Returns true if this AggregateFunction can only be applied in an OVER window.
  67. *
  68. * @return true if the AggregateFunction requires an OVER window, false otherwise.
  69. */
  70. public Boolean requiresOver = false; // PRE-DEFINED
  71. /**
  72. * Returns the TypeInformation of the AggregateFunction's result.
  73. *
  74. * @return The TypeInformation of the AggregateFunction's result or null if the result type
  75. * should be automatically inferred.
  76. */
  77. public TypeInformation<T> getResultType = null; // PRE-DEFINED
  78. /**
  79. * Returns the TypeInformation of the AggregateFunction's accumulator.
  80. *
  81. * @return The TypeInformation of the AggregateFunction's accumulator or null if the
  82. * accumulator type should be automatically inferred.
  83. */
  84. public TypeInformation<T> getAccumulatorType = null; // PRE-DEFINED
  85. }
  1. /**
  2. * Base class for aggregation functions.
  3. *
  4. * @tparam T the type of the aggregation result
  5. * @tparam ACC the type of the aggregation accumulator. The accumulator is used to keep the
  6. * aggregated values which are needed to compute an aggregation result.
  7. * AggregateFunction represents its state using accumulator, thereby the state of the
  8. * AggregateFunction must be put into the accumulator.
  9. */
  10. abstract class AggregateFunction[T, ACC] extends UserDefinedFunction {
  11. /**
  12. * Creates and init the Accumulator for this [[AggregateFunction]].
  13. *
  14. * @return the accumulator with the initial value
  15. */
  16. def createAccumulator(): ACC // MANDATORY
  17. /**
  18. * Processes the input values and update the provided accumulator instance. The method
  19. * accumulate can be overloaded with different custom types and arguments. An AggregateFunction
  20. * requires at least one accumulate() method.
  21. *
  22. * @param accumulator the accumulator which contains the current aggregated results
  23. * @param [user defined inputs] the input value (usually obtained from a new arrived data).
  24. */
  25. def accumulate(accumulator: ACC, [user defined inputs]): Unit // MANDATORY
  26. /**
  27. * Retracts the input values from the accumulator instance. The current design assumes the
  28. * inputs are the values that have been previously accumulated. The method retract can be
  29. * overloaded with different custom types and arguments. This function must be implemented for
  30. * datastream bounded over aggregate.
  31. *
  32. * @param accumulator the accumulator which contains the current aggregated results
  33. * @param [user defined inputs] the input value (usually obtained from a new arrived data).
  34. */
  35. def retract(accumulator: ACC, [user defined inputs]): Unit // OPTIONAL
  36. /**
  37. * Merges a group of accumulator instances into one accumulator instance. This function must be
  38. * implemented for datastream session window grouping aggregate and dataset grouping aggregate.
  39. *
  40. * @param accumulator the accumulator which will keep the merged aggregate results. It should
  41. * be noted that the accumulator may contain the previous aggregated
  42. * results. Therefore user should not replace or clean this instance in the
  43. * custom merge method.
  44. * @param its an [[java.lang.Iterable]] pointed to a group of accumulators that will be
  45. * merged.
  46. */
  47. def merge(accumulator: ACC, its: java.lang.Iterable[ACC]): Unit // OPTIONAL
  48. /**
  49. * Called every time when an aggregation result should be materialized.
  50. * The returned value could be either an early and incomplete result
  51. * (periodically emitted as data arrive) or the final result of the
  52. * aggregation.
  53. *
  54. * @param accumulator the accumulator which contains the current
  55. * aggregated results
  56. * @return the aggregation result
  57. */
  58. def getValue(accumulator: ACC): T // MANDATORY
  59. h/**
  60. * Resets the accumulator for this [[AggregateFunction]]. This function must be implemented for
  61. * dataset grouping aggregate.
  62. *
  63. * @param accumulator the accumulator which needs to be reset
  64. */
  65. def resetAccumulator(accumulator: ACC): Unit // OPTIONAL
  66. /**
  67. * Returns true if this AggregateFunction can only be applied in an OVER window.
  68. *
  69. * @return true if the AggregateFunction requires an OVER window, false otherwise.
  70. */
  71. def requiresOver: Boolean = false // PRE-DEFINED
  72. /**
  73. * Returns the TypeInformation of the AggregateFunction's result.
  74. *
  75. * @return The TypeInformation of the AggregateFunction's result or null if the result type
  76. * should be automatically inferred.
  77. */
  78. def getResultType: TypeInformation[T] = null // PRE-DEFINED
  79. /**
  80. * Returns the TypeInformation of the AggregateFunction's accumulator.
  81. *
  82. * @return The TypeInformation of the AggregateFunction's accumulator or null if the
  83. * accumulator type should be automatically inferred.
  84. */
  85. def getAccumulatorType: TypeInformation[ACC] = null // PRE-DEFINED }

The following example shows how to

  • define an AggregateFunction that calculates the weighted average on a given column,
  • register the function in the TableEnvironment, and
  • use the function in a query.

To calculate an weighted average value, the accumulator needs to store the weighted sum and count of all the data that has been accumulated. In our example we define a class WeightedAvgAccum to be the accumulator. Accumulators are automatically backup-ed by Flink’s checkpointing mechanism and restored in case of a failure to ensure exactly-once semantics.

The accumulate() method of our WeightedAvg AggregateFunction has three inputs. The first one is the WeightedAvgAccum accumulator, the other two are user-defined inputs: input value ivalue and weight of the input iweight. Although the retract(), merge(), and resetAccumulator() methods are not mandatory for most aggregation types, we provide them below as examples. Please note that we used Java primitive types and defined getResultType() and getAccumulatorType() methods in the Scala example because Flink type extraction does not work very well for Scala types.

  1. /**
  2. * Accumulator for WeightedAvg.
  3. */
  4. public static class WeightedAvgAccum {
  5. public long sum = 0;
  6. public int count = 0;
  7. }
  8. /**
  9. * Weighted Average user-defined aggregate function.
  10. */
  11. public static class WeightedAvg extends AggregateFunction<Long, WeightedAvgAccum> {
  12. @Override
  13. public WeightedAvgAccum createAccumulator() {
  14. return new WeightedAvgAccum();
  15. }
  16. @Override
  17. public Long getValue(WeightedAvgAccum acc) {
  18. if (acc.count == 0) {
  19. return null;
  20. } else {
  21. return acc.sum / acc.count;
  22. }
  23. }
  24. public void accumulate(WeightedAvgAccum acc, long iValue, int iWeight) {
  25. acc.sum += iValue * iWeight;
  26. acc.count += iWeight;
  27. }
  28. public void retract(WeightedAvgAccum acc, long iValue, int iWeight) {
  29. acc.sum -= iValue * iWeight;
  30. acc.count -= iWeight;
  31. }
  32. public void merge(WeightedAvgAccum acc, Iterable<WeightedAvgAccum> it) {
  33. Iterator<WeightedAvgAccum> iter = it.iterator();
  34. while (iter.hasNext()) {
  35. WeightedAvgAccum a = iter.next();
  36. acc.count += a.count;
  37. acc.sum += a.sum;
  38. }
  39. }
  40. public void resetAccumulator(WeightedAvgAccum acc) {
  41. acc.count = 0;
  42. acc.sum = 0L;
  43. }
  44. }
  45. // register function
  46. StreamTableEnvironment tEnv = ...
  47. tEnv.registerFunction("wAvg", new WeightedAvg());
  48. // use function
  49. tEnv.sqlQuery("SELECT user, wAvg(points, level) AS avgPoints FROM userScores GROUP BY user");
  1. import java.lang.{Long => JLong, Integer => JInteger}
  2. import org.apache.flink.api.java.tuple.{Tuple1 => JTuple1}
  3. import org.apache.flink.api.java.typeutils.TupleTypeInfo
  4. import org.apache.flink.table.api.Types
  5. import org.apache.flink.table.functions.AggregateFunction
  6. /**
  7. * Accumulator for WeightedAvg.
  8. */
  9. class WeightedAvgAccum extends JTuple1[JLong, JInteger] {
  10. sum = 0L
  11. count = 0
  12. }
  13. /**
  14. * Weighted Average user-defined aggregate function.
  15. */
  16. class WeightedAvg extends AggregateFunction[JLong, CountAccumulator] {
  17. override def createAccumulator(): WeightedAvgAccum = {
  18. new WeightedAvgAccum
  19. }
  20. override def getValue(acc: WeightedAvgAccum): JLong = {
  21. if (acc.count == 0) {
  22. null
  23. } else {
  24. acc.sum / acc.count
  25. }
  26. }
  27. def accumulate(acc: WeightedAvgAccum, iValue: JLong, iWeight: JInteger): Unit = {
  28. acc.sum += iValue * iWeight
  29. acc.count += iWeight
  30. }
  31. def retract(acc: WeightedAvgAccum, iValue: JLong, iWeight: JInteger): Unit = {
  32. acc.sum -= iValue * iWeight
  33. acc.count -= iWeight
  34. }
  35. def merge(acc: WeightedAvgAccum, it: java.lang.Iterable[WeightedAvgAccum]): Unit = {
  36. val iter = it.iterator()
  37. while (iter.hasNext) {
  38. val a = iter.next()
  39. acc.count += a.count
  40. acc.sum += a.sum
  41. }
  42. }
  43. def resetAccumulator(acc: WeightedAvgAccum): Unit = {
  44. acc.count = 0
  45. acc.sum = 0L
  46. }
  47. override def getAccumulatorType: TypeInformation[WeightedAvgAccum] = {
  48. new TupleTypeInfo(classOf[WeightedAvgAccum], Types.LONG, Types.INT)
  49. }
  50. override def getResultType: TypeInformation[JLong] = Types.LONG
  51. }
  52. // register function val tEnv: StreamTableEnvironment = ???
  53. tEnv.registerFunction("wAvg", new WeightedAvg())
  54. // use function tEnv.sqlQuery("SELECT user, wAvg(points, level) AS avgPoints FROM userScores GROUP BY user")

Best Practices for Implementing UDFs

The Table API and SQL code generation internally tries to work with primitive values as much as possible. A user-defined function can introduce much overhead through object creation, casting, and (un)boxing. Therefore, it is highly recommended to declare parameters and result types as primitive types instead of their boxed classes. Types.DATE and Types.TIME can also be represented as int. Types.TIMESTAMP can be represented as long.

We recommended that user-defined functions should be written by Java instead of Scala as Scala types pose a challenge for Flink’s type extractor.

Integrating UDFs with the Runtime

Sometimes it might be necessary for a user-defined function to get global runtime information or do some setup/clean-up work before the actual work. User-defined functions provide open() and close() methods that can be overridden and provide similar functionality as the methods in RichFunction of DataSet or DataStream API.

The open() method is called once before the evaluation method. The close() method after the last call to the evaluation method.

The open() method provides a FunctionContext that contains information about the context in which user-defined functions are executed, such as the metric group, the distributed cache files, or the global job parameters.

The following information can be obtained by calling the corresponding methods of FunctionContext:

Method Description
getMetricGroup() Metric group for this parallel subtask.
getCachedFile(name) Local temporary file copy of a distributed cache file.
getJobParameter(name, defaultValue) Global job parameter value associated with given key.

The following example snippet shows how to use FunctionContext in a scalar function for accessing a global job parameter:

  1. public class HashCode extends ScalarFunction {
  2. private int factor = 0;
  3. @Override
  4. public void open(FunctionContext context) throws Exception {
  5. // access "hashcode_factor" parameter
  6. // "12" would be the default value if parameter does not exist
  7. factor = Integer.valueOf(context.getJobParameter("hashcode_factor", "12"));
  8. }
  9. public int eval(String s) {
  10. return s.hashCode() * factor;
  11. }
  12. }
  13. ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
  14. BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
  15. // set job parameter
  16. Configuration conf = new Configuration();
  17. conf.setString("hashcode_factor", "31");
  18. env.getConfig().setGlobalJobParameters(conf);
  19. // register the function
  20. tableEnv.registerFunction("hashCode", new HashCode());
  21. // use the function in Java Table API
  22. myTable.select("string, string.hashCode(), hashCode(string)");
  23. // use the function in SQL
  24. tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable");
  1. object hashCode extends ScalarFunction {
  2. var hashcode_factor = 12
  3. override def open(context: FunctionContext): Unit = {
  4. // access "hashcode_factor" parameter
  5. // "12" would be the default value if parameter does not exist
  6. hashcode_factor = context.getJobParameter("hashcode_factor", "12").toInt
  7. }
  8. def eval(s: String): Int = {
  9. s.hashCode() * hashcode_factor
  10. }
  11. }
  12. val tableEnv = TableEnvironment.getTableEnvironment(env)
  13. // use the function in Scala Table API myTable.select('string, hashCode('string))
  14. // register and use the function in SQL tableEnv.registerFunction("hashCode", hashCode)
  15. tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable")