Table API

The Table API is a unified, relational API for stream and batch processing. Table API queries can be run on batch or streaming input without modifications. The Table API is a super set of the SQL language and is specially designed for working with Apache Flink. The Table API is a language-integrated API for Scala and Java. Instead of specifying queries as String values as common with SQL, Table API queries are defined in a language-embedded style in Java or Scala with IDE support like autocompletion and syntax validation.

The Table API shares many concepts and parts of its API with Flink’s SQL integration. Have a look at the Common Concepts & API to learn how to register tables or to create a Table object. The Streaming Concepts pages discuss streaming specific concepts such as dynamic tables and time attributes.

The following examples assume a registered table called Orders with attributes (a, b, c, rowtime). The rowtime field is either a logical time attribute in streaming or a regular timestamp field in batch.

Overview & Examples

The Table API is available for Scala and Java. The Scala Table API leverages on Scala expressions, the Java Table API is based on strings which are parsed and converted into equivalent expressions.

The following example shows the differences between the Scala and Java Table API. The table program is executed in a batch environment. It scans the Orders table, groups by field a, and counts the resulting rows per group. The result of the table program is converted into a DataSet of type Row and printed.

The Java Table API is enabled by importing org.apache.flink.table.api.java.*. The following example shows how a Java Table API program is constructed and how expressions are specified as strings. For the Expression DSL it is also necessary to import static org.apache.flink.table.api.Expressions.*

  1. import org.apache.flink.table.api.*
  2. import static org.apache.flink.table.api.Expressions.*
  3. // environment configuration
  4. ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
  5. BatchTableEnvironment tEnv = BatchTableEnvironment.create(env);
  6. // register Orders table in table environment
  7. // ...
  8. // specify table program
  9. Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime)
  10. Table counts = orders
  11. .groupBy($("a"))
  12. .select($("a"), $("b").count().as("cnt"));
  13. // conversion to DataSet
  14. DataSet<Row> result = tEnv.toDataSet(counts, Row.class);
  15. result.print();

The Scala Table API is enabled by importing org.apache.flink.table.api._, org.apache.flink.api.scala._, and org.apache.flink.table.api.bridge.scala._ (for bridging to/from DataStream).

The following example shows how a Scala Table API program is constructed. Table fields are referenced using Scala’s String interpolation using a dollar character ($).

  1. import org.apache.flink.api.scala._
  2. import org.apache.flink.table.api._
  3. import org.apache.flink.table.api.bridge.scala._
  4. // environment configuration
  5. val env = ExecutionEnvironment.getExecutionEnvironment
  6. val tEnv = BatchTableEnvironment.create(env)
  7. // register Orders table in table environment
  8. // ...
  9. // specify table program
  10. val orders = tEnv.from("Orders") // schema (a, b, c, rowtime)
  11. val result = orders
  12. .groupBy($"a")
  13. .select($"a", $"b".count as "cnt")
  14. .toDataSet[Row] // conversion to DataSet
  15. .print()

The Python Table API is enabled by from pyflink.table import *.

The following example shows how a Python Table API program is constructed and how expressions are specified as strings.

  1. from pyflink.table import *
  2. from pyflink.dataset import *
  3. # environment configuration
  4. env = ExecutionEnvironment.get_execution_environment()
  5. t_env = TableEnvironment.create(env, TableConfig())
  6. # register Orders table and Result table sink in table environment
  7. # ...
  8. # specify table program
  9. orders = t_env.from_path("Orders") # schema (a, b, c, rowtime)
  10. orders.group_by("a").select("a, b.count as cnt").insert_into("result")
  11. t_env.execute("python_job")

The next example shows a more complex Table API program. The program scans again the Orders table. It filters null values, normalizes the field a of type String, and calculates for each hour and product a the average billing amount b.

  1. // environment configuration
  2. // ...
  3. // specify table program
  4. Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime)
  5. Table result = orders
  6. .filter(
  7. and(
  8. $("a").isNotNull(),
  9. $("b").isNotNull(),
  10. $("c").isNotNull()
  11. ))
  12. .select($("a").lowerCase().as("a"), $("b"), $("rowtime"))
  13. .window(Tumble.over(lit(1).hours()).on($("rowtime")).as("hourlyWindow"))
  14. .groupBy($("hourlyWindow"), $("a"))
  15. .select($("a"), $("hourlyWindow").end().as("hour"), $("b").avg().as("avgBillingAmount"));
  1. // environment configuration
  2. // ...
  3. // specify table program
  4. val orders: Table = tEnv.from("Orders") // schema (a, b, c, rowtime)
  5. val result: Table = orders
  6. .filter($"a".isNotNull && $"b".isNotNull && $"c".isNotNull)
  7. .select($"a".lowerCase() as "a", $"b", $"rowtime")
  8. .window(Tumble over 1.hour on $"rowtime" as "hourlyWindow")
  9. .groupBy($"hourlyWindow", $"a")
  10. .select($"a", $"hourlyWindow".end as "hour", $"b".avg as "avgBillingAmount")
  1. # environment configuration
  2. # ...
  3. # specify table program
  4. orders = t_env.from_path("Orders") # schema (a, b, c, rowtime)
  5. result = orders.filter("a.isNotNull && b.isNotNull && c.isNotNull") \
  6. .select("a.lowerCase() as a, b, rowtime") \
  7. .window(Tumble.over("1.hour").on("rowtime").alias("hourlyWindow")) \
  8. .group_by("hourlyWindow, a") \
  9. .select("a, hourlyWindow.end as hour, b.avg as avgBillingAmount")

Since the Table API is a unified API for batch and streaming data, both example programs can be executed on batch and streaming inputs without any modification of the table program itself. In both cases, the program produces the same results given that streaming records are not late (see Streaming Concepts for details).

Back to top

Operations

The Table API supports the following operations. Please note that not all operations are available in both batch and streaming yet; they are tagged accordingly.

Scan, Projection, and Filter

OperatorsDescription
From
Batch Streaming

Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

  1. Table orders = tableEnv.from(“Orders”);
Values
Batch Streaming

Similar to the VALUES clause in a SQL query. Produces an inline table out of the provided rows.

You can use a row(...) expression to create composite rows:

  1. Table table = tEnv.fromValues(
  2. row(1, ABC”),
  3. row(2L, ABCDE”)
  4. );

will produce a Table with a schema as follows:

  1. root
  2. |— f0: BIGINT NOT NULL // original types INT and BIGINT are generalized to BIGINT
  3. |— f1: VARCHAR(5) NOT NULL // original types CHAR(3) and CHAR(5) are generalized
  4. // to VARCHAR(5). VARCHAR is used instead of CHAR so that
  5. // no padding is applied

The method will derive the types automatically from the input expressions. If types at a certain position differ, the method will try to find a common super type for all types. If a common super type does not exist, an exception will be thrown.

You can also specify the requested type explicitly. It might be helpful for assigning more generic types like e.g. DECIMAL or naming the columns.

  1. Table table = tEnv.fromValues(
  2. DataTypes.ROW(
  3. DataTypes.FIELD(“id”, DataTypes.DECIMAL(10, 2)),
  4. DataTypes.FIELD(“name”, DataTypes.STRING())
  5. ),
  6. row(1, ABC”),
  7. row(2L, ABCDE”)
  8. );

will produce a Table with a schema as follows:

  1. root
  2. |— id: DECIMAL(10, 2)
  3. |— name: STRING
Select
Batch Streaming

Similar to a SQL SELECT statement. Performs a select operation.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.select($(“a”), $(“c”).as(“d”));

You can use star () to act as a wild card, selecting all of the columns in the table.

  1. Table result = orders.select($(““));
As
Batch Streaming

Renames fields.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.as(“x, y, z, t”);
Where / Filter
Batch Streaming

Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.where($(“b”).isEqual(“red”));
or
  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.filter($(“a”).mod(2).isEqual(0));
OperatorsDescription
From
Batch Streaming

Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

  1. val orders: Table = tableEnv.from(“Orders”)
Values
Batch Streaming

Similar to the VALUES clause in a SQL query. Produces an inline table out of the provided rows.

You can use a row(...) expression to create composite rows:

  1. val table = tEnv.fromValues(
  2. row(1, ABC”),
  3. row(2L, ABCDE”)
  4. )

will produce a Table with a schema as follows:

  1. root
  2. |— f0: BIGINT NOT NULL // original types INT and BIGINT are generalized to BIGINT
  3. |— f1: VARCHAR(5) NOT NULL // original types CHAR(3) and CHAR(5) are generalized
  4. // to VARCHAR(5). VARCHAR is used instead of CHAR so that
  5. // no padding is applied

The method will derive the types automatically from the input expressions. If types at a certain position differ, the method will try to find a common super type for all types. If a common super type does not exist, an exception will be thrown.

