SELECT statements and VALUES statements are specified with the sqlQuery()
method of the TableEnvironment
. The method returns the result of the SELECT statement (or the VALUES statements) as a Table
. A Table
can be used in subsequent SQL and Table API queries, be converted into a DataSet or DataStream, or written to a TableSink. SQL and Table API queries can be seamlessly mixed and are holistically optimized and translated into a single program.
In order to access a table in a SQL query, it must be registered in the TableEnvironment. A table can be registered from a TableSource, Table, CREATE TABLE statement, DataStream, or DataSet. Alternatively, users can also register catalogs in a TableEnvironment to specify the location of the data sources.
For convenience, Table.toString()
automatically registers the table under a unique name in its TableEnvironment
and returns the name. So, Table
objects can be directly inlined into SQL queries as shown in the examples below.
Note: Queries that include unsupported SQL features cause a TableException
. The supported features of SQL on batch and streaming tables are listed in the following sections.
SELECT statements and VALUES statements are specified with the sql_query()
method of the TableEnvironment
. The method returns the result of the SELECT statement (or the VALUES statements) as a Table
. A Table
can be used in subsequent SQL and Table API queries or written to a TableSink. SQL and Table API queries can be seamlessly mixed and are holistically optimized and translated into a single program.
In order to access a table in a SQL query, it must be registered in the TableEnvironment. A table can be registered from a TableSource, Table, CREATE TABLE statement. Alternatively, users can also register catalogs in a TableEnvironment to specify the location of the data sources.
For convenience, str(Table)
automatically registers the table under a unique name in its TableEnvironment
and returns the name. So, Table
objects can be directly inlined into SQL queries as shown in the examples below.
Note: Queries that include unsupported SQL features cause a TableException
. The supported features of SQL on batch and streaming tables are listed in the following sections.
The following examples show how to specify a SQL queries on registered and inlined tables.
A SELECT statement or a VALUES statement can be executed to collect the content to local through the TableEnvironment.executeSql()
method. The method returns the result of the SELECT statement (or the VALUES statement) as a TableResult
. Similar to a SELECT statement, a Table
object can be executed using the Table.execute()
method to collect the content of the query to the local client.
TableResult.collect()
method returns a closeable row iterator. The select job will not be finished unless all result data has been collected. We should actively close the job to avoid resource leak through the CloseableIterator#close()
method.
We can also print the select result to client console through the TableResult.print()
method. The result data in TableResult
can be accessed only once. Thus, collect()
and print()
must not be called after each other.
TableResult.collect()
and TableResult.print()
have slightly different behaviors under different checkpointing settings (to enable checkpointing for a streaming job, see checkpointing config).
TableResult.collect()
and TableResult.print()
have neither exactly-once nor at-least-once guarantee. Query results are immediately accessible by the clients once they’re produced, but exceptions will be thrown when the job fails and restarts.TableResult.collect()
and TableResult.print()
guarantee an end-to-end exactly-once record delivery. A result will be accessible by clients only after its corresponding checkpoint completes.TableResult.collect()
and TableResult.print()
guarantee an end-to-end at-least-once record delivery. Query results are immediately accessible by the clients once they’re produced, but it is possible for the same result to be delivered multiple times.A SELECT statement or a VALUES statement can be executed to collect the content to local through the TableEnvironment.execute_sql()
method. The method returns the result of the SELECT statement (or the VALUES statement) as a TableResult
. Similar to a SELECT statement, a Table
object can be executed using the Table.execute()
method to collect the content of the query to the local client.
TableResult.collect()
method returns a closeable row iterator. The select job will not be finished unless all result data has been collected. We should actively close the job to avoid resource leak through the CloseableIterator#close()
method.
We can also print the select result to client console through the TableResult.print()
method. The result data in TableResult
can be accessed only once. Thus, collect()
and print()
must not be called after each other.
TableResult.collect()
and TableResult.print()
have slightly different behaviors under different checkpointing settings (to enable checkpointing for a streaming job, see checkpointing config).
TableResult.collect()
and TableResult.print()
have neither exactly-once nor at-least-once guarantee. Query results are immediately accessible by the clients once they’re produced, but exceptions will be thrown when the job fails and restarts.TableResult.collect()
and TableResult.print()
guarantee an end-to-end exactly-once record delivery. A result will be accessible by clients only after its corresponding checkpoint completes.TableResult.collect()
and TableResult.print()
guarantee an end-to-end at-least-once record delivery. Query results are immediately accessible by the clients once they’re produced, but it is possible for the same result to be delivered multiple times.Flink parses SQL using Apache Calcite, which supports standard ANSI SQL.
The following BNF-grammar describes the superset of supported SQL features in batch and streaming queries. The Operations section shows examples for the supported features and indicates which features are only supported for batch or streaming queries.
Flink SQL uses a lexical policy for identifier (table, attribute, function names) similar to Java:
"SELECT a AS `my field` FROM t"
).String literals must be enclosed in single quotes (e.g., SELECT 'Hello World'
). Duplicate a single quote for escaping (e.g., SELECT 'It''s me.'
). Unicode characters are supported in string literals. If explicit unicode code points are required, use the following syntax:
\
) as escaping character (default): SELECT U&'\263A'
SELECT U&'#263A' UESCAPE '#'
Operation | Description |
---|---|
Scan / Select / As Batch Streaming |
|
Where / Filter Batch Streaming |
|
User-defined Scalar Functions (Scalar UDF) Batch Streaming |
UDFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register scalar UDFs. |
Operation | Description |
---|---|
GroupBy Aggregation Batch Streaming Result Updating |
Note: GroupBy on a streaming table produces an updating result. See the Dynamic Tables Streaming Concepts page for details. |
GroupBy Window Aggregation Batch Streaming |
Use a group window to compute a single result row per group. See Group Windows section for more details. |
Over Window aggregation Streaming |
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 Batch Streaming Result Updating |
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. |
Grouping sets, Rollup, Cube Batch Streaming Result Updating |
Note: Streaming mode Grouping sets, Rollup and Cube are only supported in Blink planner. |
Having Batch Streaming |
|
User-defined Aggregate Functions (UDAGG) Batch Streaming |
UDAGGs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDAGGs. |
Operation | Description |
---|---|
Inner Equi-join Batch Streaming |
Currently, only equi-joins are supported, i.e., joins that have at least one conjunctive condition with an equality predicate. Arbitrary cross or theta joins are not supported. Note: The order of joins is not optimized. Tables are joined in the order in which they are specified in the FROM clause. Make sure to specify tables in an order that does not yield a cross join (Cartesian product) which are not supported and would cause a query to fail. 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 Equi-join Batch Streaming Result Updating |
Currently, only equi-joins are supported, i.e., joins that have at least one conjunctive condition with an equality predicate. Arbitrary cross or theta joins are not supported. Note: The order of joins is not optimized. Tables are joined in the order in which they are specified in the FROM clause. Make sure to specify tables in an order that does not yield a cross join (Cartesian product) which are not supported and would cause a query to fail. 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. A 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:
|
Expanding arrays into a relation Batch Streaming |
Unnesting WITH ORDINALITY is not supported yet. |
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. User-defined table functions (UDTFs) must be registered before. See the UDF documentation for details on how to specify and register UDTFs. Inner Join A row of the left (outer) table is dropped, if its table function call returns an empty result. Left Outer Join If a table function call returns an empty result, the corresponding outer row is preserved and the result padded with null values. Note: Currently, only literal |
Join with Temporal Table Batch Streaming |
Temporal Tables are tables that track changes over time. A Temporal Table provides access to the versions of a temporal table at a specific point in time. Processing-time temporal join and event-time temporal join are supported, inner join and left join are supported. The event-time temporal join is not suppored in Batch The following example assumes that LatestRates is a Temporal Table which is materialized with the latest rate. The RHS table can be named with an alias using optional clause For more information please check the more detailed Temporal Tables concept description. Only supported in Blink planner. |
Join with Temporal Table Function 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 Join with Table Function. Note: Currently only inner joins with temporal tables are supported. Assuming Rates is a temporal table function, the join can be expressed in SQL as follows: For more information please check the more detailed temporal tables concept description. |
Operation | Description |
---|---|
Union Batch |
|
UnionAll Batch Streaming |
|
Intersect / Except Batch |
|
In Batch Streaming |
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. |
Exists Batch Streaming |
Returns true if the sub-query returns at least one row. Only supported if the operation can be rewritten in a join and group operation. 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. |
Operation | Description |
---|---|
Order By Batch Streaming |
Note: The result of streaming queries must be primarily sorted on an ascending time attribute. Additional sorting attributes are supported. |
Limit Batch |
Note: The LIMIT clause requires an ORDER BY clause. |
Attention Top-N is only supported in Blink planner.
Top-N queries ask for the N smallest or largest values ordered by columns. Both smallest and largest values sets are considered Top-N queries. Top-N queries are useful in cases where the need is to display only the N bottom-most or the N top- most records from batch/streaming table on a condition. This result set can be used for further analysis.
Flink uses the combination of a OVER window clause and a filter condition to express a Top-N query. With the power of OVER window PARTITION BY
clause, Flink also supports per group Top-N. For example, the top five products per category that have the maximum sales in realtime. Top-N queries are supported for SQL on batch and streaming tables.
The following shows the syntax of the TOP-N statement:
Parameter Specification:
ROW_NUMBER()
: Assigns an unique, sequential number to each row, starting with one, according to the ordering of rows within the partition. Currently, we only support ROW_NUMBER
as the over window function. In the future, we will support RANK()
and DENSE_RANK()
.PARTITION BY col1[, col2...]
: Specifies the partition columns. Each partition will have a Top-N result.ORDER BY col1 [asc|desc][, col2 [asc|desc]...]
: Specifies the ordering columns. The ordering directions can be different on different columns.WHERE rownum <= N
: The rownum <= N
is required for Flink to recognize this query is a Top-N query. The N represents the N smallest or largest records will be retained.[AND conditions]
: It is free to add other conditions in the where clause, but the other conditions can only be combined with rownum <= N
using AND
conjunction.Attention in Streaming Mode The TopN query is Result Updating. Flink SQL will sort the input data stream according to the order key, so if the top N records have been changed, the changed ones will be sent as retraction/update records to downstream. It is recommended to use a storage which supports updating as the sink of Top-N query. In addition, if the top N records need to be stored in external storage, the result table should have the same unique key with the Top-N query.
The unique keys of Top-N query is the combination of partition columns and rownum column. Top-N query can also derive the unique key of upstream. Take following job as an example, say product_id
is the unique key of the ShopSales
, then the unique keys of the Top-N query are [category
, rownum
] and [product_id
].
The following examples show how to specify SQL queries with Top-N on streaming tables. This is an example to get “the top five products per category that have the maximum sales in realtime” we mentioned above.
As described above, the rownum
field will be written into the result table as one field of the unique key, which may lead to a lot of records being written to the result table. For example, when the record (say product-1001
) of ranking 9 is updated and its rank is upgraded to 1, all the records from ranking 1 ~ 9 will be output to the result table as update messages. If the result table receives too many data, it will become the bottleneck of the SQL job.
The optimization way is omitting rownum field in the outer SELECT clause of the Top-N query. This is reasonable because the number of the top N records is usually not large, thus the consumers can sort the records themselves quickly. Without rownum field, in the example above, only the changed record (product-1001
) needs to be sent to downstream, which can reduce much IO to the result table.
The following example shows how to optimize the above Top-N example in this way:
Attention in Streaming Mode In order to output the above query to an external storage and have a correct result, the external storage must have the same unique key with the Top-N query. In the above example query, if the product_id
is the unique key of the query, then the external table should also has product_id
as the unique key.
Attention Deduplication is only supported in Blink planner.
Deduplication is removing rows that duplicate over a set of columns, keeping only the first one or the last one. In some cases, the upstream ETL jobs are not end-to-end exactly-once, this may result in there are duplicate records in the sink in case of failover. However, the duplicate records will affect the correctness of downstream analytical jobs (e.g. SUM
, COUNT
). So a deduplication is needed before further analysis.
Flink uses ROW_NUMBER()
to remove duplicates just like the way of Top-N query. In theory, deduplication is a special case of Top-N which the N is one and order by the processing time or event time.
The following shows the syntax of the Deduplication statement:
Parameter Specification:
ROW_NUMBER()
: Assigns an unique, sequential number to each row, starting with one.PARTITION BY col1[, col2...]
: Specifies the partition columns, i.e. the deduplicate key.ORDER BY time_attr [asc|desc]
: Specifies the ordering column, it must be a time attribute. Currently Flink supports processing time attribute and event time atttribute. Ordering by ASC means keeping the first row, ordering by DESC means keeping the last row.WHERE rownum = 1
: The rownum = 1
is required for Flink to recognize this query is deduplication.The following examples show how to specify SQL queries with Deduplication on streaming tables.
Deduplication can keep the time attribute of input stream, this is very helpful when the downstream operation is window aggregation or join operation. Both processing-time deduplication and event-time deduplication support working on mini-batch mode, this is more performance friendly, please see also mini-batch configuration for how to enable mini-batch mode..
Group windows are defined in the GROUP BY
clause of a SQL query. Just like queries with regular GROUP BY
clauses, queries with a GROUP BY
clause that includes a group window function compute a single result row per group. The following group windows functions are supported for SQL on batch and streaming tables.
Group Window Function | Description |
---|---|
TUMBLE(time_attr, interval) |
Defines a tumbling time window. A tumbling time window assigns rows to non-overlapping, continuous windows with a fixed duration (interval ). For example, a tumbling window of 5 minutes groups rows in 5 minutes intervals. Tumbling windows can be defined on event-time (stream + batch) or processing-time (stream). |
HOP(time_attr, interval, interval) |
Defines a hopping time window (called sliding window in the Table API). A hopping time window has a fixed duration (second interval parameter) and hops by a specified hop interval (first interval parameter). If the hop interval is smaller than the window size, hopping windows are overlapping. Thus, rows can be assigned to multiple windows. For example, a hopping window of 15 minutes size and 5 minute hop interval assigns each row to 3 different windows of 15 minute size, which are evaluated in an interval of 5 minutes. Hopping windows can be defined on event-time (stream + batch) or processing-time (stream). |
SESSION(time_attr, interval) |
Defines a session time window. Session time windows do not have a fixed duration but their bounds are defined by a time interval of inactivity, i.e., a session window is closed 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 (stream + batch) or processing-time (stream). |
For SQL queries on streaming tables, the time_attr
argument of the group window function must refer to a valid time attribute that specifies the processing time or event time of rows. See the documentation of time attributes to learn how to define time attributes.
For SQL on batch tables, the time_attr
argument of the group window function must be an attribute of type TIMESTAMP
.
The start and end timestamps of group windows as well as time attributes can be selected with the following auxiliary functions:
Auxiliary Function | Description |
---|---|
TUMBLE_START(time_attr, interval) HOP_START(time_attr, interval, interval) SESSION_START(time_attr, interval) |
Returns the timestamp of the inclusive lower bound of the corresponding tumbling, hopping, or session window. |
TUMBLE_END(time_attr, interval) HOP_END(time_attr, interval, interval) SESSION_END(time_attr, interval) |
Returns the timestamp of the exclusive upper bound of the corresponding tumbling, hopping, or session window. Note: The exclusive upper bound timestamp cannot be used as a rowtime attribute in subsequent time-based operations, such as interval joins and group window or over window aggregations. |
TUMBLE_ROWTIME(time_attr, interval) HOP_ROWTIME(time_attr, interval, interval) SESSION_ROWTIME(time_attr, interval) |
Returns the timestamp of the inclusive upper bound of the corresponding tumbling, hopping, or session window. The resulting attribute is a rowtime attribute that can be used in subsequent time-based operations such as interval joins and group window or over window aggregations. |
TUMBLE_PROCTIME(time_attr, interval) HOP_PROCTIME(time_attr, interval, interval) SESSION_PROCTIME(time_attr, interval) |
Returns a proctime attribute that can be used in subsequent time-based operations such as interval joins and group window or over window aggregations. |
Note: Auxiliary functions must be called with exactly same arguments as the group window function in the GROUP BY
clause.
The following examples show how to specify SQL queries with group windows on streaming tables.
Operation | Description |
---|---|
MATCH_RECOGNIZE Streaming |
Searches for a given pattern in a streaming table according to the For a more detailed description, see the dedicated page for detecting patterns in tables. |