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.

  1. import org.apache.flink.table.api._
  2. import org.apache.flink.table.api.java._
  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.scan("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.api.scala. and org.apache.flink.table.api.scala..

The following example shows how a Scala Table API program is constructed. Table attributes are referenced using Scala Symbols, which start with an apostrophe character (').

  1. import org.apache.flink.api.scala._
  2. import org.apache.flink.table.api._
  3. import org.apache.flink.table.api.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.scan("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.scan("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.scan("Orders"); // schema (a, b, c, rowtime)
  5. Table result = 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. val orders: Table = tEnv.scan("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.scan("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).

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
ScanBatchStreamingSimilar to the FROM clause in a SQL query. Performs a scan of a registered table.
  1. Table orders = tableEnv.scan("Orders");
SelectBatchStreamingSimilar to a SQL SELECT statement. Performs a select operation.
  1. Table orders = tableEnv.scan("Orders");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("");
AsBatchStreamingRenames fields.
  1. Table orders = tableEnv.scan("Orders");Table result = orders.as("x, y, z, t");
Where / FilterBatchStreamingSimilar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.
  1. Table orders = tableEnv.scan("Orders");Table result = orders.where("b === 'red'");
or
  1. Table orders = tableEnv.scan("Orders");Table result = orders.filter("a % 2 === 0");
OperatorsDescription
ScanBatchStreamingSimilar to the FROM clause in a SQL query. Performs a scan of a registered table.
  1. val orders: Table = tableEnv.scan("Orders")
SelectBatchStreamingSimilar to a SQL SELECT statement. Performs a select operation.
  1. val orders: Table = tableEnv.scan("Orders")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.scan("Orders")val result = orders.select(')
AsBatchStreamingRenames fields.
  1. val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't)
Where / FilterBatchStreamingSimilar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.
  1. val orders: Table = tableEnv.scan("Orders")val result = orders.filter('a % 2 === 0)
or
  1. val orders: Table = tableEnv.scan("Orders")val result = orders.where('b === "red")
OperatorsDescription
ScanBatchStreamingSimilar to the FROM clause in a SQL query. Performs a scan of a registered table.
  1. orders = table_env.scan("Orders")
SelectBatchStreamingSimilar to a SQL SELECT statement. Performs a select operation.
  1. orders = table_env.scan("Orders")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("")
AliasBatchStreamingRenames fields.
  1. orders = table_env.scan("Orders")result = orders.alias("x, y, z, t")
Where / FilterBatchStreamingSimilar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.
  1. orders = table_env.scan("Orders")result = orders.where("b === 'red'")
or
  1. orders = table_env.scan("Orders")result = orders.filter("a % 2 === 0")

Column Operations

OperatorsDescription
AddColumnsBatchStreamingPerforms a field add operation. It will throw an exception if the added fields already exist.
  1. Table orders = tableEnv.scan("Orders");Table result = orders.addColumns("concat(c, 'sunny')");
AddOrReplaceColumnsBatchStreamingPerforms 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.scan("Orders");Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc");
DropColumnsBatchStreamingPerforms a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.
  1. Table orders = tableEnv.scan("Orders");Table result = orders.dropColumns("b, c");
RenameColumnsBatchStreamingPerforms a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.
  1. Table orders = tableEnv.scan("Orders");Table result = orders.renameColumns("b as b2, c as c2");
OperatorsDescription
AddColumnsBatchStreamingPerforms a field add operation. It will throw an exception if the added fields already exist.
  1. val orders = tableEnv.scan("Orders");val result = orders.addColumns(concat('c, "Sunny"))
AddOrReplaceColumnsBatchStreamingPerforms 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.scan("Orders");val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc)
DropColumnsBatchStreamingPerforms a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.
  1. val orders = tableEnv.scan("Orders");val result = orders.dropColumns('b, 'c)
RenameColumnsBatchStreamingPerforms a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.
  1. val orders = tableEnv.scan("Orders");val result = orders.renameColumns('b as 'b2, 'c as 'c2)
OperatorsDescription
AddColumnsBatchStreamingPerforms a field add operation. It will throw an exception if the added fields already exist.
  1. orders = table_env.scan("Orders")result = orders.add_columns("concat(c, 'sunny')")
AddOrReplaceColumnsBatchStreamingPerforms 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.scan("Orders")result = orders.add_or_replace_columns("concat(c, 'sunny') as desc")
DropColumnsBatchStreamingPerforms a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.
  1. orders = table_env.scan("Orders")result = orders.drop_columns("b, c")
RenameColumnsBatchStreamingPerforms a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.
  1. orders = table_env.scan("Orders")result = orders.rename_columns("b as b2, c as c2")

Aggregations

OperatorsDescription
GroupBy AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders");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 AggregationBatchStreamingGroups and aggregates a table on a group window and possibly one or more grouping keys.
  1. Table orders = tableEnv.scan("Orders");Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window .select("a, w.start, w.end, w.rowtime, b.sum as d"); // access window properties and aggregate
Over Window AggregationStreamingSimilar 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.scan("Orders");Table result = orders // define window .window(Over .partitionBy("a") .orderBy("rowtime") .preceding("UNBOUNDED_RANGE") .following("CURRENT_RANGE") .as("w")) .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 AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders");// Distinct aggregation on group byTable groupByDistinctResult = orders .groupBy("a") .select("a, b.sum.distinct as d");// Distinct aggregation on time window group byTable groupByWindowDistinctResult = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")).groupBy("a, w") .select("a, b.sum.distinct as d");// Distinct aggregation on over windowTable result = orders .window(Over .partitionBy("a") .orderBy("rowtime") .preceding("UNBOUNDED_RANGE") .as("w")) .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. Table orders = tEnv.scan("Orders");// Use distinct aggregation for user-defined aggregate functionstEnv.registerFunction("myUdagg", new MyUdagg());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.
DistinctBatchStreamingResult UpdatingSimilar to a SQL DISTINCT clause. Returns records with distinct value combinations.
  1. Table orders = tableEnv.scan("Orders");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 AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders")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 AggregationBatchStreamingGroups and aggregates a table on a group window and possibly one or more grouping keys.
  1. val orders: Table = tableEnv.scan("Orders")val result: Table = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window .select('a, w.start, 'w.end, 'w.rowtime, 'b.sum as 'd) // access window properties and aggregate
Over Window AggregationStreamingSimilar 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.scan("Orders")val result: Table = orders // define window .window(Over partitionBy 'a orderBy 'rowtime preceding UNBOUNDED_RANGE following CURRENT_RANGE as 'w) .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 AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders");// Distinct aggregation on group byval groupByDistinctResult = orders .groupBy('a) .select('a, 'b.sum.distinct as 'd)// Distinct aggregation on time window group byval groupByWindowDistinctResult = orders .window(Tumble over 5.minutes on 'rowtime as 'w).groupBy('a, 'w) .select('a, 'b.sum.distinct as 'd)// Distinct aggregation on over windowval result = orders .window(Over partitionBy 'a orderBy 'rowtime preceding UNBOUNDED_RANGE as 'w) .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.scan("Orders");// Use distinct aggregation for user-defined aggregate functionsval myUdagg = new MyUdagg();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.
DistinctBatchStreamingResult UpdatingSimilar to a SQL DISTINCT clause. Returns records with distinct value combinations.
  1. val orders: Table = tableEnv.scan("Orders")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 AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders")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 AggregationBatchStreamingGroups and aggregates a table on a group window and possibly one or more grouping keys.
  1. orders = table_env.scan("Orders")result = orders.window(Tumble.over("5.minutes").on("rowtime").alias("w")) \ .group_by("a, w") \ .select("a, w.start, w.end, w.rowtime, b.sum as d")
Over Window AggregationStreamingSimilar 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.scan("Orders")result = orders.over_window(Over.partition_by("a").order_by("rowtime") .preceding("UNBOUNDED_RANGE").following("CURRENT_RANGE") .alias("w")) \ .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 AggregationBatchStreamingResult UpdatingSimilar 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.scan("Orders")# Distinct aggregation on group bygroup_by_distinct_result = orders.group_by("a") \ .select("a, b.sum.distinct as d")# Distinct aggregation on time window group bygroup_by_window_distinct_result = orders.window( Tumble.over("5.minutes").on("rowtime").alias("w")).groupBy("a, w") \ .select("a, b.sum.distinct as d")# Distinct aggregation on over windowresult = orders.over_window(Over .partition_by("a") .order_by("rowtime") .preceding("UNBOUNDED_RANGE") .alias("w")) \ .select( "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.
DistinctBatchStreamingResult UpdatingSimilar to a SQL DISTINCT clause. Returns records with distinct value combinations.
  1. orders = table_env.scan("Orders")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.

Joins

OperatorsDescription
Inner JoinBatchStreamingSimilar 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");Table right = tableEnv.fromDataSet(ds2, "d, e, f");Table 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 JoinBatchStreamingResult UpdatingSimilar 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");Table right = tableEnv.fromDataSet(ds2, "d, e, f");Table leftOuterResult = left.leftOuterJoin(right, "a = d").select("a, b, e");Table rightOuterResult = left.rightOuterJoin(right, "a = d").select("a, b, e");Table 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.
Time-windowed JoinBatchStreamingNote: Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.A time-windowed 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 window join conditions:- ltime === rtime- ltime >= rtime && ltime < rtime + 10.minutes
  1. Table left = tableEnv.fromDataSet(ds1, "a, b, c, ltime.rowtime");Table right = tableEnv.fromDataSet(ds2, "d, e, f, rtime.rowtime");Table result = left.join(right) .where("a = d && ltime >= rtime - 5.minutes && ltime < rtime + 10.minutes") .select("a, b, e, ltime");
Inner Join with Table Function (UDTF)BatchStreamingJoins 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 FunctionTableFunction<String> split = new MySplitUDTF();tableEnv.registerFunction("split", split);// joinTable orders = tableEnv.scan("Orders");Table result = orders .joinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v");
Left Outer Join with Table Function (UDTF)BatchStreamingJoins 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 FunctionTableFunction<String> split = new MySplitUDTF();tableEnv.registerFunction("split", split);// joinTable orders = tableEnv.scan("Orders");Table result = orders .leftOuterJoinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v");
Join with Temporal TableStreamingTemporal 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.scan("RatesHistory");// register temporal table function with a time attribute and primary keyTemporalTableFunction rates = ratesHistory.createTemporalTableFunction( "r_proctime", "r_currency");tableEnv.registerFunction("rates", rates);// join with "Orders" based on the time attribute and keyTable orders = tableEnv.scan("Orders");Table result = orders .joinLateral("rates(o_proctime)", "o_currency = r_currency")
For more information please check the more detailed temporal tables concept description.
OperatorsDescription
Inner JoinBatchStreamingSimilar 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)val right = ds2.toTable(tableEnv, 'd, 'e, 'f)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 JoinBatchStreamingResult UpdatingSimilar 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)val right = tableEnv.fromDataSet(ds2, 'd, 'e, 'f)val leftOuterResult = left.leftOuterJoin(right, 'a === 'd).select('a, 'b, 'e)val rightOuterResult = left.rightOuterJoin(right, 'a === 'd).select('a, 'b, 'e)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.
Time-windowed JoinBatchStreamingNote: Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.A time-windowed 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 window join conditions:- 'ltime === 'rtime- 'ltime >= 'rtime && 'ltime < 'rtime + 10.minutes
  1. val left = ds1.toTable(tableEnv, 'a, 'b, 'c, 'ltime.rowtime)val right = ds2.toTable(tableEnv, 'd, 'e, 'f, 'rtime.rowtime)val result = left.join(right) .where('a === 'd && 'ltime >= 'rtime - 5.minutes && 'ltime < 'rtime + 10.minutes) .select('a, 'b, 'e, 'ltime)
Inner Join with Table Function (UDTF)BatchStreamingJoins 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 Functionval split: TableFunction[] = new MySplitUDTF()// joinval result: Table = table .joinLateral(split('c) as ('s, 't, 'v)) .select('a, 'b, 's, 't, 'v)
Left Outer Join with Table Function (UDTF)BatchStreamingJoins 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 Functionval split: TableFunction[] = new MySplitUDTF()// joinval result: Table = table .leftOuterJoinLateral(split('c) as ('s, 't, 'v)) .select('a, 'b, 's, 't, 'v)
Join with Temporal TableStreamingTemporal 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.scan("RatesHistory")// register temporal table function with a time attribute and primary keyval rates = ratesHistory.createTemporalTableFunction('r_proctime, 'r_currency)// join with "Orders" based on the time attribute and keyval orders = tableEnv.scan("Orders")val result = orders .joinLateral(rates('o_rowtime), 'r_currency === 'o_currency)
For more information please check the more detailed temporal tables concept description.
OperatorsDescription
Inner JoinBatchStreamingSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("d, e, f")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 JoinBatchStreamingResult UpdatingSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("d, e, f")left_outer_result = left.left_outer_join(right, "a = d").select("a, b, e")right_outer_result = left.right_outer_join(right, "a = d").select("a, b, e")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.
Time-windowed JoinBatchStreamingCurrently not supported in python API.
Inner Join with Table Function (UDTF)BatchStreamingJoins 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 Functiontable_env.register_java_function("split", "com.my.udf.MySplitUDTF")# joinorders = table_env.scan("Orders")result = orders.join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v")
Left Outer Join with Table Function (UDTF)BatchStreamingJoins 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 Functiontable_env.register_java_function("split", "com.my.udf.MySplitUDTF")# joinorders = table_env.scan("Orders")result = orders.left_outer_join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v")
Join with Temporal TableStreamingCurrently not supported in python API.

Set Operations

OperatorsDescription
UnionBatchSimilar 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");Table right = tableEnv.fromDataSet(ds2, "a, b, c");Table result = left.union(right);
UnionAllBatchStreamingSimilar 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");Table right = tableEnv.fromDataSet(ds2, "a, b, c");Table result = left.unionAll(right);
IntersectBatchSimilar 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");Table right = tableEnv.fromDataSet(ds2, "d, e, f");Table result = left.intersect(right);
IntersectAllBatchSimilar 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");Table right = tableEnv.fromDataSet(ds2, "d, e, f");Table result = left.intersectAll(right);
MinusBatchSimilar 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");Table right = tableEnv.fromDataSet(ds2, "a, b, c");Table result = left.minus(right);
MinusAllBatchSimilar 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");Table right = tableEnv.fromDataSet(ds2, "a, b, c");Table result = left.minusAll(right);
InBatchStreamingSimilar 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");Table right = ds2.toTable(tableEnv, "a");// using implicit registrationTable result = left.select("a, b, c").where("a.in(" + right + ")");// using explicit registrationtableEnv.registerTable("RightTable", right);Table 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.
OperatorsDescription
UnionBatchSimilar 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)val right = ds2.toTable(tableEnv, 'a, 'b, 'c)val result = left.union(right)
UnionAllBatchStreamingSimilar 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)val right = ds2.toTable(tableEnv, 'a, 'b, 'c)val result = left.unionAll(right)
IntersectBatchSimilar 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)val right = ds2.toTable(tableEnv, 'e, 'f, 'g)val result = left.intersect(right)
IntersectAllBatchSimilar 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)val right = ds2.toTable(tableEnv, 'e, 'f, 'g)val result = left.intersectAll(right)
MinusBatchSimilar 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)val right = ds2.toTable(tableEnv, 'a, 'b, 'c)val result = left.minus(right)
MinusAllBatchSimilar 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)val right = ds2.toTable(tableEnv, 'a, 'b, 'c)val result = left.minusAll(right)
InBatchStreamingSimilar 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)val right = ds2.toTable(tableEnv, 'a)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
UnionBatchSimilar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical field types.
  1. left = table_env.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.union(right)
UnionAllBatchStreamingSimilar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types.
  1. left = table_env.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.union_all(right)
IntersectBatchSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.intersect(right)
IntersectAllBatchSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.intersect_all(right)
MinusBatchSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.minus(right);
MinusAllBatchSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a, b, c")result = left.minus_all(right)
InBatchStreamingSimilar 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.scan("Source1").select("a, b, c")right = table_env.scan("Source2").select("a")# using implicit registrationresult = left.select("a, b, c").where("a.in(%s)" % right)# using explicit registrationtable_env.register_table("RightTable", right)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.

OrderBy, Offset & Fetch

OperatorsDescription
Order ByBatchSimilar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.
  1. Table in = tableEnv.fromDataSet(ds, "a, b, c");Table result = in.orderBy("a.asc");
Offset & FetchBatchSimilar 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");// returns the first 5 records from the sorted resultTable result1 = in.orderBy("a.asc").fetch(5);// skips the first 3 records and returns all following records from the sorted resultTable result2 = in.orderBy("a.asc").offset(3);// skips the first 10 records and returns the next 5 records from the sorted resultTable result3 = in.orderBy("a.asc").offset(10).fetch(5);
OperatorsDescription
Order ByBatchSimilar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.
  1. val in = ds.toTable(tableEnv, 'a, 'b, 'c)val result = in.orderBy('a.asc)
Offset & FetchBatchSimilar 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)// returns the first 5 records from the sorted resultval result1: Table = in.orderBy('a.asc).fetch(5)// skips the first 3 records and returns all following records from the sorted resultval result2: Table = in.orderBy('a.asc).offset(3)// skips the first 10 records and returns the next 5 records from the sorted resultval result3: Table = in.orderBy('a.asc).offset(10).fetch(5)
OperatorsDescription
Order ByBatchSimilar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.
  1. in = table_env.scan("Source1").select("a, b, c")result = in.order_by("a.asc")
Offset & FetchBatchSimilar 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.scan("Source1").select("a, b, c")# returns the first 5 records from the sorted resultresult1 = in.order_by("a.asc").fetch(5)# skips the first 3 records and returns all following records from the sorted resultresult2 = in.order_by("a.asc").offset(3)# skips the first 10 records and returns the next 5 records from the sorted resultresult3 = in.order_by("a.asc").offset(10).fetch(5)

Insert

OperatorsDescription
Insert IntoBatchStreamingSimilar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.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. Table orders = tableEnv.scan("Orders");orders.insertInto("OutOrders");
OperatorsDescription
Insert IntoBatchStreamingSimilar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.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. val orders: Table = tableEnv.scan("Orders")orders.insertInto("OutOrders")
OperatorsDescription
Insert IntoBatchStreamingSimilar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.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.scan("Orders");orders.insert_into("OutOrders");

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("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 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("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 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("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 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"))

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
partitionByOptionalDefines 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.
orderByRequiredDefines 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.
precedingOptionalDefines 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.
followingOptionalDefines 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.
asRequiredAssigns 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("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. .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"))

Row-based Operations

The row-based operations generate outputs with multiple columns.

OperatorsDescription
MapBatchStreamingPerforms 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 { public Row eval(String a) { return Row.of(a, "pre-" + a); } @Override public TypeInformation<?> getResultType(Class<?>[] signature) { return Types.ROW(Types.STRING(), Types.STRING()); }}ScalarFunction func = new MyMapFunction();tableEnv.registerFunction("func", func);Table table = input .map("func(c)").as("a, b")
FlatMapBatchStreamingPerforms a flatMap operation with a table function.
  1. public class MyFlatMapFunction extends TableFunction<Row> { public void eval(String str) { if (str.contains("#")) { String[] array = str.split("#"); for (int i = 0; i < array.length; ++i) { collect(Row.of(array[i], array[i].length())); } } } @Override public TypeInformation<Row> getResultType() { return Types.ROW(Types.STRING(), Types.INT()); }}TableFunction func = new MyFlatMapFunction();tableEnv.registerFunction("func", func);Table table = input .flatMap("func(c)").as("a, b")
AggregateBatchStreamingResult UpdatingPerforms 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 { public int min = 0; public int max = 0;}public class MyMinMax extends AggregateFunction<Row, MyMinMaxAcc> { public void accumulate(MyMinMaxAcc acc, int value) { if (value < acc.min) { acc.min = value; } if (value > acc.max) { acc.max = value; } } @Override public MyMinMaxAcc createAccumulator() { return new MyMinMaxAcc(); } public void resetAccumulator(MyMinMaxAcc acc) { acc.min = 0; acc.max = 0; } @Override public Row getValue(MyMinMaxAcc acc) { return Row.of(acc.min, acc.max); } @Override public TypeInformation<Row> getResultType() { return new RowTypeInfo(Types.INT, Types.INT); }}AggregateFunction myAggFunc = new MyMinMax();tableEnv.registerFunction("myAggFunc", myAggFunc);Table table = input .groupBy("key") .aggregate("myAggFunc(a) as (x, y)") .select("key, x, y")
Group Window AggregateBatchStreamingGroups 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();tableEnv.registerFunction("myAggFunc", myAggFunc);Table table = input .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("key, w") // group by key and window .aggregate("myAggFunc(a) as (x, y)") .select("key, x, y, w.start, w.end"); // access window properties and aggregate results
FlatAggregateStreamingResult UpdatingSimilar 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. /public class Top2Accum { public Integer first; public Integer second;}/**
  3. The top2 user-defined table aggregate function. /public class Top2 extends TableAggregateFunction<Tuple2<Integer, Integer>, Top2Accum> { @Override public Top2Accum createAccumulator() { Top2Accum acc = new Top2Accum(); acc.first = Integer.MIN_VALUE; acc.second = Integer.MIN_VALUE; return acc; } public void accumulate(Top2Accum acc, Integer v) { if (v > acc.first) { acc.second = acc.first; acc.first = v; } else if (v > acc.second) { acc.second = v; } } public void merge(Top2Accum acc, java.lang.Iterable<Top2Accum> iterable) { for (Top2Accum otherAcc : iterable) { accumulate(acc, otherAcc.first); accumulate(acc, otherAcc.second); } } public void emitValue(Top2Accum acc, Collector<Tuple2<Integer, Integer>> out) { // emit the value and rank if (acc.first != Integer.MIN_VALUE) { out.collect(Tuple2.of(acc.first, 1)); } if (acc.second != Integer.MIN_VALUE) { out.collect(Tuple2.of(acc.second, 2)); } }}tEnv.registerFunction("top2", new Top2());Table orders = tableEnv.scan("Orders");Table result = orders .groupBy("key") .flatAggregate("top2(a) as (v, rank)") .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 FlatAggregateStreamingGroups 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());Table orders = tableEnv.scan("Orders");Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window .flatAggregate("top2(b) as (v, rank)") .select("a, w.start, w.end, w.rowtime, v, rank"); // access window properties and aggregate results
OperatorsDescription
MapBatchStreamingPerforms 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 { def eval(a: String): Row = { Row.of(a, "pre-" + a) } override def getResultType(signature: Array[Class[]]): TypeInformation[] = Types.ROW(Types.STRING, Types.STRING)}val func = new MyMapFunction()val table = input .map(func('c)).as('a, 'b)
FlatMapBatchStreamingPerforms a flatMap operation with a table function.
  1. class MyFlatMapFunction extends TableFunction[Row] { def eval(str: String): Unit = { if (str.contains("#")) { str.split("#").foreach({ s => val row = new Row(2) row.setField(0, s) row.setField(1, s.length) collect(row) }) } } override def getResultType: TypeInformation[Row] = { Types.ROW(Types.STRING, Types.INT) }}val func = new MyFlatMapFunctionval table = input .flatMap(func('c)).as('a, 'b)
AggregateBatchStreamingResult UpdatingPerforms 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)class MyMinMax extends AggregateFunction[Row, MyMinMaxAcc] { def accumulate(acc: MyMinMaxAcc, value: Int): Unit = { if (value < acc.min) { acc.min = value } if (value > acc.max) { acc.max = value } } override def createAccumulator(): MyMinMaxAcc = MyMinMaxAcc(0, 0) def resetAccumulator(acc: MyMinMaxAcc): Unit = { acc.min = 0 acc.max = 0 } override def getValue(acc: MyMinMaxAcc): Row = { Row.of(Integer.valueOf(acc.min), Integer.valueOf(acc.max)) } override def getResultType: TypeInformation[Row] = { new RowTypeInfo(Types.INT, Types.INT) }}val myAggFunc = new MyMinMaxval table = input .groupBy('key) .aggregate(myAggFunc('a) as ('x, 'y)) .select('key, 'x, 'y)
Group Window AggregateBatchStreamingGroups 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 MyMinMaxval table = input .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('key, 'w) // group by key and window .aggregate(myAggFunc('a) as ('x, 'y)) .select('key, 'x, 'y, 'w.start, 'w.end) // access window properties and aggregate results
FlatAggregateStreamingResult UpdatingSimilar 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}import org.apache.flink.table.api.Typesimport org.apache.flink.table.functions.TableAggregateFunction/**
  2. Accumulator for top2. /class Top2Accum { var first: JInteger = var second: JInteger = }/**
  3. The top2 user-defined table aggregate function. /class Top2 extends TableAggregateFunction[JTuple2[JInteger, JInteger], Top2Accum] { override def createAccumulator(): Top2Accum = { val acc = new Top2Accum acc.first = Int.MinValue acc.second = Int.MinValue acc } def accumulate(acc: Top2Accum, v: Int) { if (v > acc.first) { acc.second = acc.first acc.first = v } else if (v > acc.second) { acc.second = v } } def merge(acc: Top2Accum, its: JIterable[Top2Accum]): Unit = { val iter = its.iterator() while (iter.hasNext) { val top2 = iter.next() accumulate(acc, top2.first) accumulate(acc, top2.second) } } def emitValue(acc: Top2Accum, out: Collector[JTuple2[JInteger, JInteger]]): Unit = { // emit the value and rank if (acc.first != Int.MinValue) { out.collect(JTuple2.of(acc.first, 1)) } if (acc.second != Int.MinValue) { out.collect(JTuple2.of(acc.second, 2)) } }}val top2 = new Top2val orders: Table = tableEnv.scan("Orders")val result = orders .groupBy('key) .flatAggregate(top2('a) as ('v, 'rank)) .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 FlatAggregateStreamingGroups 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 Top2val orders: Table = tableEnv.scan("Orders")val result = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window .flatAggregate(top2('b) as ('v, 'rank)) .select('a, w.start, 'w.end, 'w.rowtime, 'v, 'rank) // access window properties and aggregate results

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.

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.scala._ 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.