You can also specify the requested type explicitly. It might be helpful for assigning more generic types like e.g. DECIMAL or naming the columns.

  1. val table = tEnv.fromValues(
  2. DataTypes.ROW(
  3. DataTypes.FIELD(“id”, DataTypes.DECIMAL(10, 2)),
  4. DataTypes.FIELD(“name”, DataTypes.STRING())
  5. ),
  6. row(1, ABC”),
  7. row(2L, ABCDE”)
  8. )

will produce a Table with a schema as follows:

  1. root
  2. |— id: DECIMAL(10, 2)
  3. |— name: STRING
Select
Batch Streaming

Similar to a SQL SELECT statement. Performs a select operation.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.select($a”, $c as d”)

You can use star () to act as a wild card, selecting all of the columns in the table.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.select($“)
As
Batch Streaming

Renames fields.

  1. val orders: Table = tableEnv.from(“Orders”).as(“x”, y”, z”, t”)
Where / Filter
Batch Streaming

Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.filter($a % 2 === 0)
or
  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.where($b === red”)
OperatorsDescription
Scan
Batch Streaming

Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

  1. orders = table_env.from_path(“Orders”)
Select
Batch Streaming

Similar to a SQL SELECT statement. Performs a select operation.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.select(“a, c as d”)

You can use star () to act as a wild card, selecting all of the columns in the table.

  1. result = orders.select(““)
Alias
Batch Streaming

Renames fields.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.alias(“x, y, z, t”)
Where / Filter
Batch Streaming

Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.where(“b === red’”)
or
  1. orders = table_env.from_path(“Orders”)
  2. result = orders.filter(“a % 2 === 0”)

Back to top

Column Operations

OperatorsDescription
AddColumns
Batch Streaming

Performs a field add operation. It will throw an exception if the added fields already exist.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.addColumns(concat($(“c”), sunny”));
AddOrReplaceColumns
Batch Streaming

Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.addOrReplaceColumns(concat($(“c”), sunny”).as(“desc”));
DropColumns
Batch Streaming

Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.dropColumns($(“b”), $(“c”));
RenameColumns
Batch Streaming

Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.renameColumns($(“b”).as(“b2”), $(“c”).as(“c2”));
OperatorsDescription
AddColumns
Batch Streaming

Performs a field add operation. It will throw an exception if the added fields already exist.

  1. val orders = tableEnv.from(“Orders”);
  2. val result = orders.addColumns(concat($c”, Sunny”))
AddOrReplaceColumns
Batch Streaming

Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

  1. val orders = tableEnv.from(“Orders”);
  2. val result = orders.addOrReplaceColumns(concat($c”, Sunny”) as desc”)
DropColumns
Batch Streaming

Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

  1. val orders = tableEnv.from(“Orders”);
  2. val result = orders.dropColumns($b”, $c”)
RenameColumns
Batch Streaming

Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

  1. val orders = tableEnv.from(“Orders”);
  2. val result = orders.renameColumns($b as b2”, $c as c2”)
OperatorsDescription
AddColumns
Batch Streaming

Performs a field add operation. It will throw an exception if the added fields already exist.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.add_columns(“concat(c, sunny’)”)
AddOrReplaceColumns
Batch Streaming

Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.add_or_replace_columns(“concat(c, sunny’) as desc”)
DropColumns
Batch Streaming

Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.drop_columns(“b, c”)
RenameColumns
Batch Streaming

Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.rename_columns(“b as b2, c as c2”)

Back to top

Aggregations

OperatorsDescription
GroupBy Aggregation
Batch Streaming
Result Updating

Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.groupBy($(“a”)).select($(“a”), $(“b”).sum().as(“d”));

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

GroupBy Window Aggregation
Batch Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders
  3. .window(Tumble.over(lit(5).minutes())).on($(“rowtime”)).as(“w”)) // define window
  4. .groupBy($(“a”), $(“w”)) // group by key and window
  5. // access window properties and aggregate
  6. .select(
  7. $(“a”),
  8. $(“w”).start(),
  9. $(“w”).end(),
  10. $(“w”).rowtime(),
  11. $(“b”).sum().as(“d”)
  12. );
Over Window Aggregation
Streaming

Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders
  3. // define window
  4. .window(
  5. Over
  6. .partitionBy($(“a”))
  7. .orderBy($(“rowtime”))
  8. .preceding(UNBOUNDED_RANGE)
  9. .following(CURRENT_RANGE)
  10. .as(“w”))
  11. // sliding aggregate
  12. .select(
  13. $(“a”),
  14. $(“b”).avg().over($(“w”)),
  15. $(“b”).max().over($(“w”)),
  16. $(“b”).min().over($(“w”))
  17. );

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute.

Distinct Aggregation
Batch Streaming
Result Updating

Similar to a SQL DISTINCT aggregation clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

  1. Table orders = tableEnv.from(“Orders”);
  2. // Distinct aggregation on group by
  3. Table groupByDistinctResult = orders
  4. .groupBy($(“a”))
  5. .select($(“a”), $(“b”).sum().distinct().as(“d”));
  6. // Distinct aggregation on time window group by
  7. Table groupByWindowDistinctResult = orders
  8. .window(Tumble
  9. .over(lit(5).minutes()))
  10. .on($(“rowtime”))
  11. .as(“w”)
  12. )
  13. .groupBy($(“a”), $(“w”))
  14. .select($(“a”), $(“b”).sum().distinct().as(“d”));
  15. // Distinct aggregation on over window
  16. Table result = orders
  17. .window(Over
  18. .partitionBy($(“a”))
  19. .orderBy($(“rowtime”))
  20. .preceding(UNBOUNDED_RANGE)
  21. .as(“w”))
  22. .select(
  23. $(“a”), $(“b”).avg().distinct().over($(“w”)),
  24. $(“b”).max().over($(“w”)),
  25. $(“b”).min().over($(“w”))
  26. );

User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

  1. Table orders = tEnv.from(“Orders”);
  2. // Use distinct aggregation for user-defined aggregate functions
  3. tEnv.registerFunction(“myUdagg”, new MyUdagg());
  4. orders.groupBy(“users”)
  5. .select(
  6. $(“users”),
  7. call(“myUdagg”, $(“points”)).distinct().as(“myDistinctResult”)
  8. );

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Distinct
Batch Streaming
Result Updating

Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

  1. Table orders = tableEnv.from(“Orders”);
  2. Table result = orders.distinct();

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. If state cleaning is enabled, distinct have to emit messages to prevent too early state eviction of downstream operators which makes distinct contains result updating. See Query Configuration for details.

OperatorsDescription
GroupBy Aggregation
Batch Streaming
Result Updating

Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.groupBy($a”).select($a”, $b”.sum().as(“d”))

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

GroupBy Window Aggregation
Batch Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result: Table = orders
  3. .window(Tumble over 5.minutes on $rowtime as w”) // define window
  4. .groupBy($a”, $w”) // group by key and window
  5. .select($a”, $w”.start, $w”.end, $w”.rowtime, $b”.sum as d”) // access window properties and aggregate
Over Window Aggregation
Streaming

Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result: Table = orders
  3. // define window
  4. .window(
  5. Over
  6. partitionBy $a
  7. orderBy $rowtime
  8. preceding UNBOUNDED_RANGE
  9. following CURRENT_RANGE
  10. as w”)
  11. .select($a”, $b”.avg over $w”, $b”.max().over($w”), $b”.min().over($w”)) // sliding aggregate

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute.

Distinct Aggregation
Batch Streaming
Result Updating

Similar to a SQL DISTINCT AGGREGATION clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

  1. val orders: Table = tableEnv.from(“Orders”);
  2. // Distinct aggregation on group by
  3. val groupByDistinctResult = orders
  4. .groupBy($a”)
  5. .select($a”, $b”.sum.distinct as d”)
  6. // Distinct aggregation on time window group by
  7. val groupByWindowDistinctResult = orders
  8. .window(Tumble over 5.minutes on $rowtime as w”).groupBy($a”, $w”)
  9. .select($a”, $b”.sum.distinct as d”)
  10. // Distinct aggregation on over window
  11. val result = orders
  12. .window(Over
  13. partitionBy $a
  14. orderBy $rowtime
  15. preceding UNBOUNDED_RANGE
  16. as $w”)
  17. .select($a”, $b”.avg.distinct over $w”, $b”.max over $w”, $b”.min over $w”)

User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

  1. val orders: Table = tEnv.from(“Orders”);
  2. // Use distinct aggregation for user-defined aggregate functions
  3. val myUdagg = new MyUdagg();
  4. orders.groupBy($users”).select($users”, myUdagg.distinct($points”) as myDistinctResult”);

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Distinct
Batch Streaming
Result Updating

Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. val result = orders.distinct()

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. If state cleaning is enabled, distinct have to emit messages to prevent too early state eviction of downstream operators which makes distinct contains result updating. See Query Configuration for details.

