--- title: "Table API" nav-parent_id: tableapi nav-pos: 20 --- 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]({{ site.baseurl }}/dev/table/common.html) to learn how to register tables or to create a `Table` object. The [Streaming Concepts](./streaming) 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](./streaming/time_attributes.html) in streaming or a regular timestamp field in batch. * This will be replaced by the TOC {:toc} 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. {% highlight java %} // environment configuration ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv = BatchTableEnvironment.create(env); // register Orders table in table environment // ... // specify table program Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) Table counts = orders .groupBy("a") .select("a, b.count as cnt"); // conversion to DataSet DataSet result = tEnv.toDataSet(counts, Row.class); result.print(); {% endhighlight %}
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](http://scala-lang.org/files/archive/spec/2.12/01-lexical-syntax.html#symbol-literals), which start with an apostrophe character (`'`). {% highlight scala %} import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ // environment configuration val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = BatchTableEnvironment.create(env) // register Orders table in table environment // ... // specify table program val orders = tEnv.scan("Orders") // schema (a, b, c, rowtime) val result = orders .groupBy('a) .select('a, 'b.count as 'cnt) .toDataSet[Row] // conversion to DataSet .print() {% endhighlight %}
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`.
{% highlight java %} // environment configuration // ... // specify table program Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) Table result = 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"); {% endhighlight %}
{% highlight scala %} // environment configuration // ... // specify table program val orders: Table = tEnv.scan("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) {% endhighlight %}
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](streaming) for details). {% top %} Operations ---------- The Table API supports the following operations. Please note that not all operations are available in both batch and streaming yet; they are tagged accordingly. ### Scan, Projection, and Filter
Operators Description
Scan
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); {% endhighlight %}
Select
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.select("a, c as d"); {% endhighlight %}

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

{% highlight java %} Table result = orders.select("*"); {% endhighlight %}
As
Batch Streaming

Renames fields.

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.as("x, y, z, t"); {% endhighlight %}
Where / Filter
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.where("b === 'red'"); {% endhighlight %} or {% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.filter("a % 2 === 0"); {% endhighlight %}
Operators Description
Scan
Batch Streaming

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

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") {% endhighlight %}
Select
Batch Streaming

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

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.select('a, 'c as 'd) {% endhighlight %}

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

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.select('*) {% endhighlight %}
As
Batch Streaming

Renames fields.

{% highlight scala %} val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't) {% endhighlight %}
Where / Filter
Batch Streaming

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

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.filter('a % 2 === 0) {% endhighlight %} or {% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.where('b === "red") {% endhighlight %}
操作 描述
Scan
批处理 流处理

类似于SQL请求中的FROM子句,将一个环境中已注册的表转换成Table对象。

{% highlight python %} orders = table_env.scan("Orders"); {% endhighlight %}
Select
批处理 流处理

类似于SQL请求中的SELECT子句,执行一个select操作。

{% highlight python %} orders = table_env.scan("Orders"); result = orders.select("a, c as d"); {% endhighlight %}

您可以使用星号 (*) 表示选择表中的所有列。

{% highlight python %} result = orders.select("*"); {% endhighlight %}
Alias
批处理 流处理

重命名字段。

{% highlight python %} orders = table_env.scan("Orders"); result = orders.alias("x, y, z, t"); {% endhighlight %}
Where / Filter
批处理 流处理

类似于SQL请求中的WHERE子句,过滤掉表中不满足条件的行。

{% highlight python %} orders = table_env.scan("Orders"); result = orders.where("b === 'red'"); {% endhighlight %} or {% highlight python %} orders = table_env.scan("Orders"); result = orders.filter("a % 2 === 0"); {% endhighlight %}
{% top %} ### Column Operations
Operators Description
AddColumns
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.addColumns("concat(c, 'sunny')"); {% endhighlight %}
AddOrReplaceColumns
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc"); {% endhighlight %}
DropColumns
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.dropColumns("b, c"); {% endhighlight %}
RenameColumns
Batch Streaming

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

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.renameColumns("b as b2, c as c2"); {% endhighlight %}
Operators Description
AddColumns
Batch Streaming

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

{% highlight scala %} val orders = tableEnv.scan("Orders"); val result = orders.addColumns(concat('c, "Sunny")) {% endhighlight %}
AddOrReplaceColumns
Batch Streaming

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

{% highlight scala %} val orders = tableEnv.scan("Orders"); val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc) {% endhighlight %}
DropColumns
Batch Streaming

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

{% highlight scala %} val orders = tableEnv.scan("Orders"); val result = orders.dropColumns('b, 'c) {% endhighlight %}
RenameColumns
Batch Streaming

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

{% highlight scala %} val orders = tableEnv.scan("Orders"); val result = orders.renameColumns('b as 'b2, 'c as 'c2) {% endhighlight %}
{% top %} ### 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.

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.groupBy("a").select("a, b.sum as d"); {% endhighlight %}

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.

{% highlight java %} 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 {% endhighlight %}
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.

{% highlight java %} 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 {% endhighlight %}

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.

{% highlight java %} Table orders = tableEnv.scan("Orders"); // Distinct aggregation on group by Table groupByDistinctResult = orders .groupBy("a") .select("a, b.sum.distinct as d"); // Distinct aggregation on time window group by Table 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 window Table 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"); {% endhighlight %}

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.

{% highlight java %} Table orders = tEnv.scan("Orders"); // Use distinct aggregation for user-defined aggregate functions tEnv.registerFunction("myUdagg", new MyUdagg()); orders.groupBy("users").select("users, myUdagg.distinct(points) as myDistinctResult"); {% endhighlight %}

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.

{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders.distinct(); {% endhighlight %}

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.

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.

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.groupBy('a).select('a, 'b.sum as 'd) {% endhighlight %}

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.

{% highlight scala %} 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 {% endhighlight %}
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.

{% highlight scala %} 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 {% endhighlight %}

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.

{% highlight scala %} val orders: Table = tableEnv.scan("Orders"); // Distinct aggregation on group by val groupByDistinctResult = orders .groupBy('a) .select('a, 'b.sum.distinct as 'd) // Distinct aggregation on time window group by val 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 window val 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) {% endhighlight %}

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.

{% highlight scala %} val orders: Table = tEnv.scan("Orders"); // Use distinct aggregation for user-defined aggregate functions val myUdagg = new MyUdagg(); orders.groupBy('users).select('users, myUdagg.distinct('points) as 'myDistinctResult); {% endhighlight %}

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

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

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") val result = orders.distinct() {% endhighlight %}

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.

{% top %} ### 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.

{% highlight java %} 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"); {% endhighlight %}

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.

{% highlight java %} 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"); {% endhighlight %}

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 Join
Batch Streaming

Note: 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
{% highlight java %} 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"); {% endhighlight %}
Inner Join with Table Function
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.

{% highlight java %} // register User-Defined Table Function TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join Table orders = tableEnv.scan("Orders"); Table result = orders .joinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); {% endhighlight %}
Left Outer Join with Table Function
Batch Streaming

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

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

{% highlight java %} // register User-Defined Table Function TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join Table orders = tableEnv.scan("Orders"); Table result = orders .leftOuterJoinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); {% endhighlight %}
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.

{% highlight java %} Table ratesHistory = tableEnv.scan("RatesHistory"); // register temporal table function with a time attribute and primary key TemporalTableFunction rates = ratesHistory.createTemporalTableFunction( "r_proctime", "r_currency"); tableEnv.registerFunction("rates", rates); // join with "Orders" based on the time attribute and key Table orders = tableEnv.scan("Orders"); Table result = orders .joinLateral("rates(o_proctime)", "o_currency = r_currency") {% endhighlight %}

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.

{% highlight scala %} 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) {% endhighlight %}

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.

{% highlight scala %} 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) {% endhighlight %}

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 Join
Batch Streaming

Note: 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
{% highlight scala %} 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) {% endhighlight %}
Inner Join with Table Function
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.

{% highlight scala %} // instantiate User-Defined Table Function val split: TableFunction[_] = new MySplitUDTF() // join val result: Table = table .joinLateral(split('c) as ('s, 't, 'v)) .select('a, 'b, 's, 't, 'v) {% endhighlight %}
Left Outer Join with Table Function
Batch Streaming

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

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

{% highlight scala %} // instantiate User-Defined Table Function val split: TableFunction[_] = new MySplitUDTF() // join val result: Table = table .leftOuterJoinLateral(split('c) as ('s, 't, 'v)) .select('a, 'b, 's, 't, 'v) {% endhighlight %}
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.

{% highlight scala %} val ratesHistory = tableEnv.scan("RatesHistory") // register temporal table function with a time attribute and primary key val rates = ratesHistory.createTemporalTableFunction('r_proctime, 'r_currency) // join with "Orders" based on the time attribute and key val orders = tableEnv.scan("Orders") val result = orders .joinLateral(rates('o_rowtime), 'r_currency === 'o_currency) {% endhighlight %}

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

{% top %} ### 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.

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "a, b, c"); Table result = left.union(right); {% endhighlight %}
UnionAll
Batch Streaming

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

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "a, b, c"); Table result = left.unionAll(right); {% endhighlight %}
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.

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "d, e, f"); Table result = left.intersect(right); {% endhighlight %}
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.

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "d, e, f"); Table result = left.intersectAll(right); {% endhighlight %}
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.

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "a, b, c"); Table result = left.minus(right); {% endhighlight %}
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.

{% highlight java %} Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "a, b, c"); Table result = left.minusAll(right); {% endhighlight %}
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.

{% highlight java %} Table left = ds1.toTable(tableEnv, "a, b, c"); Table right = ds2.toTable(tableEnv, "a"); // using implicit registration Table result = left.select("a, b, c").where("a.in(" + right + ")"); // using explicit registration tableEnv.registerTable("RightTable", right); Table result = left.select("a, b, c").where("a.in(RightTable)"); {% endhighlight %}

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.

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'a, 'b, 'c) val result = left.union(right) {% endhighlight %}
UnionAll
Batch Streaming

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

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'a, 'b, 'c) val result = left.unionAll(right) {% endhighlight %}
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.

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'e, 'f, 'g) val result = left.intersect(right) {% endhighlight %}
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.

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'e, 'f, 'g) val result = left.intersectAll(right) {% endhighlight %}
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.

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'a, 'b, 'c) val result = left.minus(right) {% endhighlight %}
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.

{% highlight scala %} val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'a, 'b, 'c) val result = left.minusAll(right) {% endhighlight %}
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.

{% highlight scala %} 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)) {% endhighlight %}

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.

{% top %} ### OrderBy, Offset & Fetch
Operators Description
Order By
Batch

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

{% highlight java %} Table in = tableEnv.fromDataSet(ds, "a, b, c"); Table result = in.orderBy("a.asc"); {% endhighlight %}
Offset & Fetch
Batch

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

{% highlight java %} Table in = tableEnv.fromDataSet(ds, "a, b, c"); // returns the first 5 records from the sorted result Table result1 = in.orderBy("a.asc").fetch(5); // skips the first 3 records and returns all following records from the sorted result Table result2 = in.orderBy("a.asc").offset(3); // skips the first 10 records and returns the next 5 records from the sorted result Table result3 = in.orderBy("a.asc").offset(10).fetch(5); {% endhighlight %}
Operators Description
Order By
Batch

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

{% highlight scala %} val in = ds.toTable(tableEnv, 'a, 'b, 'c) val result = in.orderBy('a.asc) {% endhighlight %}
Offset & Fetch
Batch

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

{% highlight scala %} val in = ds.toTable(tableEnv, 'a, 'b, 'c) // returns the first 5 records from the sorted result val result1: Table = in.orderBy('a.asc).fetch(5) // skips the first 3 records and returns all following records from the sorted result val result2: Table = in.orderBy('a.asc).offset(3) // skips the first 10 records and returns the next 5 records from the sorted result val result3: Table = in.orderBy('a.asc).offset(10).fetch(5) {% endhighlight %}
### Insert
Operators Description
Insert Into
Batch Streaming

Similar 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.

{% highlight java %} Table orders = tableEnv.scan("Orders"); orders.insertInto("OutOrders"); {% endhighlight %}
Operators Description
Insert Into
Batch Streaming

Similar 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.

{% highlight scala %} val orders: Table = tableEnv.scan("Orders") orders.insertInto("OutOrders") {% endhighlight %}
{% top %} ### Group Windows Group window aggregates group rows into finite groups based on time or row-count intervals and evaluate aggregation functions once per group. For batch tables, windows are a convenient shortcut to group records by time intervals. Windows are defined using the `window(w: GroupWindow)` clause and require an alias, which is specified using the `as` clause. In order to group a table by a window, the window alias must be referenced in the `groupBy(...)` clause like a regular grouping attribute. The following example shows how to define a window aggregation on a table.
{% highlight java %} 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 {% endhighlight %}
{% highlight scala %} 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 {% endhighlight %}
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.
{% highlight java %} 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 {% endhighlight %}
{% highlight scala %} 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 {% endhighlight %}
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.
{% highlight java %} 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 {% endhighlight %}
{% highlight scala %} 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 {% endhighlight %}
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.
{% highlight java %} // 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")); {% endhighlight %}
{% highlight scala %} // 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) {% endhighlight %}
#### 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.
{% highlight java %} // 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")); {% endhighlight %}
{% highlight scala %} // 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) {% endhighlight %}
#### 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.
{% highlight java %} // 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")); {% endhighlight %}
{% highlight scala %} // 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) {% endhighlight %}
{% top %} ### Over Windows Over window aggregates are known from standard SQL (`OVER` clause) and defined in the `SELECT` clause of a query. Unlike group windows, which are specified in the `GROUP BY` clause, over windows do not collapse rows. Instead over window aggregates compute an aggregate for each input row over a range of its neighboring rows. Over windows are defined using the `window(w: OverWindow*)` clause and referenced via an alias in the `select()` method. The following example shows how to define an over window aggregation on a table.
{% highlight java %} 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 {% endhighlight %}
{% highlight scala %} 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 {% endhighlight %}
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 partitionBy(...) the stream is processed by a single, non-parallel task.

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., 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.

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:

  • 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.

as Required

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

**Note:** Currently, all aggregation functions in the same `select()` call must be computed of the same over window. #### Unbounded Over Windows
{% highlight java %} // 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")); {% endhighlight %}
{% highlight scala %} // 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) {% endhighlight %}
#### Bounded Over Windows
{% highlight java %} // 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")) {% endhighlight %}
{% highlight scala %} // 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) {% endhighlight %}
{% top %} ### Row-based Operations
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.

{% highlight java %} ScalarFunction func = new MyMapFunction(); tableEnv.registerFunction("func", func); Table table = input .map(func("c")).as("a, b") {% endhighlight %}
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.

{% highlight scala %} val func: ScalarFunction = new MyMapFunction() val table = input .map(func('c)).as('a, 'b) {% endhighlight %}
{% top %} Data Types ---------- The Table API is built on top of Flink's DataSet and DataStream APIs. Internally, it also uses Flink's `TypeInformation` to define data types. Fully supported types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between Table API types, SQL types, and the resulting Java class. | Table API | SQL | Java type | | :--------------------- | :-------------------------- | :--------------------- | | `Types.STRING` | `VARCHAR` | `java.lang.String` | | `Types.BOOLEAN` | `BOOLEAN` | `java.lang.Boolean` | | `Types.BYTE` | `TINYINT` | `java.lang.Byte` | | `Types.SHORT` | `SMALLINT` | `java.lang.Short` | | `Types.INT` | `INTEGER, INT` | `java.lang.Integer` | | `Types.LONG` | `BIGINT` | `java.lang.Long` | | `Types.FLOAT` | `REAL, FLOAT` | `java.lang.Float` | | `Types.DOUBLE` | `DOUBLE` | `java.lang.Double` | | `Types.DECIMAL` | `DECIMAL` | `java.math.BigDecimal` | | `Types.SQL_DATE` | `DATE` | `java.sql.Date` | | `Types.SQL_TIME` | `TIME` | `java.sql.Time` | | `Types.SQL_TIMESTAMP` | `TIMESTAMP(3)` | `java.sql.Timestamp` | | `Types.INTERVAL_MONTHS`| `INTERVAL YEAR TO MONTH` | `java.lang.Integer` | | `Types.INTERVAL_MILLIS`| `INTERVAL DAY TO SECOND(3)` | `java.lang.Long` | | `Types.PRIMITIVE_ARRAY`| `ARRAY` | e.g. `int[]` | | `Types.OBJECT_ARRAY` | `ARRAY` | e.g. `java.lang.Byte[]`| | `Types.MAP` | `MAP` | `java.util.HashMap` | | `Types.MULTISET` | `MULTISET` | e.g. `java.util.HashMap` for a multiset of `String` | | `Types.ROW` | `ROW` | `org.apache.flink.types.Row` | 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](functions.html#value-access-functions). Generic types are treated as a black box and can be passed on or processed by [user-defined functions](udfs.html). {% top %} Expression Syntax ----------------- Some of the operators in previous sections expect one or more expressions. Expressions can be specified using an embedded Scala DSL or as Strings. Please refer to the examples above to learn how expressions can be specified. This is the EBNF grammar for expressions: {% highlight ebnf %} 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" ) ; {% endhighlight %} **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. {% top %}