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, Java and Python. Instead of specifying queries as String values as common with SQL, Table API queries are defined in a language-embedded style in Java, Scala or Python 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, Java and Python. The Scala Table API leverages on Scala expressions, the Java Table API supports both Expression DSL and strings which are parsed and converted into equivalent expressions, the Python Table API currently only supports strings which are parsed and converted into equivalent expressions.
The following example shows the differences between the Scala, Java and Python 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 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.*
import org.apache.flink.table.api.*
import static org.apache.flink.table.api.Expressions.*
// environment configuration
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
BatchTableEnvironment tEnv = BatchTableEnvironment.create(env);
// register Orders table in table environment
// ...
// specify table program
Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime)
Table counts = orders
.groupBy($("a"))
.select($("a"), $("b").count().as("cnt"));
// conversion to DataSet
DataSet<Row> result = tEnv.toDataSet(counts, Row.class);
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 ($
).
import org.apache.flink.api.scala._
import org.apache.flink.table.api._
import org.apache.flink.table.api.bridge.scala._
// environment configuration
val env = ExecutionEnvironment.getExecutionEnvironment
val tEnv = BatchTableEnvironment.create(env)
// register Orders table in table environment
// ...
// specify table program
val orders = tEnv.from("Orders") // schema (a, b, c, rowtime)
val result = orders
.groupBy($"a")
.select($"a", $"b".count as "cnt")
.toDataSet[Row] // conversion to DataSet
.print()
下面这个例子演示了如何组织一个 Python Table API 程序,以及字符串形式的表达式用法。
from pyflink.table import *
# environment configuration
t_env = BatchTableEnvironment.create(
environment_settings=EnvironmentSettings.new_instance().in_batch_mode().use_blink_planner().build())
# register Orders table and Result table sink in table environment
source_data_path = "/path/to/source/directory/"
result_data_path = "/path/to/result/directory/"
source_ddl = f"""
create table Orders(
a VARCHAR,
b BIGINT,
c BIGINT,
rowtime TIMESTAMP(3),
WATERMARK FOR rowtime AS rowtime - INTERVAL '1' SECOND
) with (
'connector' = 'filesystem',
'format' = 'csv',
'path' = '{source_data_path}'
)
"""
t_env.execute_sql(source_ddl)
sink_ddl = f"""
create table `Result`(
a VARCHAR,
cnt BIGINT
) with (
'connector' = 'filesystem',
'format' = 'csv',
'path' = '{result_data_path}'
)
"""
t_env.execute_sql(sink_ddl)
# specify table program
orders = t_env.from_path("Orders") # schema (a, b, c, rowtime)
orders.group_by("a").select(orders.a, orders.b.count.alias('cnt')).execute_insert("result").wait()
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
.
// environment configuration
// ...
// specify table program
Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime)
Table result = orders
.filter(
and(
$("a").isNotNull(),
$("b").isNotNull(),
$("c").isNotNull()
))
.select($("a").lowerCase().as("a"), $("b"), $("rowtime"))
.window(Tumble.over(lit(1).hours()).on($("rowtime")).as("hourlyWindow"))
.groupBy($("hourlyWindow"), $("a"))
.select($("a"), $("hourlyWindow").end().as("hour"), $("b").avg().as("avgBillingAmount"));
// environment configuration
// ...
// specify table program
val orders: Table = tEnv.from("Orders") // schema (a, b, c, rowtime)
val result: Table = orders
.filter($"a".isNotNull && $"b".isNotNull && $"c".isNotNull)
.select($"a".lowerCase() as "a", $"b", $"rowtime")
.window(Tumble over 1.hour on $"rowtime" as "hourlyWindow")
.groupBy($"hourlyWindow", $"a")
.select($"a", $"hourlyWindow".end as "hour", $"b".avg as "avgBillingAmount")
# specify table program
from pyflink.table.expressions import col, lit
orders = t_env.from_path("Orders") # schema (a, b, c, rowtime)
result = orders.filter(orders.a.is_not_null & orders.b.is_not_null & orders.c.is_not_null) \
.select(orders.a.lower_case.alias('a'), orders.b, orders.rowtime) \
.window(Tumble.over(lit(1).hour).on(orders.rowtime).alias("hourly_window")) \
.group_by(col('hourly_window'), col('a')) \
.select(col('a'), col('hourly_window').end.alias('hour'), b.avg.alias('avg_billing_amount'))
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
Operators | Description |
---|---|
From Batch Streaming | Similar to the FROM clause in a SQL query. Performs a scan of a registered table. |
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 will produce a Table with a schema as follows: 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. will produce a Table with a schema as follows: |
Select Batch Streaming | Similar to a SQL SELECT statement. Performs a select operation. You can use star ( |
As Batch Streaming | Renames fields. |
Where / Filter Batch Streaming | Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate. or |
Operators | Description |
---|---|
From Batch Streaming | Similar to the FROM clause in a SQL query. Performs a scan of a registered table. |
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 will produce a Table with a schema as follows: 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. will produce a Table with a schema as follows: |
Select Batch Streaming | Similar to a SQL SELECT statement. Performs a select operation. You can use star ( |
As Batch Streaming | Renames fields. |
Where / Filter Batch Streaming | Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate. or |
操作 | 描述 |
---|---|
FromPath 批处理 流处理 | 类似于 SQL 请求中的 FROM 子句,将一个环境中已注册的表转换成 Table 对象。 |
FromElements 批处理 流处理 | Similar to the VALUES clause in a SQL query. Produces an inline table out of the provided rows. will produce a Table with a schema as follows: The method will derive the types automatically from the input data. The elements types must be acceptable atomic types or acceptable composite types. All elements must be of the same type. If the elements types are composite types, the composite types must be strictly equal, and its subtypes must also be acceptable types, e.g. if the elements are tuples, the length of the tuples must be equal, the element types of the tuples must be equal in order. The built-in acceptable atomic element types contains: int, long, str, unicode, bool, float, bytearray, datetime.date, datetime.time, datetime.datetime, datetime.timedelta, decimal.Decimal The built-in acceptable composite element types contains: list, tuple, dict, array, pyflink.table.Row 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. will produce a Table with a schema as follows: |
Select 批处理 流处理 | 类似于 SQL 请求中的 SELECT 子句,执行一个 select 操作。 你可以使用星号( |
Alias 批处理 流处理 | 重命名字段。 |
Where / Filter 批处理 流处理 | 类似于 SQL 请求中的 WHERE 子句,过滤掉表中不满足条件的行。 or |
Column Operations
Operators | Description |
---|---|
AddColumns Batch Streaming | Performs a field add operation. It will throw an exception if the added fields already exist. |
AddOrReplaceColumns Batch Streaming | Performs a field add operation. Existing fields will be replaced if the added column name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used. |
DropColumns Batch Streaming | Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped. |
RenameColumns Batch Streaming | Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed. |
Operators | Description |
---|---|
AddColumns Batch Streaming | Performs a field add operation. It will throw an exception if the added fields already exist. |
AddOrReplaceColumns Batch Streaming | Performs a field add operation. Existing fields will be replaced if the added column name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used. |
DropColumns Batch Streaming | Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped. |
RenameColumns Batch Streaming | Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed. |
操作 | 描述 |
---|---|
AddColumns 批处理 流处理 | 执行新增字段操作。如果欲添加字段已经存在,将会抛出异常。 |
AddOrReplaceColumns 批处理 流处理 | 执行新增字段操作。如果欲添加字段已经存在,将会替换该字段。如果新增字段列表中有同名字段,取最靠后的为有效字段。 |
DropColumns 批处理 流处理 | 执行删除字段操作。参数必须是字段列表,并且必须是已经存在的字段才能被删除。 |
RenameColumns 批处理 流处理 | 执行重命名字段操作。参数必须是字段别名(例:b as b2)列表,并且必须是已经存在的字段才能被重命名。 |
Aggregations
Operators | Description |
---|---|
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. 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. |
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. 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. 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. 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. 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. |
Operators | Description |
---|---|
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. 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. |
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. 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. 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. 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. 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. |
操作 | 描述 |
---|---|
GroupBy Aggregation 批处理 流处理 结果持续更新 | 类似于 SQL 的 GROUP BY 子句。将数据按照指定字段进行分组,之后对各组内数据执行聚合操作。 注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体情况取决于聚合操作的类型和分组的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
GroupBy Window Aggregation 批处理 流处理 | 在一个窗口上分组和聚合数据,可包含其它分组字段。 |
Over Window Aggregation 流处理 | 类似于 SQL 中的 OVER 开窗函数。Over 窗口聚合对每一行都进行一次聚合计算,聚合的对象是以当前行的位置为基准,向前向后取一个区间范围内的所有数据。详情请见 Over 窗口一节。 注意: 所有的聚合操作必须在同一个窗口上定义,即分组,排序,范围等属性必须一致。目前,窗口区间范围的向前(PRECEDING)取值没有限制,可以为无界(UNBOUNDED),但是向后(FOLLOWING)只支持当前行(CURRENT ROW),其它向后范围取值暂不支持。排序(ORDER BY)属性必须指定单个时间属性。 |
Distinct Aggregation 批处理 流处理 结果持续更新 | 类似于 SQL 聚合函数中的的 DISTINCT 关键字比如 COUNT(DISTINCT a)。带有 distinct 标记的聚合函数只会接受不重复的输入,重复输入将被丢弃。这个去重特性可以在分组聚合(GroupBy Aggregation),分组窗口聚合(GroupBy Window Aggregation)以及 Over窗口聚合(Over Window Aggregation)上使用。 注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体情况取决于执行去重判断时参与判断的字段的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
Distinct 批处理 流处理 结果持续更新 | 类似于 SQL 中的 DISTINCT 子句。返回去重后的数据。 注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体情况取决于执行去重判断时参与判断的字段的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
Joins
Operators | Description |
---|---|
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. 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. 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. |
Inner/Outer Interval Join Batch Streaming | Note: Interval joins are a subset of regular joins that can be processed in a streaming fashion. Both inner and outer joins are supported. 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 ( For example, the following predicates are valid interval join conditions:
|
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. |
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 |
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. For more information please check the more detailed temporal tables concept description. |
Operators | Description |
---|---|
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. 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. 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 ( For example, the following predicates are valid interval join conditions:
|
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. |
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 |
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. For more information please check the more detailed temporal tables concept description. |
操作 | 描述 |
---|---|
Inner Join 批处理 流处理 | 类似于 SQL 的 JOIN 子句。对两张表执行内连接操作。两张表必须具有不同的字段名称,并且必须在 join 方法或者随后的 where 或 filter 方法中定义至少一个等值连接条件。 注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体取决于不重复的输入行的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
Outer Join 批处理 流处理 结果持续更新 | 类似于 SQL 的 LEFT/RIGHT/FULL OUTER JOIN 子句。对两张表执行外连接操作。两张表必须具有不同的字段名称,并且必须定义至少一个等值连接条件。 注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体取决于不重复的输入行的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
Interval Join 批处理 流处理 | 注意: Interval Join 是所有常规流式数据处理 join 操作中的其中一种场景。 Interval Join 要求至少有一个等值连接条件以及一个用于划定时间间隔的条件。对两条数据流时间的划定可以通过两个范围比较确定一个合适时间区间( 如下示例是合法的 Interval Join 条件:
|
Inner Join with Table Function (UDTF) 批处理 流处理 | 将一张表与一个表函数的执行结果执行内连接操作。左表的每一行都会进行一次表函数调用,调用将会返回 0 个,1 个或多个结果,再与这些结果执行连接操作。如果一行数据对应的表函数调用返回了一个空的结果集,则这行数据会被丢弃。 |
Left Outer Join with Table Function (UDTF) 批处理 流处理 | 将一张表与一个表函数的执行结果执行左连接操作。左表的每一行都会进行一次表函数调用,调用将会返回 0 个,1 个或多个结果,再与这些结果执行连接操作。如果一行数据对应的表函数调用返回了一个空的结果集,这行数据依然会被保留,对应的右表数值用 null(python为None)填充。 注意:目前,表函数的左连接操作的连接条件(join predicate)只能为空或者为”true”常量。 |
Join with Temporal Table 流处理 | Python Table API 暂不支持。 |
Set Operations
Operators | Description |
---|---|
Union Batch | Similar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical field types. |
UnionAll Batch Streaming | Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types. |
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. |
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. |
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. |
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. |
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. 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. |
Operators | Description |
---|---|
Union Batch | Similar to a SQL UNION clause. Unions two tables with duplicate records removed, both tables must have identical field types. |
UnionAll Batch Streaming | Similar to a SQL UNION ALL clause. Unions two tables, both tables must have identical field types. |
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. |
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. |
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. |
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. |
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. 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. |
Operators | Description |
---|---|
Union 批处理 | 类似于 SQL 的 UNION 子句。将两张表组合成一张表,这张表拥有二者去除重复后的全部数据。两张表的字段和类型必须完全一致。 |
UnionAll 批处理 流处理 | 类似于 SQL 的 UNION ALL 子句。将两张表组合成一张表,这张表拥有二者的全部数据。两张表的字段和类型必须完全一致。 |
Intersect 批处理 | 类似于 SQL 的 INTERSECT 子句。Intersect 返回在两张表中都存在的数据。如果一个记录在两张表中不止出现一次,则只返回一次,即结果表没有重复记录。两张表的字段和类型必须完全一致。 |
IntersectAll 批处理 | 类似于 SQL 的 INTERSECT ALL 子句。IntersectAll 返回在两张表中都存在的数据。如果一个记录在两张表中不止出现一次,则按照它在两张表中都出现的次数返回,即结果表可能包含重复数据。两张表的字段和类型必须完全一致。 |
Minus 批处理 | 类似于 SQL 的 EXCEPT 子句。Minus 返回仅存在于左表,不存在于右表中的数据。左表中的相同数据只会返回一次,即数据会被去重。两张表的字段和类型必须完全一致。 |
MinusAll 批处理 | 类似于 SQL 的 EXCEPT ALL 子句。MinusAll 返回仅存在于左表,不存在于右表中的数据。如果一条数据在左表中出现了 n 次,在右表中出现了 m 次,最终这条数据将会被返回(n - m)次,即按右表中出现的次数来移除数据。两张表的字段和类型必须完全一致。 |
In 批处理 流处理 | 类似于 SQL 的 IN 子句。如果 In 左边表达式的值在给定的子查询结果中则返回 true。子查询的结果必须为单列。此列数据类型必须和表达式一致。 注意: 对于流式查询,这个操作会被替换成一个连接操作和一个分组操作。计算查询结果所需的状态(state)可能会无限增长,具体取决于不重复的输入行的数量。你可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置。 |
OrderBy, Offset & Fetch
Operators | Description |
---|---|
Order By Batch Streaming | Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions. For unbounded tables, this operation requires a sorting on a time attribute or a subsequent fetch operation. |
Offset & Fetch Batch Streaming | Similar to the SQL OFFSET and FETCH clauses. The offset operation limits a (possibly sorted) result from an offset position. The fetch operation limits a (possibly sorted) result to the first n rows. Usually, the two operations are preceded by an ordering operator. For unbounded tables, a fetch operation is required for an offset operation. |
Operators | Description |
---|---|
Order By Batch Streaming | Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions. For unbounded tables, this operation requires a sorting on a time attribute or a subsequent fetch operation. |
Offset & Fetch Batch Streaming Result Updating | Similar to the SQL OFFSET and FETCH clauses. The offset operation limits a (possibly sorted) result from an offset position. The fetch operation limits a (possibly sorted) result to the first n rows. Usually, the two operations are preceded by an ordering operator. For unbounded tables, a fetch operation is required for an offset operation. |
操作 | 描述 |
---|---|
Order By 批处理 | 类似于 SQL 的 ORDER BY 子句。返回包括所有子并发分区内所有数据的全局排序结果。 |
Offset & Fetch 批处理 | 类似于 SQL 的 OFFSET 和 FETCH 子句。Offset 和 Fetch 从已排序的结果中返回指定数量的数据。Offset 和 Fetch 在技术上是 Order By 操作的一部分,因此必须紧跟其后出现。 |
Insert
Operators | Description |
---|---|
Insert Into Batch Streaming | Similar to the 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. |
Operators | Description |
---|---|
Insert Into Batch Streaming | Similar to the 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. |
操作 | 描述 |
---|---|
Insert Into 批处理 流处理 | 类似于 SQL 请求中的 INSERT INTO 子句。将数据输出到一个已注册的输出表中。 输出表必须先在 TableEnvironment 中注册(详见注册一个 TableSink)。此外,注册的表的模式(schema)必须和请求的结果的模式(schema)相匹配。 |
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(GroupWindow w)
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.
Table table = input
.window([GroupWindow w].as("w")) // define window with alias w
.groupBy($("w")) // group the table by window w
.select($("b").sum()); // aggregate
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.
val table = input
.window([w: GroupWindow] as $"w") // define window with alias w
.groupBy($"w") // group the table by window w
.select($"b".sum) // aggregate
Windows are defined using the window(w: GroupWindow)
clause and require an alias, which is specified using the alias
clause. In order to group a table by a window, the window alias must be referenced in the group_by(...)
clause like a regular grouping attribute. The following example shows how to define a window aggregation on a table.
# define window with alias w, group the table by window w, then aggregate
table = input.window([w: GroupWindow].alias("w")) \
.group_by(col('w')).select(input.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.
Table table = input
.window([GroupWindow w].as("w")) // define window with alias w
.groupBy($("w"), $("a")) // group the table by attribute a and window w
.select($("a"), $("b").sum()); // aggregate
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.
val table = input
.window([w: GroupWindow] as $"w") // define window with alias w
.groupBy($"w", $"a") // group the table by attribute a and window w
.select($"a", $"b".sum) // aggregate
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 group_by(...)
clause references a window alias and at least one additional attribute. A group_by(...)
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.
# define window with alias w, group the table by attribute a and window w,
# then aggregate
table = input.window([w: GroupWindow].alias("w")) \
.group_by(col('w'), input.a).select(input.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.
Table table = input
.window([GroupWindow w].as("w")) // define window with alias w
.groupBy($("w"), $("a")) // group the table by attribute a and window w
.select($("a"), $("w").start(), $("w").end(), $("w").rowtime(), $("b").count()); // aggregate and add window start, end, and rowtime timestamps
val table = input
.window([w: GroupWindow] as $"w") // define window with alias w
.groupBy($"w", $"a") // group the table by attribute a and window w
.select($"a", $"w".start, $"w".end, $"w".rowtime, $"b".count) // aggregate and add window start, end, and rowtime timestamps
# define window with alias w, group the table by attribute a and window w,
# then aggregate and add window start, end, and rowtime timestamps
table = input.window([w: GroupWindow].alias("w")) \
.group_by(col('w'), input.a) \
.select(input.a, col('w').start, col('w').end, col('w').rowtime, input.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:
Method | Description |
---|---|
over | Defines the length the window, either as time or row-count interval. |
on | The 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. |
as | Assigns 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. |
// Tumbling Event-time Window
.window(Tumble.over(lit(10).minutes()).on($("rowtime")).as("w"));
// Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(lit(10).minutes()).on($("proctime")).as("w"));
// Tumbling Row-count Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(rowInterval(10)).on($("proctime")).as("w"));
Tumbling windows are defined by using the Tumble
class as follows:
Method | Description |
---|---|
over | Defines the length the window, either as time or row-count interval. |
on | The 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. |
as | Assigns 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. |
// Tumbling Event-time Window
.window(Tumble over 10.minutes on $"rowtime" as $"w")
// Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
.window(Tumble over 10.minutes on $"proctime" as $"w")
// Tumbling Row-count Window (assuming a processing-time attribute "proctime")
.window(Tumble over 10.rows on $"proctime" as $"w")
Tumbling windows are defined by using the Tumble
class as follows:
Method | Description |
---|---|
over | Defines the length the window, either as time or row-count interval. |
on | The 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. |
alias | Assigns an alias to the window. The alias is used to reference the window in the following group_by() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause. |
# Tumbling Event-time Window
.window(Tumble.over(lit(10).minutes).on(col('rowtime')).alias("w"))
# Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(lit(10).minutes).on(col('proctime')).alias("w"))
# Tumbling Row-count Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(row_interval(10)).on(col('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:
Method | Description |
---|---|
over | Defines the length of the window, either as time or row-count interval. |
every | Defines the slide interval, either as time or row-count interval. The slide interval must be of the same type as the size interval. |
on | The 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. |
as | Assigns 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. |
// Sliding Event-time Window
.window(Slide.over(lit(10).minutes())
.every(lit(5).minutes())
.on($("rowtime"))
.as("w"));
// Sliding Processing-time window (assuming a processing-time attribute "proctime")
.window(Slide.over(lit(10).minutes())
.every(lit(5).minutes())
.on($("proctime"))
.as("w"));
// Sliding Row-count window (assuming a processing-time attribute "proctime")
.window(Slide.over(rowInterval(10)).every(rowInterval(5)).on($("proctime")).as("w"));
Sliding windows are defined by using the Slide
class as follows:
Method | Description |
---|---|
over | Defines the length of the window, either as time or row-count interval. |
every | Defines the slide interval, either as time or row-count interval. The slide interval must be of the same type as the size interval. |
on | The 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. |
as | Assigns 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. |
// Sliding Event-time Window
.window(Slide over 10.minutes every 5.minutes on $"rowtime" as $"w")
// Sliding Processing-time window (assuming a processing-time attribute "proctime")
.window(Slide over 10.minutes every 5.minutes on $"proctime" as $"w")
// Sliding Row-count window (assuming a processing-time attribute "proctime")
.window(Slide over 10.rows every 5.rows on $"proctime" as $"w")
Sliding windows are defined by using the Slide
class as follows:
Method | Description |
---|---|
over | Defines the length of the window, either as time or row-count interval. |
every | Defines the slide interval, either as time or row-count interval. The slide interval must be of the same type as the size interval. |
on | The 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. |
alias | Assigns an alias to the window. The alias is used to reference the window in the following group_by() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause. |
# Sliding Event-time Window
.window(Slide.over(lit(10).minutes).every(lit(5).minutes).on(col('rowtime')).alias("w"))
# Sliding Processing-time window (assuming a processing-time attribute "proctime")
.window(Slide.over(lit(10).minutes).every(lit(5).minutes).on(col('proctime')).alias("w"))
# Sliding Row-count window (assuming a processing-time attribute "proctime")
.window(Slide.over(row_interval(10)).every(row_interval(5)).on(col('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:
Method | Description |
---|---|
withGap | Defines the gap between two windows as time interval. |
on | The 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. |
as | Assigns 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. |
// Session Event-time Window
.window(Session.withGap(lit(10).minutes()).on($("rowtime")).as("w"));
// Session Processing-time Window (assuming a processing-time attribute "proctime")
.window(Session.withGap(lit(10).minutes()).on($("proctime")).as("w"));
A session window is defined by using the Session
class as follows:
Method | Description |
---|---|
withGap | Defines the gap between two windows as time interval. |
on | The 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. |
as | Assigns 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. |
// Session Event-time Window
.window(Session withGap 10.minutes on $"rowtime" as $"w")
// Session Processing-time Window (assuming a processing-time attribute "proctime")
.window(Session withGap 10.minutes on $"proctime" as $"w")
A session window is defined by using the Session
class as follows:
Method | Description |
---|---|
with_gap | Defines the gap between two windows as time interval. |
on | The 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. |
alias | Assigns an alias to the window. The alias is used to reference the window in the following group_by() clause and optionally to select window properties such as window start, end, or rowtime timestamps in the select() clause. |
# Session Event-time Window
.window(Session.with_gap(lit(10).minutes).on(col('rowtime')).alias("w"))
# Session Processing-time Window (assuming a processing-time attribute "proctime")
.window(Session.with_gap(lit(10).minutes).on(col('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.
Table table = input
.window([OverWindow w].as("w")) // define over window with alias w
.select($("a"), $("b").sum().over($("w")), $("c").min().over($("w"))); // aggregate over the over window w
val table = input
.window([w: OverWindow] as $"w") // define over window with alias w
.select($"a", $"b".sum over $"w", $"c".min over $"w") // aggregate over the over window w
# define over window with alias w and aggregate over the over window w
table = input.over_window([w: OverWindow].alias("w")) \
.select(input.a, input.b.sum.over(col('w')), input.c.min.over(col('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:
Method | Required | Description |
---|---|---|
partitionBy | Optional | 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 |
orderBy | Required | 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. |
preceding | Optional | 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., Unbounded over windows are specified using a constant, i.e., If the |
following | Optional | 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:
If the |
as | Required | Assigns an alias to the over window. The alias is used to reference the over window in the following |
Method | Required | Description |
---|---|---|
partition_by | Optional | 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 |
order_by | Required | 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. |
preceding | Optional | 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., Unbounded over windows are specified using a constant, i.e., If the |
following | Optional | 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:
If the |
alias | Required | Assigns an alias to the over window. The alias is used to reference the over window in the following |
Note: Currently, all aggregation functions in the same select()
call must be computed of the same over window.
Unbounded Over Windows
// Unbounded Event-time over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(UNBOUNDED_RANGE).as("w"));
// Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
.window(Over.partitionBy($("a")).orderBy("proctime").preceding(UNBOUNDED_RANGE).as("w"));
// Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(UNBOUNDED_ROW).as("w"));
// Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(UNBOUNDED_ROW).as("w"));
// Unbounded Event-time over window (assuming an event-time attribute "rowtime")
.window(Over partitionBy $"a" orderBy $"rowtime" preceding UNBOUNDED_RANGE as "w")
// Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
.window(Over partitionBy $"a" orderBy $"proctime" preceding UNBOUNDED_RANGE as "w")
// Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over partitionBy $"a" orderBy $"rowtime" preceding UNBOUNDED_ROW as "w")
// Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.window(Over partitionBy $"a" orderBy $"proctime" preceding UNBOUNDED_ROW as "w")
# Unbounded Event-time over window (assuming an event-time attribute "rowtime")
.over_window(Over.partition_by(col('a')).order_by(col('rowtime')).preceding(UNBOUNDED_RANGE).alias("w"))
# Unbounded Processing-time over window (assuming a processing-time attribute "proctime")
.over_window(Over.partition_by(col('a')).order_by(col('proctime')).preceding(UNBOUNDED_RANGE).alias("w"))
# Unbounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.over_window(Over.partition_by(col('a')).order_by(col('rowtime')).preceding(UNBOUNDED_ROW).alias("w"))
# Unbounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.over_window(Over.partition_by(col('a')).order_by(col('proctime')).preceding(UNBOUNDED_ROW).alias("w"))
Bounded Over Windows
// Bounded Event-time over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(lit(1).minutes()).as("w"))
// Bounded Processing-time over window (assuming a processing-time attribute "proctime")
.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(lit(1).minutes()).as("w"))
// Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(rowInterval(10)).as("w"))
// Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(rowInterval(10)).as("w"))
// Bounded Event-time over window (assuming an event-time attribute "rowtime")
.window(Over partitionBy $"a" orderBy $"rowtime" preceding 1.minutes as "w")
// Bounded Processing-time over window (assuming a processing-time attribute "proctime")
.window(Over partitionBy $"a" orderBy $"proctime" preceding 1.minutes as "w")
// Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over partitionBy $"a" orderBy $"rowtime" preceding 10.rows as "w")
// Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.window(Over partitionBy $"a" orderBy $"proctime" preceding 10.rows as "w")
# Bounded Event-time over window (assuming an event-time attribute "rowtime")
.over_window(Over.partition_by(col('a')).order_by(col('rowtime')).preceding(lit(1).minutes).alias("w"))
# Bounded Processing-time over window (assuming a processing-time attribute "proctime")
.over_window(Over.partition_by(col('a')).order_by(col('proctime')).preceding(lit(1).minutes).alias("w"))
# Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.over_window(Over.partition_by(col('a')).order_by(col('rowtime')).preceding(row_interval(10)).alias("w"))
# Bounded Processing-time Row-count over window (assuming a processing-time attribute "proctime")
.over_window(Over.partition_by(col('a')).order_by(col('proctime')).preceding(row_interval(10)).alias("w"))
Row-based Operations
The row-based operations generate outputs with multiple columns.
Operators | Description |
---|---|
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. |
FlatMap Batch Streaming | Performs a flatMap operation with a table function. |
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. |
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. |
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 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. |
Operators | Description |
---|---|
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. |
FlatMap Batch Streaming | Performs a flatMap operation with a table function. |
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. |
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. |
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 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. |
Operators | Description |
---|---|
Map Batch Streaming | Python Table API 暂不支持。 |
FlatMap Batch Streaming | Python Table API 暂不支持。 |
Aggregate Batch Streaming Result Updating | Python Table API 暂不支持。 |
Group Window Aggregate Batch Streaming | Python Table API 暂不支持。 |
FlatAggregate Streaming Result Updating | Python Table API 暂不支持。 |
Group Window FlatAggregate Streaming | Python Table API 暂不支持。 |
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:
expressionList = expression , { "," , expression } ;
expression = overConstant | alias ;
alias = logic | ( logic , "as" , fieldReference ) | ( logic , "as" , "(" , fieldReference , { "," , fieldReference } , ")" ) ;
logic = comparison , [ ( "&&" | "||" ) , comparison ] ;
comparison = term , [ ( "=" | "==" | "===" | "!=" | "!==" | ">" | ">=" | "<" | "<=" ) , term ] ;
term = product , [ ( "+" | "-" ) , product ] ;
product = unary , [ ( "*" | "/" | "%") , unary ] ;
unary = [ "!" | "-" | "+" ] , composite ;
composite = over | suffixed | nullLiteral | prefixed | atom ;
suffixed = interval | suffixAs | suffixCast | suffixIf | suffixDistinct | suffixFunctionCall | timeIndicator ;
prefixed = prefixAs | prefixCast | prefixIf | prefixDistinct | prefixFunctionCall ;
interval = timeInterval | rowInterval ;
timeInterval = composite , "." , ("year" | "years" | "quarter" | "quarters" | "month" | "months" | "week" | "weeks" | "day" | "days" | "hour" | "hours" | "minute" | "minutes" | "second" | "seconds" | "milli" | "millis") ;
rowInterval = composite , "." , "rows" ;
suffixCast = composite , ".cast(" , dataType , ")" ;
prefixCast = "cast(" , expression , dataType , ")" ;
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 , ")" ) ;
suffixAs = composite , ".as(" , fieldReference , ")" ;
prefixAs = "as(" , expression, fieldReference , ")" ;
suffixIf = composite , ".?(" , expression , "," , expression , ")" ;
prefixIf = "?(" , expression , "," , expression , "," , expression , ")" ;
suffixDistinct = composite , "distinct.()" ;
prefixDistinct = functionIdentifier , ".distinct" , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
suffixFunctionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
prefixFunctionCall = functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ;
atom = ( "(" , expression , ")" ) | literal | fieldReference ;
fieldReference = "*" | identifier ;
nullLiteral = "nullOf(" , dataType , ")" ;
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" ;
timePointUnit = "YEAR" | "MONTH" | "DAY" | "HOUR" | "MINUTE" | "SECOND" | "QUARTER" | "WEEK" | "MILLISECOND" | "MICROSECOND" ;
over = composite , "over" , fieldReference ;
overConstant = "current_row" | "current_range" | "unbounded_row" | "unbounded_row" ;
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.