OperatorsDescription
GroupBy Aggregation
Batch Streaming
Result Updating

Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.group_by(“a”).select(“a, b.sum as d”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

GroupBy Window Aggregation
Batch Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.window(Tumble.over(“5.minutes”).on(“rowtime”).alias(“w”)) \
  3. .group_by(“a, w”) \
  4. .select(“a, w.start, w.end, w.rowtime, b.sum as d”)
Over Window Aggregation
Streaming

Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.over_window(Over.partition_by(“a”).order_by(“rowtime”)
  3. .preceding(“UNBOUNDED_RANGE”).following(“CURRENT_RANGE”)
  4. .alias(“w”)) \
  5. .select(“a, b.avg over w, b.max over w, b.min over w”)

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute.

Distinct Aggregation
Batch Streaming
Result Updating

Similar to a SQL DISTINCT aggregation clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

  1. orders = table_env.from_path(“Orders”)
  2. # Distinct aggregation on group by
  3. group_by_distinct_result = orders.group_by(“a”) \
  4. .select(“a, b.sum.distinct as d”)
  5. # Distinct aggregation on time window group by
  6. group_by_window_distinct_result = orders.window(
  7. Tumble.over(“5.minutes”).on(“rowtime”).alias(“w”)).groupBy(“a, w”) \
  8. .select(“a, b.sum.distinct as d”)
  9. # Distinct aggregation on over window
  10. result = orders.over_window(Over
  11. .partition_by(“a”)
  12. .order_by(“rowtime”)
  13. .preceding(“UNBOUNDED_RANGE”)
  14. .alias(“w”)) \
  15. .select(
  16. a, b.avg.distinct over w, b.max over w, b.min over w”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Distinct
Batch Streaming
Result Updating

Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

  1. orders = table_env.from_path(“Orders”)
  2. result = orders.distinct()

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Back to top

Joins

OperatorsDescription
Inner Join
Batch Streaming

Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, d, e, f”);
  3. Table result = left.join(right)
  4. .where($(“a”).isEqual($(“d”)))
  5. .select($(“a”), $(“b”), $(“e”));

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Outer Join
Batch Streaming Result Updating

Similar to SQL LEFT/RIGHT/FULL OUTER JOIN clauses. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, d, e, f”);
  3. Table leftOuterResult = left.leftOuterJoin(right, $(“a”).isEqual($(“d”)))
  4. .select($(“a”), $(“b”), $(“e”));
  5. Table rightOuterResult = left.rightOuterJoin(right, $(“a”).isEqual($(“d”)))
  6. .select($(“a”), $(“b”), $(“e”));
  7. Table fullOuterResult = left.fullOuterJoin(right, $(“a”).isEqual($(“d”)))
  8. .select($(“a”), $(“b”), $(“e”));

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Interval Join
Batch Streaming

Note: Interval joins are a subset of regular joins that can be processed in a streaming fashion.

An interval join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

For example, the following predicates are valid interval join conditions:

  • ltime === rtime
  • ltime >= rtime && ltime < rtime + 10.minutes
  1. Table left = tableEnv.fromDataSet(ds1, $(“a”), $(“b”), $(“c”), $(“ltime”).rowtime());
  2. Table right = tableEnv.fromDataSet(ds2, $(“d”), $(“e”), $(“f”), $(“rtime”).rowtime()));
  3. Table result = left.join(right)
  4. .where(
  5. and(
  6. $(“a”).isEqual($(“d”)),
  7. $(“ltime”).isGreaterEqual($(“rtime”).minus(lit(5).minutes())),
  8. $(“ltime”).isLess($(“rtime”).plus(lit(10).minutes()))
  9. ))
  10. .select($(“a”), $(“b”), $(“e”), $(“ltime”));
Inner Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. A row of the left (outer) table is dropped, if its table function call returns an empty result.

  1. // register User-Defined Table Function
  2. TableFunction<String> split = new MySplitUDTF();
  3. tableEnv.registerFunction(“split”, split);
  4. // join
  5. Table orders = tableEnv.from(“Orders”);
  6. Table result = orders
  7. .joinLateral(call(“split”, $(“c”)).as(“s”, t”, v”))
  8. .select($(“a”), $(“b”), $(“s”), $(“t”), $(“v”));
Left Outer Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. If a table function call returns an empty result, the corresponding outer row is preserved and the result padded with null values.

Note: Currently, the predicate of a table function left outer join can only be empty or literal true.

  1. // register User-Defined Table Function
  2. TableFunction<String> split = new MySplitUDTF();
  3. tableEnv.registerFunction(“split”, split);
  4. // join
  5. Table orders = tableEnv.from(“Orders”);
  6. Table result = orders
  7. .leftOuterJoinLateral(call(“split”, $(“c”)).as(“s”, t”, v”))
  8. .select($(“a”), $(“b”), $(“s”), $(“t”), $(“v”));
Join with Temporal Table
Streaming

Temporal tables are tables that track changes over time.

A temporal table function provides access to the state of a temporal table at a specific point in time. The syntax to join a table with a temporal table function is the same as in Inner Join with Table Function.

Currently only inner joins with temporal tables are supported.

  1. Table ratesHistory = tableEnv.from(“RatesHistory”);
  2. // register temporal table function with a time attribute and primary key
  3. TemporalTableFunction rates = ratesHistory.createTemporalTableFunction(
  4. r_proctime”,
  5. r_currency”);
  6. tableEnv.registerFunction(“rates”, rates);
  7. // join with “Orders” based on the time attribute and key
  8. Table orders = tableEnv.from(“Orders”);
  9. Table result = orders
  10. .joinLateral(call(“rates”, $(“o_proctime”)), $(“o_currency”).isEqual($(“r_currency”)))

For more information please check the more detailed temporal tables concept description.

OperatorsDescription
Inner Join
Batch Streaming

Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $d”, $e”, $f”)
  3. val result = left.join(right).where($a === $d”).select($a”, $b”, $e”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Outer Join
Batch Streaming Result Updating

Similar to SQL LEFT/RIGHT/FULL OUTER JOIN clauses. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined.

  1. val left = tableEnv.fromDataSet(ds1, $a”, $b”, $c”)
  2. val right = tableEnv.fromDataSet(ds2, $d”, $e”, $f”)
  3. val leftOuterResult = left.leftOuterJoin(right, $a === $d”).select($a”, $b”, $e”)
  4. val rightOuterResult = left.rightOuterJoin(right, $a === $d”).select($a”, $b”, $e”)
  5. val fullOuterResult = left.fullOuterJoin(right, $a === $d”).select($a”, $b”, $e”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Interval Join
Batch Streaming

Note: Interval joins are a subset of regular joins that can be processed in a streaming fashion.

An interval join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

For example, the following predicates are valid interval join conditions:

  • $”ltime” === $”rtime”
  • $”ltime” >= $”rtime” && $”ltime” < $”rtime” + 10.minutes
  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”, $ltime”.rowtime)
  2. val right = ds2.toTable(tableEnv, $d”, $e”, $f”, $rtime”.rowtime)
  3. val result = left.join(right)
  4. .where($a === $d && $ltime >= $rtime - 5.minutes && $ltime < $rtime + 10.minutes)
  5. .select($a”, $b”, $e”, $ltime”)
Inner Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. A row of the left (outer) table is dropped, if its table function call returns an empty result.

  1. // instantiate User-Defined Table Function
  2. val split: TableFunction[] = new MySplitUDTF()
  3. // join
  4. val result: Table = table
  5. .joinLateral(split($c”) as (“s”, t”, v”))
  6. .select($a”, $b”, $s”, $t”, $v”)
Left Outer Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. If a table function call returns an empty result, the corresponding outer row is preserved and the result padded with null values.

Note: Currently, the predicate of a table function left outer join can only be empty or literal true.

  1. // instantiate User-Defined Table Function
  2. val split: TableFunction[] = new MySplitUDTF()
  3. // join
  4. val result: Table = table
  5. .leftOuterJoinLateral(split($c”) as (“s”, t”, v”))
  6. .select($a”, $b”, $s”, $t”, $v”)
Join with Temporal Table
Streaming

Temporal tables are tables that track their changes over time.

A temporal table function provides access to the state of a temporal table at a specific point in time. The syntax to join a table with a temporal table function is the same as in Inner Join with Table Function.

Currently only inner joins with temporal tables are supported.

  1. val ratesHistory = tableEnv.from(“RatesHistory”)
  2. // register temporal table function with a time attribute and primary key
  3. val rates = ratesHistory.createTemporalTableFunction($r_proctime”, $r_currency”)
  4. // join with “Orders” based on the time attribute and key
  5. val orders = tableEnv.from(“Orders”)
  6. val result = orders
  7. .joinLateral(rates($o_rowtime”), $r_currency === $o_currency”)

For more information please check the more detailed temporal tables concept description.

OperatorsDescription
Inner Join
Batch Streaming

Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“d, e, f”)
  3. result = left.join(right).where(“a = d”).select(“a, b, e”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Outer Join
Batch Streaming Result Updating

Similar to SQL LEFT/RIGHT/FULL OUTER JOIN clauses. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“d, e, f”)
  3. left_outer_result = left.left_outer_join(right, a = d”).select(“a, b, e”)
  4. right_outer_result = left.right_outer_join(right, a = d”).select(“a, b, e”)
  5. full_outer_result = left.full_outer_join(right, a = d”).select(“a, b, e”)

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Interval Join
Batch Streaming

Note: Interval joins are a subset of regular joins that can be processed in a streaming fashion.

An interval join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

For example, the following predicates are valid interval join conditions:

  • ltime = rtime
  • ltime >= rtime && ltime < rtime + 2.second
  1. left = table_env.from_path(“Source1”).select(“a, b, c, rowtime1”)
  2. right = table_env.from_path(“Source2”).select(“d, e, f, rowtime2”)
  3. result = left.join(right).where(“a = d && rowtime1 >= rowtime2 - 1.second
  4. && rowtime1 <= rowtime2 + 2.second”).select(“a, b, e, rowtime1”)
Inner Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. A row of the left (outer) table is dropped, if its table function call returns an empty result.

  1. # register Java User-Defined Table Function
  2. table_env.register_java_function(“split”, com.my.udf.MySplitUDTF”)
  3. # join
  4. orders = table_env.from_path(“Orders”)
  5. result = orders.join_lateral(“split(c).as(s, t, v)”).select(“a, b, s, t, v”)
Left Outer Join with Table Function (UDTF)
Batch Streaming

Joins a table with the results of a table function. Each row of the left (outer) table is joined with all rows produced by the corresponding call of the table function. If a table function call returns an empty result, the corresponding outer row is preserved and the result padded with null values.

Note: Currently, the predicate of a table function left outer join can only be empty or literal true.

  1. # register Java User-Defined Table Function
  2. table_env.register_java_function(“split”, com.my.udf.MySplitUDTF”)
  3. # join
  4. orders = table_env.from_path(“Orders”)
  5. result = orders.left_outer_join_lateral(“split(c).as(s, t, v)”).select(“a, b, s, t, v”)
Join with Temporal Table
Streaming

Currently not supported in python API.

Back to top

Set Operations

OperatorsDescription
Union
Batch

Similar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, a, b, c”);
  3. Table result = left.union(right);
UnionAll
Batch Streaming

Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, a, b, c”);
  3. Table result = left.unionAll(right);
Intersect
Batch

Similar to a SQL INTERSECT clause. Intersect returns records that exist in both tables. If a record is present one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, d, e, f”);
  3. Table result = left.intersect(right);
IntersectAll
Batch

Similar to a SQL INTERSECT ALL clause. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, d, e, f”);
  3. Table result = left.intersectAll(right);
Minus
Batch

Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, a, b, c”);
  3. Table result = left.minus(right);
MinusAll
Batch

Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.

  1. Table left = tableEnv.fromDataSet(ds1, a, b, c”);
  2. Table right = tableEnv.fromDataSet(ds2, a, b, c”);
  3. Table result = left.minusAll(right);
In
Batch Streaming

Similar to a SQL IN clause. In returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

  1. Table left = ds1.toTable(tableEnv, a, b, c”);
  2. Table right = ds2.toTable(tableEnv, a”);
  3. Table result = left.select($(“a”), $(“b”), $(“c”)).where($(“a”).in(right));

Note: For streaming queries the operation is rewritten in a join and group operation. The required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

OperatorsDescription
Union
Batch

Similar to a SQL UNION clause. Unions two tables with duplicate records removed, both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $a”, $b”, $c”)
  3. val result = left.union(right)
UnionAll
Batch Streaming

Similar to a SQL UNION ALL clause. Unions two tables, both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $a”, $b”, $c”)
  3. val result = left.unionAll(right)
Intersect
Batch

Similar to a SQL INTERSECT clause. Intersect returns records that exist in both tables. If a record is present in one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $e”, $f”, $g”)
  3. val result = left.intersect(right)
IntersectAll
Batch

Similar to a SQL INTERSECT ALL clause. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $e”, $f”, $g”)
  3. val result = left.intersectAll(right)
Minus
Batch

Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $a”, $b”, $c”)
  3. val result = left.minus(right)
MinusAll
Batch

Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $a”, $b”, $c”)
  3. val result = left.minusAll(right)
In
Batch Streaming

Similar to a SQL IN clause. In returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

  1. val left = ds1.toTable(tableEnv, $a”, $b”, $c”)
  2. val right = ds2.toTable(tableEnv, $a”)
  3. val result = left.select($a”, $b”, $c”).where($a”.in(right))

Note: For streaming queries the operation is rewritten in a join and group operation. The required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

OperatorsDescription
Union
Batch

Similar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.union(right)
UnionAll
Batch Streaming

Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.union_all(right)
Intersect
Batch

Similar to a SQL INTERSECT clause. Intersect returns records that exist in both tables. If a record is present one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.intersect(right)
IntersectAll
Batch

Similar to a SQL INTERSECT ALL clause. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.intersect_all(right)
Minus
Batch

Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.minus(right);
MinusAll
Batch

Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a, b, c”)
  3. result = left.minus_all(right)
In
Batch Streaming

Similar to a SQL IN clause. In returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

  1. left = table_env.from_path(“Source1”).select(“a, b, c”)
  2. right = table_env.from_path(“Source2”).select(“a”)
  3. # using implicit registration
  4. result = left.select(“a, b, c”).where(“a.in(%s)” % right)
  5. # using explicit registration
  6. table_env.register_table(“RightTable”, right)
  7. result = left.select(“a, b, c”).where(“a.in(RightTable)”)

Note: For streaming queries the operation is rewritten in a join and group operation. The required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

Back to top

OrderBy, Offset & Fetch

OperatorsDescription
Order By
Batch

Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.

  1. Table in = tableEnv.fromDataSet(ds, a, b, c”);
  2. Table result = in.orderBy($(“a”).asc()”);
Offset & Fetch
Batch

Similar to the SQL OFFSET and FETCH clauses. Offset and Fetch limit the number of records returned from a sorted result. Offset and Fetch are technically part of the Order By operator and thus must be preceded by it.

  1. Table in = tableEnv.fromDataSet(ds, a, b, c”);
  2. // returns the first 5 records from the sorted result
  3. Table result1 = in.orderBy($(“a”).asc()).fetch(5);
  4. // skips the first 3 records and returns all following records from the sorted result
  5. Table result2 = in.orderBy($(“a”).asc()).offset(3);
  6. // skips the first 10 records and returns the next 5 records from the sorted result
  7. Table result3 = in.orderBy($(“a”).asc()).offset(10).fetch(5);
OperatorsDescription
Order By
Batch

Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.

  1. val in = ds.toTable(tableEnv, $a”, $b”, $c”)
  2. val result = in.orderBy($a”.asc)
Offset & Fetch
Batch

Similar to the SQL OFFSET and FETCH clauses. Offset and Fetch limit the number of records returned from a sorted result. Offset and Fetch are technically part of the Order By operator and thus must be preceded by it.

  1. val in = ds.toTable(tableEnv, $a”, $b”, $c”)
  2. // returns the first 5 records from the sorted result
  3. val result1: Table = in.orderBy($a”.asc).fetch(5)
  4. // skips the first 3 records and returns all following records from the sorted result
  5. val result2: Table = in.orderBy($a”.asc).offset(3)
  6. // skips the first 10 records and returns the next 5 records from the sorted result
  7. val result3: Table = in.orderBy($a”.asc).offset(10).fetch(5)
OperatorsDescription
Order By
Batch

Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.

  1. in = table_env.from_path(“Source1”).select(“a, b, c”)
  2. result = in.order_by(“a.asc”)
Offset & Fetch
Batch

Similar to the SQL OFFSET and FETCH clauses. Offset and Fetch limit the number of records returned from a sorted result. Offset and Fetch are technically part of the Order By operator and thus must be preceded by it.

  1. in = table_env.from_path(“Source1”).select(“a, b, c”)
  2. # returns the first 5 records from the sorted result
  3. result1 = in.order_by(“a.asc”).fetch(5)
  4. # skips the first 3 records and returns all following records from the sorted result
  5. result2 = in.order_by(“a.asc”).offset(3)
  6. # skips the first 10 records and returns the next 5 records from the sorted result
  7. result3 = in.order_by(“a.asc”).offset(10).fetch(5)

Insert

OperatorsDescription
Insert Into
Batch Streaming

Similar to the INSERT INTO clause in a SQL query, the method performs an insertion into a registered output table. The executeInsert() method will immediately submit a Flink job which execute the insert operation.

Output tables must be registered in the TableEnvironment (see Connector tables). Moreover, the schema of the registered table must match the schema of the query.

  1. Table orders = tableEnv.from(“Orders”);
  2. orders.executeInsert(“OutOrders”);
OperatorsDescription
Insert Into
Batch Streaming

Similar to the INSERT INTO clause in a SQL query, the method performs an insertion into a registered output table. The executeInsert() method will immediately submit a Flink job which execute the insert operation.

Output tables must be registered in the TableEnvironment (see Connector tables). Moreover, the schema of the registered table must match the schema of the query.

  1. val orders: Table = tableEnv.from(“Orders”)
  2. orders.executeInsert(“OutOrders”)
OperatorsDescription
Insert Into
Batch Streaming

Similar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table. The executeInsert method will immediately submit a flink job which execute the insert operation.

Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

  1. orders = table_env.from_path(“Orders”)
  2. orders.execute_insert(“OutOrders”)

Back to top

Group Windows

Group window aggregates group rows into finite groups based on time or row-count intervals and evaluate aggregation functions once per group. For batch tables, windows are a convenient shortcut to group records by time intervals.

Windows are defined using the window(w: GroupWindow) clause and require an alias, which is specified using the as clause. In order to group a table by a window, the window alias must be referenced in the groupBy(...) clause like a regular grouping attribute. The following example shows how to define a window aggregation on a table.

  1. Table table = input
  2. .window([GroupWindow w].as("w")) // define window with alias w
  3. .groupBy($("w")) // group the table by window w
  4. .select($("b").sum()); // aggregate
  1. val table = input
  2. .window([w: GroupWindow] as $"w") // define window with alias w
  3. .groupBy($"w") // group the table by window w
  4. .select($"b".sum) // aggregate
  1. # define window with alias w, group the table by window w, then aggregate
  2. table = input.window([GroupWindow w].alias("w")) \
  3. .group_by("w").select("b.sum")

In streaming environments, window aggregates can only be computed in parallel if they group on one or more attributes in addition to the window, i.e., the groupBy(...) clause references a window alias and at least one additional attribute. A groupBy(...) clause that only references a window alias (such as in the example above) can only be evaluated by a single, non-parallel task. The following example shows how to define a window aggregation with additional grouping attributes.

  1. Table table = input
  2. .window([GroupWindow w].as("w")) // define window with alias w
  3. .groupBy($("w"), $("a")) // group the table by attribute a and window w
  4. .select($("a"), $("b").sum()); // aggregate
  1. val table = input
  2. .window([w: GroupWindow] as $"w") // define window with alias w
  3. .groupBy($"w", $"a") // group the table by attribute a and window w
  4. .select($"a", $"b".sum) // aggregate
  1. # define window with alias w, group the table by attribute a and window w,
  2. # then aggregate
  3. table = input.window([GroupWindow w].alias("w")) \
  4. .group_by("w, a").select("b.sum")

Window properties such as the start, end, or rowtime timestamp of a time window can be added in the select statement as a property of the window alias as w.start, w.end, and w.rowtime, respectively. The window start and rowtime timestamps are the inclusive lower and upper window boundaries. In contrast, the window end timestamp is the exclusive upper window boundary. For example a tumbling window of 30 minutes that starts at 2pm would have 14:00:00.000 as start timestamp, 14:29:59.999 as rowtime timestamp, and 14:30:00.000 as end timestamp.

  1. Table table = input
  2. .window([GroupWindow w].as("w")) // define window with alias w
  3. .groupBy($("w"), $("a")) // group the table by attribute a and window w
  4. .select($("a"), $("w").start(), $("w").end(), $("w").rowtime(), $("b").count()); // aggregate and add window start, end, and rowtime timestamps
  1. val table = input
  2. .window([w: GroupWindow] as $"w") // define window with alias w
  3. .groupBy($"w", $"a") // group the table by attribute a and window w
  4. .select($"a", $"w".start, $"w".end, $"w".rowtime, $"b".count) // aggregate and add window start, end, and rowtime timestamps
  1. # define window with alias w, group the table by attribute a and window w,
  2. # then aggregate and add window start, end, and rowtime timestamps
  3. table = input.window([GroupWindow w].alias("w")) \
  4. .group_by("w, a") \
  5. .select("a, w.start, w.end, w.rowtime, b.count")

The Window parameter defines how rows are mapped to windows. Window is not an interface that users can implement. Instead, the Table API provides a set of predefined Window classes with specific semantics, which are translated into underlying DataStream or DataSet operations. The supported window definitions are listed below.

Tumble (Tumbling Windows)

A tumbling window assigns rows to non-overlapping, continuous windows of fixed length. For example, a tumbling window of 5 minutes groups rows in 5 minutes intervals. Tumbling windows can be defined on event-time, processing-time, or on a row-count.

Tumbling windows are defined by using the Tumble class as follows:

MethodDescription
overDefines the length the window, either as time or row-count interval.
onThe time attribute to group (time interval) or sort (row count) on. For batch queries this might be any Long or Timestamp attribute. For streaming queries this must be a declared event-time or processing-time time attribute.
asAssigns an alias to the window. The alias is used to reference the window in the following groupBy() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause.
  1. // Tumbling Event-time Window
  2. .window(Tumble.over(lit(10).minutes()).on($("rowtime")).as("w"));
  3. // Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Tumble.over(lit(10).minutes()).on($("proctime")).as("w"));
  5. // Tumbling Row-count Window (assuming a processing-time attribute "proctime")
  6. .window(Tumble.over(rowInterval(10)).on($("proctime")).as("w"));
  1. // Tumbling Event-time Window
  2. .window(Tumble over 10.minutes on $"rowtime" as $"w")
  3. // Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Tumble over 10.minutes on $"proctime" as $"w")
  5. // Tumbling Row-count Window (assuming a processing-time attribute "proctime")
  6. .window(Tumble over 10.rows on $"proctime" as $"w")
  1. # Tumbling Event-time Window
  2. .window(Tumble.over("10.minutes").on("rowtime").alias("w"))
  3. # Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Tumble.over("10.minutes").on("proctime").alias("w"))
  5. # Tumbling Row-count Window (assuming a processing-time attribute "proctime")
  6. .window(Tumble.over("10.rows").on("proctime").alias("w"));

Slide (Sliding Windows)

A sliding window has a fixed size and slides by a specified slide interval. If the slide interval is smaller than the window size, sliding windows are overlapping. Thus, rows can be assigned to multiple windows. For example, a sliding window of 15 minutes size and 5 minute slide interval assigns each row to 3 different windows of 15 minute size, which are evaluated in an interval of 5 minutes. Sliding windows can be defined on event-time, processing-time, or on a row-count.

Sliding windows are defined by using the Slide class as follows:

MethodDescription
overDefines the length of the window, either as time or row-count interval.
everyDefines the slide interval, either as time or row-count interval. The slide interval must be of the same type as the size interval.
onThe time attribute to group (time interval) or sort (row count) on. For batch queries this might be any Long or Timestamp attribute. For streaming queries this must be a declared event-time or processing-time time attribute.
asAssigns an alias to the window. The alias is used to reference the window in the following groupBy() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause.
  1. // Sliding Event-time Window
  2. .window(Slide.over(lit(10).minutes())
  3. .every(lit(5).minutes())
  4. .on($("rowtime"))
  5. .as("w"));
  6. // Sliding Processing-time window (assuming a processing-time attribute "proctime")
  7. .window(Slide.over(lit(10).minutes())
  8. .every(lit(5).minutes())
  9. .on($("proctime"))
  10. .as("w"));
  11. // Sliding Row-count window (assuming a processing-time attribute "proctime")
  12. .window(Slide.over(rowInterval(10)).every(rowInterval(5)).on($("proctime")).as("w"));
  1. // Sliding Event-time Window
  2. .window(Slide over 10.minutes every 5.minutes on $"rowtime" as $"w")
  3. // Sliding Processing-time window (assuming a processing-time attribute "proctime")
  4. .window(Slide over 10.minutes every 5.minutes on $"proctime" as $"w")
  5. // Sliding Row-count window (assuming a processing-time attribute "proctime")
  6. .window(Slide over 10.rows every 5.rows on $"proctime" as $"w")
  1. # Sliding Event-time Window
  2. .window(Slide.over("10.minutes").every("5.minutes").on("rowtime").alias("w"))
  3. # Sliding Processing-time window (assuming a processing-time attribute "proctime")
  4. .window(Slide.over("10.minutes").every("5.minutes").on("proctime").alias("w"))
  5. # Sliding Row-count window (assuming a processing-time attribute "proctime")
  6. .window(Slide.over("10.rows").every("5.rows").on("proctime").alias("w"))

Session (Session Windows)

Session windows do not have a fixed size but their bounds are defined by an interval of inactivity, i.e., a session window is closes if no event appears for a defined gap period. For example a session window with a 30 minute gap starts when a row is observed after 30 minutes inactivity (otherwise the row would be added to an existing window) and is closed if no row is added within 30 minutes. Session windows can work on event-time or processing-time.

A session window is defined by using the Session class as follows:

MethodDescription
withGapDefines the gap between two windows as time interval.
onThe time attribute to group (time interval) or sort (row count) on. For batch queries this might be any Long or Timestamp attribute. For streaming queries this must be a declared event-time or processing-time time attribute.
asAssigns an alias to the window. The alias is used to reference the window in the following groupBy() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause.
  1. // Session Event-time Window
  2. .window(Session.withGap(lit(10).minutes()).on($("rowtime")).as("w"));
  3. // Session Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Session.withGap(lit(10).minutes()).on($("proctime")).as("w"));
  1. // Session Event-time Window
  2. .window(Session withGap 10.minutes on $"rowtime" as $"w")
  3. // Session Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Session withGap 10.minutes on $"proctime" as $"w")
  1. # Session Event-time Window
  2. .window(Session.with_gap("10.minutes").on("rowtime").alias("w"))
  3. # Session Processing-time Window (assuming a processing-time attribute "proctime")
  4. .window(Session.with_gap("10.minutes").on("proctime").alias("w"))

Back to top

Over Windows

Over window aggregates are known from standard SQL (OVER clause) and defined in the SELECT clause of a query. Unlike group windows, which are specified in the GROUP BY clause, over windows do not collapse rows. Instead over window aggregates compute an aggregate for each input row over a range of its neighboring rows.

Over windows are defined using the window(w: OverWindow*) clause (using over_window(*OverWindow) in Python API) and referenced via an alias in the select() method. The following example shows how to define an over window aggregation on a table.

  1. Table table = input
  2. .window([OverWindow w].as("w")) // define over window with alias w
  3. .select($("a"), $("b").sum().over($("w")), $("c").min().over($("w"))); // aggregate over the over window w
  1. val table = input
  2. .window([w: OverWindow] as $"w") // define over window with alias w
  3. .select($"a", $"b".sum over $"w", $"c".min over $"w") // aggregate over the over window w
  1. # define over window with alias w and aggregate over the over window w
  2. table = input.over_window([OverWindow w].alias("w")) \
  3. .select("a, b.sum over w, c.min over w")

The OverWindow defines a range of rows over which aggregates are computed. OverWindow is not an interface that users can implement. Instead, the Table API provides the Over class to configure the properties of the over window. Over windows can be defined on event-time or processing-time and on ranges specified as time interval or row-count. The supported over window definitions are exposed as methods on Over (and other classes) and are listed below:

MethodRequiredDescription
partitionByOptional

Defines a partitioning of the input on one or more attributes. Each partition is individually sorted and aggregate functions are applied to each partition separately.

Note: In streaming environments, over window aggregates can only be computed in parallel if the window includes a partition by clause. Without partitionBy(…) the stream is processed by a single, non-parallel task.

orderByRequired

Defines the order of rows within each partition and thereby the order in which the aggregate functions are applied to rows.

Note: For streaming queries this must be a declared event-time or processing-time time attribute. Currently, only a single sort attribute is supported.

precedingOptional

Defines the interval of rows that are included in the window and precede the current row. The interval can either be specified as time or row-count interval.

Bounded over windows are specified with the size of the interval, e.g., 10.minutes for a time interval or 10.rows for a row-count interval.

Unbounded over windows are specified using a constant, i.e., UNBOUNDED_RANGE for a time interval or UNBOUNDED_ROW for a row-count interval. Unbounded over windows start with the first row of a partition.

If the preceding clause is omitted, UNBOUNDED_RANGE and CURRENT_RANGE are used as the default preceding and following for the window.

followingOptional

Defines the window interval of rows that are included in the window and follow the current row. The interval must be specified in the same unit as the preceding interval (time or row-count).

At the moment, over windows with rows following the current row are not supported. Instead you can specify one of two constants:

  • CURRENT_ROW sets the upper bound of the window to the current row.
  • CURRENT_RANGE sets the upper bound of the window to sort key of the current row, i.e., all rows with the same sort key as the current row are included in the window.

If the following clause is omitted, the upper bound of a time interval window is defined as CURRENT_RANGE and the upper bound of a row-count interval window is defined as CURRENT_ROW.

asRequired

Assigns an alias to the over window. The alias is used to reference the over window in the following select() clause.

Note: Currently, all aggregation functions in the same select() call must be computed of the same over window.

Unbounded Over Windows

  1. // Unbounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(UNBOUNDED_RANGE).as("w"));
  3. // Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .window(Over.partitionBy($("a")).orderBy("proctime").preceding(UNBOUNDED_RANGE).as("w"));
  5. // Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(UNBOUNDED_ROW).as("w"));
  7. // Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(UNBOUNDED_ROW).as("w"));
  1. // Unbounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .window(Over partitionBy $"a" orderBy $"rowtime" preceding UNBOUNDED_RANGE as "w")
  3. // Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .window(Over partitionBy $"a" orderBy $"proctime" preceding UNBOUNDED_RANGE as "w")
  5. // Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .window(Over partitionBy $"a" orderBy $"rowtime" preceding UNBOUNDED_ROW as "w")
  7. // Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .window(Over partitionBy $"a" orderBy $"proctime" preceding UNBOUNDED_ROW as "w")
  1. # Unbounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .over_window(Over.partition_by("a").order_by("rowtime").preceding("unbounded_range").alias("w"))
  3. # Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .over_window(Over.partition_by("a").order_by("proctime").preceding("unbounded_range").alias("w"))
  5. # Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .over_window(Over.partition_by("a").order_by("rowtime").preceding("unbounded_row").alias("w"))
  7. # Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .over_window(Over.partition_by("a").order_by("proctime").preceding("unbounded_row").alias("w"))

Bounded Over Windows

  1. // Bounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(lit(1).minutes()).as("w"))
  3. // Bounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(lit(1).minutes()).as("w"))
  5. // Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(rowInterval(10)).as("w"))
  7. // Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(rowInterval(10)).as("w"))
  1. // Bounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .window(Over partitionBy $"a" orderBy $"rowtime" preceding 1.minutes as "w")
  3. // Bounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .window(Over partitionBy $"a" orderBy $"proctime" preceding 1.minutes as "w")
  5. // Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .window(Over partitionBy $"a" orderBy $"rowtime" preceding 10.rows as "w")
  7. // Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .window(Over partitionBy $"a" orderBy $"proctime" preceding 10.rows as "w")
  1. # Bounded Event-time over window (assuming an event-time attribute "rowtime")
  2. .over_window(Over.partition_by("a").order_by("rowtime").preceding("1.minutes").alias("w"))
  3. # Bounded Processing-time over window (assuming a processing-time attribute "proctime")
  4. .over_window(Over.partition_by("a").order_by("proctime").preceding("1.minutes").alias("w"))
  5. # Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
  6. .over_window(Over.partition_by("a").order_by("rowtime").preceding("10.rows").alias("w"))
  7. # Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
  8. .over_window(Over.partition_by("a").order_by("proctime").preceding("10.rows").alias("w"))

Back to top

Row-based Operations

The row-based operations generate outputs with multiple columns.

OperatorsDescription
Map
Batch Streaming

Performs a map operation with a user-defined scalar function or built-in scalar function. The output will be flattened if the output type is a composite type.

  1. public class MyMapFunction extends ScalarFunction {
  2. public Row eval(String a) {
  3. return Row.of(a, pre-“ + a);
  4. }
  5. @Override
  6. public TypeInformation<?> getResultType(Class<?>[] signature) {
  7. return Types.ROW(Types.STRING(), Types.STRING());
  8. }
  9. }
  10. ScalarFunction func = new MyMapFunction();
  11. tableEnv.registerFunction(“func”, func);
  12. Table table = input
  13. .map(call(“func”, $(“c”)).as(“a”, b”))
FlatMap
Batch Streaming

Performs a flatMap operation with a table function.

  1. public class MyFlatMapFunction extends TableFunction<Row> {
  2. public void eval(String str) {
  3. if (str.contains(“#”)) {
  4. String[] array = str.split(“#”);
  5. for (int i = 0; i < array.length; ++i) {
  6. collect(Row.of(array[i], array[i].length()));
  7. }
  8. }
  9. }
  10. @Override
  11. public TypeInformation<Row> getResultType() {
  12. return Types.ROW(Types.STRING(), Types.INT());
  13. }
  14. }
  15. TableFunction func = new MyFlatMapFunction();
  16. tableEnv.registerFunction(“func”, func);
  17. Table table = input
  18. .flatMap(call(“func”, $(“c”)).as(“a”, b”))
Aggregate
Batch Streaming Result Updating

Performs an aggregate operation with an aggregate function. You have to close the “aggregate” with a select statement and the select statement does not support aggregate functions. The output of aggregate will be flattened if the output type is a composite type.

  1. public class MyMinMaxAcc {
  2. public int min = 0;
  3. public int max = 0;
  4. }
  5. public class MyMinMax extends AggregateFunction<Row, MyMinMaxAcc> {
  6. public void accumulate(MyMinMaxAcc acc, int value) {
  7. if (value < acc.min) {
  8. acc.min = value;
  9. }
  10. if (value > acc.max) {
  11. acc.max = value;
  12. }
  13. }
  14. @Override
  15. public MyMinMaxAcc createAccumulator() {
  16. return new MyMinMaxAcc();
  17. }
  18. public void resetAccumulator(MyMinMaxAcc acc) {
  19. acc.min = 0;
  20. acc.max = 0;
  21. }
  22. @Override
  23. public Row getValue(MyMinMaxAcc acc) {
  24. return Row.of(acc.min, acc.max);
  25. }
  26. @Override
  27. public TypeInformation<Row> getResultType() {
  28. return new RowTypeInfo(Types.INT, Types.INT);
  29. }
  30. }
  31. AggregateFunction myAggFunc = new MyMinMax();
  32. tableEnv.registerFunction(“myAggFunc”, myAggFunc);
  33. Table table = input
  34. .groupBy($(“key”))
  35. .aggregate(call(“myAggFunc”, $(“a”)).as(“x”, y”))
  36. .select($(“key”), $(“x”), $(“y”))
Group Window Aggregate
Batch Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the “aggregate” with a select statement. And the select statement does not support ““ or aggregate functions.

  1. AggregateFunction myAggFunc = new MyMinMax();
  2. tableEnv.registerFunction(“myAggFunc”, myAggFunc);
  3. Table table = input
  4. .window(Tumble.over(lit(5).minutes())
  5. .on($(“rowtime”))
  6. .as(“w”)) // define window
  7. .groupBy($(“key”), $(“w”)) // group by key and window
  8. .aggregate(call(“myAggFunc”, $(“a”)).as(“x”, y”))
  9. .select($(“key”), $(“x”), $(“y”), $(“w”).start(), $(“w”).end()); // access window properties and aggregate results
FlatAggregate
Streaming
Result Updating

Similar to a GroupBy Aggregation. Groups the rows on the grouping keys with the following running table aggregation operator to aggregate rows group-wise. The difference from an AggregateFunction is that TableAggregateFunction may return 0 or more records for a group. You have to close the “flatAggregate” with a select statement. And the select statement does not support aggregate functions.

Instead of using emitValue to output results, you can also use the emitUpdateWithRetract method. Different from emitValue, emitUpdateWithRetract is used to emit values that have been updated. This method outputs data incrementally in retract mode, i.e., once there is an update, we have to retract old records before sending new updated ones. The emitUpdateWithRetract method will be used in preference to the emitValue method if both methods are defined in the table aggregate function, because the method is treated to be more efficient than emitValue as it can output values incrementally. See Table Aggregation Functions for details.

  1. /**
  2. Accumulator for Top2.
  3. /
  4. public class Top2Accum {
  5. public Integer first;
  6. public Integer second;
  7. }
  8. /**
  9. The top2 user-defined table aggregate function.
  10. */
  11. public class Top2 extends TableAggregateFunction<Tuple2<Integer, Integer>, Top2Accum> {
  12. @Override
  13. public Top2Accum createAccumulator() {
  14. Top2Accum acc = new Top2Accum();
  15. acc.first = Integer.MIN_VALUE;
  16. acc.second = Integer.MIN_VALUE;
  17. return acc;
  18. }
  19. public void accumulate(Top2Accum acc, Integer v) {
  20. if (v > acc.first) {
  21. acc.second = acc.first;
  22. acc.first = v;
  23. } else if (v > acc.second) {
  24. acc.second = v;
  25. }
  26. }
  27. public void merge(Top2Accum acc, java.lang.Iterable<Top2Accum> iterable) {
  28. for (Top2Accum otherAcc : iterable) {
  29. accumulate(acc, otherAcc.first);
  30. accumulate(acc, otherAcc.second);
  31. }
  32. }
  33. public void emitValue(Top2Accum acc, Collector<Tuple2<Integer, Integer>> out) {
  34. // emit the value and rank
  35. if (acc.first != Integer.MIN_VALUE) {
  36. out.collect(Tuple2.of(acc.first, 1));
  37. }
  38. if (acc.second != Integer.MIN_VALUE) {
  39. out.collect(Tuple2.of(acc.second, 2));
  40. }
  41. }
  42. }
  43. tEnv.registerFunction(“top2”, new Top2());
  44. Table orders = tableEnv.from(“Orders”);
  45. Table result = orders
  46. .groupBy($(“key”))
  47. .flatAggregate(call(“top2”, $(“a”)).as(“v”, rank”))
  48. .select($(“key”), $(“v”), $(“rank”);

Note: For streaming queries, the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with a valid retention interval to prevent excessive state size. See Query Configuration for details.

Group Window FlatAggregate
Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the “flatAggregate” with a select statement. And the select statement does not support aggregate functions.

  1. tableEnv.registerFunction(“top2”, new Top2());
  2. Table orders = tableEnv.from(“Orders”);
  3. Table result = orders
  4. .window(Tumble.over(lit(5).minutes())
  5. .on($(“rowtime”))
  6. .as(“w”)) // define window
  7. .groupBy($(“a”), $(“w”)) // group by key and window
  8. .flatAggregate(call(“top2”, $(“b”).as(“v”, rank”))
  9. .select($(“a”), $(“w”).start(), $(“w”).end(), $(“w”).rowtime(), $(“v”), $(“rank”)); // access window properties and aggregate results
OperatorsDescription
Map
Batch Streaming

Performs a map operation with a user-defined scalar function or built-in scalar function. The output will be flattened if the output type is a composite type.

  1. class MyMapFunction extends ScalarFunction {
  2. def eval(a: String): Row = {
  3. Row.of(a, pre-“ + a)
  4. }
  5. override def getResultType(signature: Array[Class[]]): TypeInformation[] =
  6. Types.ROW(Types.STRING, Types.STRING)
  7. }
  8. val func = new MyMapFunction()
  9. val table = input
  10. .map(func($c”)).as(“a”, b”)
FlatMap
Batch Streaming

Performs a flatMap operation with a table function.

  1. class MyFlatMapFunction extends TableFunction[Row] {
  2. def eval(str: String): Unit = {
  3. if (str.contains(“#”)) {
  4. str.split(“#”).foreach({ s =>
  5. val row = new Row(2)
  6. row.setField(0, s)
  7. row.setField(1, s.length)
  8. collect(row)
  9. })
  10. }
  11. }
  12. override def getResultType: TypeInformation[Row] = {
  13. Types.ROW(Types.STRING, Types.INT)
  14. }
  15. }
  16. val func = new MyFlatMapFunction
  17. val table = input
  18. .flatMap(func($c”)).as(“a”, b”)
Aggregate
Batch Streaming Result Updating

Performs an aggregate operation with an aggregate function. You have to close the “aggregate” with a select statement and the select statement does not support aggregate functions. The output of aggregate will be flattened if the output type is a composite type.

  1. case class MyMinMaxAcc(var min: Int, var max: Int)
  2. class MyMinMax extends AggregateFunction[Row, MyMinMaxAcc] {
  3. def accumulate(acc: MyMinMaxAcc, value: Int): Unit = {
  4. if (value < acc.min) {
  5. acc.min = value
  6. }
  7. if (value > acc.max) {
  8. acc.max = value
  9. }
  10. }
  11. override def createAccumulator(): MyMinMaxAcc = MyMinMaxAcc(0, 0)
  12. def resetAccumulator(acc: MyMinMaxAcc): Unit = {
  13. acc.min = 0
  14. acc.max = 0
  15. }
  16. override def getValue(acc: MyMinMaxAcc): Row = {
  17. Row.of(Integer.valueOf(acc.min), Integer.valueOf(acc.max))
  18. }
  19. override def getResultType: TypeInformation[Row] = {
  20. new RowTypeInfo(Types.INT, Types.INT)
  21. }
  22. }
  23. val myAggFunc = new MyMinMax
  24. val table = input
  25. .groupBy($key”)
  26. .aggregate(myAggFunc($a”) as (“x”, y”))
  27. .select($key”, $x”, $y”)
Group Window Aggregate
Batch Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the “aggregate” with a select statement. And the select statement does not support ““ or aggregate functions.

  1. val myAggFunc = new MyMinMax
  2. val table = input
  3. .window(Tumble over 5.minutes on $rowtime as w”) // define window
  4. .groupBy($key”, $w”) // group by key and window
  5. .aggregate(myAggFunc($a”) as (“x”, y”))
  6. .select($key”, $x”, $y”, $w”.start, $w”.end) // access window properties and aggregate results
FlatAggregate
Streaming
Result Updating

Similar to a GroupBy Aggregation. Groups the rows on the grouping keys with the following running table aggregation operator to aggregate rows group-wise. The difference from an AggregateFunction is that TableAggregateFunction may return 0 or more records for a group. You have to close the “flatAggregate” with a select statement. And the select statement does not support aggregate functions.

Instead of using emitValue to output results, you can also use the emitUpdateWithRetract method. Different from emitValue, emitUpdateWithRetract is used to emit values that have been updated. This method outputs data incrementally in retract mode, i.e., once there is an update, we have to retract old records before sending new updated ones. The emitUpdateWithRetract method will be used in preference to the emitValue method if both methods are defined in the table aggregate function, because the method is treated to be more efficient than emitValue as it can output values incrementally. See Table Aggregation Functions for details.

  1. import java.lang.{Integer => JInteger}
  2. import org.apache.flink.table.api.Types
  3. import org.apache.flink.table.functions.TableAggregateFunction
  4. /**
  5. Accumulator for top2.
  6. /
  7. class Top2Accum {
  8. var first: JInteger =
  9. var second: JInteger =
  10. }
  11. /**
  12. The top2 user-defined table aggregate function.
  13. */
  14. class Top2 extends TableAggregateFunction[JTuple2[JInteger, JInteger], Top2Accum] {
  15. override def createAccumulator(): Top2Accum = {
  16. val acc = new Top2Accum
  17. acc.first = Int.MinValue
  18. acc.second = Int.MinValue
  19. acc
  20. }
  21. def accumulate(acc: Top2Accum, v: Int) {
  22. if (v > acc.first) {
  23. acc.second = acc.first
  24. acc.first = v
  25. } else if (v > acc.second) {
  26. acc.second = v
  27. }
  28. }
  29. def merge(acc: Top2Accum, its: JIterable[Top2Accum]): Unit = {
  30. val iter = its.iterator()
  31. while (iter.hasNext) {
  32. val top2 = iter.next()
  33. accumulate(acc, top2.first)
  34. accumulate(acc, top2.second)
  35. }
  36. }
  37. def emitValue(acc: Top2Accum, out: Collector[JTuple2[JInteger, JInteger]]): Unit = {
  38. // emit the value and rank
  39. if (acc.first != Int.MinValue) {
  40. out.collect(JTuple2.of(acc.first, 1))
  41. }
  42. if (acc.second != Int.MinValue) {
  43. out.collect(JTuple2.of(acc.second, 2))
  44. }
  45. }
  46. }
  47. val top2 = new Top2
  48. val orders: Table = tableEnv.from(“Orders”)
  49. val result = orders
  50. .groupBy($key”)
  51. .flatAggregate(top2($a”) as ($v”, $rank”))
  52. .select($key”, $v”, $rank”)

Note: For streaming queries, the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with a valid retention interval to prevent excessive state size. See Query Configuration for details.

Group Window FlatAggregate
Streaming

Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the “flatAggregate” with a select statement. And the select statement does not support aggregate functions.

  1. val top2 = new Top2
  2. val orders: Table = tableEnv.from(“Orders”)
  3. val result = orders
  4. .window(Tumble over 5.minutes on $rowtime as w”) // define window
  5. .groupBy($a”, $w”) // group by key and window
  6. .flatAggregate(top2($b”) as ($v”, $rank”))
  7. .select($a”, w.start, $w”.end, $w”.rowtime, $v”, $rank”) // access window properties and aggregate results

Back to top

Data Types

Please see the dedicated page about data types.

Generic types and (nested) composite types (e.g., POJOs, tuples, rows, Scala case classes) can be fields of a row as well.

Fields of composite types with arbitrary nesting can be accessed with value access functions.

Generic types are treated as a black box and can be passed on or processed by user-defined functions.

Back to top

Expression Syntax

Some of the operators in previous sections expect one or more expressions. Expressions can be specified using an embedded Scala DSL or as Strings. Please refer to the examples above to learn how expressions can be specified.

This is the EBNF grammar for expressions:

  1. expressionList = expression , { "," , expression } ;
  2. expression = overConstant | alias ;
  3. alias = logic | ( logic , "as" , fieldReference ) | ( logic , "as" , "(" , fieldReference , { "," , fieldReference } , ")" ) ;
  4. logic = comparison , [ ( "&&" | "||" ) , comparison ] ;
  5. comparison = term , [ ( "=" | "==" | "===" | "!=" | "!==" | ">" | ">=" | "<" | "<=" ) , term ] ;
  6. term = product , [ ( "+" | "-" ) , product ] ;
  7. product = unary , [ ( "*" | "/" | "%") , unary ] ;
  8. unary = [ "!" | "-" | "+" ] , composite ;
  9. composite = over | suffixed | nullLiteral | prefixed | atom ;
  10. suffixed = interval | suffixAs | suffixCast | suffixIf | suffixDistinct | suffixFunctionCall | timeIndicator ;
  11. prefixed = prefixAs | prefixCast | prefixIf | prefixDistinct | prefixFunctionCall ;
  12. interval = timeInterval | rowInterval ;
  13. timeInterval = composite , "." , ("year" | "years" | "quarter" | "quarters" | "month" | "months" | "week" | "weeks" | "day" | "days" | "hour" | "hours" | "minute" | "minutes" | "second" | "seconds" | "milli" | "millis") ;
  14. rowInterval = composite , "." , "rows" ;
  15. suffixCast = composite , ".cast(" , dataType , ")" ;
  16. prefixCast = "cast(" , expression , dataType , ")" ;
  17. dataType = "BYTE" | "SHORT" | "INT" | "LONG" | "FLOAT" | "DOUBLE" | "BOOLEAN" | "STRING" | "DECIMAL" | "SQL_DATE" | "SQL_TIME" | "SQL_TIMESTAMP" | "INTERVAL_MONTHS" | "INTERVAL_MILLIS" | ( "MAP" , "(" , dataType , "," , dataType , ")" ) | ( "PRIMITIVE_ARRAY" , "(" , dataType , ")" ) | ( "OBJECT_ARRAY" , "(" , dataType , ")" ) ;
  18. suffixAs = composite , ".as(" , fieldReference , ")" ;
  19. prefixAs = "as(" , expression, fieldReference , ")" ;
  20. suffixIf = composite , ".?(" , expression , "," , expression , ")" ;
  21. prefixIf = "?(" , expression , "," , expression , "," , expression , ")" ;
  22. suffixDistinct = composite , "distinct.()" ;
  23. prefixDistinct = functionIdentifier , ".distinct" , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
  24. suffixFunctionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
  25. prefixFunctionCall = functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
  26. atom = ( "(" , expression , ")" ) | literal | fieldReference ;
  27. fieldReference = "*" | identifier ;
  28. nullLiteral = "nullOf(" , dataType , ")" ;
  29. timeIntervalUnit = "YEAR" | "YEAR_TO_MONTH" | "MONTH" | "QUARTER" | "WEEK" | "DAY" | "DAY_TO_HOUR" | "DAY_TO_MINUTE" | "DAY_TO_SECOND" | "HOUR" | "HOUR_TO_MINUTE" | "HOUR_TO_SECOND" | "MINUTE" | "MINUTE_TO_SECOND" | "SECOND" ;
  30. timePointUnit = "YEAR" | "MONTH" | "DAY" | "HOUR" | "MINUTE" | "SECOND" | "QUARTER" | "WEEK" | "MILLISECOND" | "MICROSECOND" ;
  31. over = composite , "over" , fieldReference ;
  32. overConstant = "current_row" | "current_range" | "unbounded_row" | "unbounded_row" ;
  33. timeIndicator = fieldReference , "." , ( "proctime" | "rowtime" ) ;

Literals: Here, literal is a valid Java literal. String literals can be specified using single or double quotes. Duplicate the quote for escaping (e.g. 'It''s me.' or "I ""like"" dogs.").

Null literals: Null literals must have a type attached. Use nullOf(type) (e.g. nullOf(INT)) for creating a null value.

Field references: The fieldReference specifies a column in the data (or all columns if * is used), and functionIdentifier specifies a supported scalar function. The column names and function names follow Java identifier syntax.

Function calls: Expressions specified as strings can also use prefix notation instead of suffix notation to call operators and functions.

Decimals: If working with exact numeric values or large decimals is required, the Table API also supports Java’s BigDecimal type. In the Scala Table API decimals can be defined by BigDecimal("123456") and in Java by appending a “p” for precise e.g. 123456p.

Time representation: In order to work with temporal values the Table API supports Java SQL’s Date, Time, and Timestamp types. In the Scala Table API literals can be defined by using java.sql.Date.valueOf("2016-06-27"), java.sql.Time.valueOf("10:10:42"), or java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123"). The Java and Scala Table API also support calling "2016-06-27".toDate(), "10:10:42".toTime(), and "2016-06-27 10:10:42.123".toTimestamp() for converting Strings into temporal types. Note: Since Java’s temporal SQL types are time zone dependent, please make sure that the Flink Client and all TaskManagers use the same time zone.

Temporal intervals: Temporal intervals can be represented as number of months (Types.INTERVAL_MONTHS) or number of milliseconds (Types.INTERVAL_MILLIS). Intervals of same type can be added or subtracted (e.g. 1.hour + 10.minutes). Intervals of milliseconds can be added to time points (e.g. "2016-08-10".toDate + 5.days).

Scala expressions: Scala expressions use implicit conversions. Therefore, make sure to add the wildcard import org.apache.flink.table.api._ to your programs. In case a literal is not treated as an expression, use .toExpr such as 3.toExpr to force a literal to be converted.

Back to top