Table API#

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

The Table API shares many concepts and parts of its API with Flink's SQL integration. Have a look at the [Common Concepts & API]({{ site.baseurl }}/dev/table/common.html) to learn how to register tables or to create a Table object. The [Streaming Concepts]({{ site.baseurl }}/dev/table/streaming.html) page discusses streaming specific concepts such as dynamic tables and time attributes.

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

Overview & Examples#

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

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

The Java Table API is enabled by importing org.apache.flink.table.api.java.*. The following example shows how a Java Table API program is constructed and how expressions are specified as strings.

// environment configuration
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(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<Row> result = tableEnv.toDataSet(counts, Row.class);
result.print();

The Scala Table API is enabled by importing org.apache.flink.api.scala._ and org.apache.flink.table.api.scala._.

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

import org.apache.flink.api.scala._
import org.apache.flink.table.api.scala._

// environment configuration
val env = ExecutionEnvironment.getExecutionEnvironment
val tEnv = TableEnvironment.getTableEnvironment(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()

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.scan("Orders"); // schema (a, b, c, rowtime)

Table result = orders
        .filter("a.isNotNull && b.isNotNull && c.isNotNull")
        .select("a.lowerCase(), b, rowtime")
        .window(Tumble.over("1.hour").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.scan("Orders") // schema (a, b, c, rowtime)

val result: Table = orders
        .filter('a.isNotNull && 'b.isNotNull && 'c.isNotNull)
        .select('a.lowerCase(), 'b, 'rowtime)
        .window(Tumble over 1.hour on 'rowtime as 'hourlyWindow)
        .groupBy('hourlyWindow, 'a)
        .select('a, 'hourlyWindow.end as 'hour, 'b.avg as 'avgBillingAmount)

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

Operations -

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

Scan, Projection, and Filter#

<tr>
  <td>
    <strong>As</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Renames fields.</p>
Table orders = tableEnv.scan("Orders");
Table result = orders.as("x, y, z, t");
  </td>
</tr>

<tr>
  <td>
    <strong>Where / Filter</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.</p>
Table orders = tableEnv.scan("Orders");
Table result = orders.where("b === 'red'");

or

Table orders = tableEnv.scan("Orders");
Table result = orders.filter("a % 2 === 0");
  </td>
</tr>
Operators Description
Scan
Batch Streaming

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

```java Table orders = tableEnv.scan("Orders"); ```
Select
Batch Streaming

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

```java Table orders = tableEnv.scan("Orders"); Table result = orders.select("a, c as d"); ```

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

```java Table result = orders.select("*"); ```
<tr>
  <td>
    <strong>As</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Renames fields.</p>
val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't)
  </td>
</tr>

<tr>
  <td>
    <strong>Where / Filter</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.</p>
val orders: Table = tableEnv.scan("Orders")
val result = orders.filter('a % 2 === 0)

or

val orders: Table = tableEnv.scan("Orders")
val result = orders.where('b === "red")
  </td>
</tr>
Operators Description
Scan
Batch Streaming

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

```scala val orders: Table = tableEnv.scan("Orders") ```
Select
Batch Streaming

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

```scala val orders: Table = tableEnv.scan("Orders") val result = orders.select('a, 'c as 'd) ```

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

```scala val orders: Table = tableEnv.scan("Orders") val result = orders.select('*) ```

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.

```java Table orders = tableEnv.scan("Orders"); Table result = orders.groupBy("a").select("a, b.sum as d"); ```

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

GroupBy Window Aggregation
Batch Streaming

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

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

```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 ```

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

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

```java Table orders = tableEnv.scan("Orders"); Table result = orders.distinct(); ```

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

<tr>
  <td>
    <strong>GroupBy Aggregation</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span><br>
    <span class="label label-info">Result Updating</span>
  </td>
  <td>
    <p>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.</p>
val orders: Table = tableEnv.scan("Orders")
val result = orders.groupBy('a).select('a, 'b.sum as 'd)
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>
<tr>
  <td>
    <strong>GroupBy Window Aggregation</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Groups and aggregates a table on a <a href="#group-windows">group window</a> and possibly one or more grouping keys.</p>
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
  </td>
</tr>
<tr>
  <td>
    <strong>Over Window Aggregation</strong><br>
    <span class="label label-primary">Streaming</span>
  </td>
  <td>
   <p>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 <a href="#over-windows">over windows section</a> for more details.</p>
   ```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

       <p><b>Note:</b> 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 <a href="streaming.html#time-attributes">time attribute</a>.</p>
      </td>
    </tr>
    <tr>
      <td>
        <strong>Distinct</strong><br>
        <span class="label label-primary">Batch</span>
      </td>
      <td>
        <p>Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.</p>
```scala
val orders: Table = tableEnv.scan("Orders")
val result = orders.distinct()
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>
Operators Description

Joins#

<tr>
  <td>
    <strong>Outer Join</strong><br>
    <span class="label label-primary">Batch</span>
    <span class="label label-primary">Streaming</span>
    <span class="label label-info">Result Updating</span>
  </td>
  <td>
    <p>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.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "d, e, f");

Table leftOuterResult = left.leftOuterJoin(right, "a = d").select("a, b, e");
Table rightOuterResult = left.rightOuterJoin(right, "a = d").select("a, b, e");
Table fullOuterResult = left.fullOuterJoin(right, "a = d").select("a, b, e");

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

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.

```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"); ```

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 Streaming Concepts 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.

    <p>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 (<code>&lt;, &lt;=, &gt;=, &gt;</code>) or a single equality predicate that compares <a href="streaming.html#time-attributes">time attributes</a> of the same type (i.e., processing time or event time) of both input tables.</p> 
    <p>For example, the following predicates are valid window join conditions:</p>

    <ul>
      <li><code>ltime === rtime</code></li>
      <li><code>ltime &gt;= rtime &amp;&amp; ltime &lt; rtime + 10.minutes</code></li>
    </ul>
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");
  </td>
</tr>
<tr>
  <td>
    <strong>TableFunction Inner Join</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Joins a table with a 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.
    </p>
// register function
TableFunction<String> split = new MySplitUDTF();
tEnv.registerFunction("split", split);

// join
Table orders = tableEnv.scan("Orders");
Table result = orders
    .join(new Table(tEnv, "split(c)").as("s", "t", "v"))
    .select("a, b, s, t, v");
  </td>
</tr>
<tr>
  <td>
    <strong>TableFunction Left Outer Join</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Joins a table with a 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.
    <p><b>Note:</b> Currently, the predicate of a table function left outer join can only be empty or literal <code>true</code>.</p>
    </p>
// register function
TableFunction<String> split = new MySplitUDTF();
tEnv.registerFunction("split", split);

// join
Table orders = tableEnv.scan("Orders");
Table result = orders
    .leftOuterJoin(new Table(tEnv, "split(c)").as("s", "t", "v"))
    .select("a, b, s, t, v");
  </td>
</tr>
<tr>
  <td>
    <strong>Inner Join</strong><br>
    <span class="label label-primary">Batch</span>
    <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>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.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'd, 'e, 'f)
val result = left.join(right).where('a === 'd).select('a, 'b, 'e)

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

Operators Description
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.

```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)

<p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
      </td>
    </tr>
    <tr>
      <td><strong>Time-windowed Join</strong><br>
        <span class="label label-primary">Batch</span>
        <span class="label label-primary">Streaming</span>
      </td>
      <td>
        <p><b>Note:</b> Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.</p>

        <p>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 (<code>&lt;, &lt;=, &gt;=, &gt;</code>) or a single equality predicate that compares <a href="streaming.html#time-attributes">time attributes</a> of the same type (i.e., processing time or event time) of both input tables.</p> 
        <p>For example, the following predicates are valid window join conditions:</p>

        <ul>
          <li><code>'ltime === 'rtime</code></li>
          <li><code>'ltime &gt;= 'rtime &amp;&amp; 'ltime &lt; 'rtime + 10.minutes</code></li>
        </ul>

```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)
  </td>
</tr>
<tr>
  <td>
    <strong>TableFunction Inner Join</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Joins a table with a 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.
    </p>
    ```scala

// instantiate function val split: TableFunction[_] = new MySplitUDTF()

// join val result: Table = table .join(split('c) as ('s, 't, 'v)) .select('a, 'b, 's, 't, 'v)

        </td>
    </tr>
    <tr>
      <td>
        <strong>TableFunction Left Outer Join</strong><br>
        <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td>
      <td>
        <p>Joins a table with a 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.
        <p><b>Note:</b> Currently, the predicate of a table function left outer join can only be empty or literal <code>true</code>.</p>
        </p>
```scala
// instantiate function
val split: TableFunction[_] = new MySplitUDTF()

// join
val result: Table = table
    .leftOuterJoin(split('c) as ('s, 't, 'v))
    .select('a, 'b, 's, 't, 'v)
  </td>
</tr>

Set Operations#

<tr>
  <td>
    <strong>UnionAll</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "a, b, c");
Table result = left.unionAll(right);
  </td>
</tr>

<tr>
  <td>
    <strong>Intersect</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "d, e, f");
Table result = left.intersect(right);
  </td>
</tr>

<tr>
  <td>
    <strong>IntersectAll</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "d, e, f");
Table result = left.intersectAll(right);
  </td>
</tr>

<tr>
  <td>
    <strong>Minus</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "a, b, c");
Table result = left.minus(right);
  </td>
</tr>

<tr>
  <td>
    <strong>MinusAll</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
Table left = tableEnv.fromDataSet(ds1, "a, b, c");
Table right = tableEnv.fromDataSet(ds2, "a, b, c");
Table result = left.minusAll(right);
  </td>
</tr>

<tr>
  <td>
    <strong>In</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>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.</p>
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)");
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>
Operators Description
Union
Batch

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

```java Table left = tableEnv.fromDataSet(ds1, "a, b, c"); Table right = tableEnv.fromDataSet(ds2, "a, b, c"); Table result = left.union(right); ```
<tr>
  <td>
    <strong>UnionAll</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>

  </td>
  <td>
    <p>Similar to a SQL UNION ALL clause. Unions two tables, both tables must have identical field types.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'a, 'b, 'c)
val result = left.unionAll(right)
  </td>
</tr>

<tr>
  <td>
    <strong>Intersect</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'e, 'f, 'g)
val result = left.intersect(right)
  </td>
</tr>

<tr>
  <td>
    <strong>IntersectAll</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'e, 'f, 'g)
val result = left.intersectAll(right)
  </td>
</tr>

<tr>
  <td>
    <strong>Minus</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'a, 'b, 'c)
val result = left.minus(right)
  </td>
</tr>

<tr>
  <td>
    <strong>MinusAll</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
val left = ds1.toTable(tableEnv, 'a, 'b, 'c)
val right = ds2.toTable(tableEnv, 'a, 'b, 'c)
val result = left.minusAll(right)
  </td>
</tr>

<tr>
  <td>
    <strong>In</strong><br>
    <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
  </td>
  <td>
    <p>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.</p>
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))
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>
Operators Description
Union
Batch

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

```scala val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'a, 'b, 'c) val result = left.union(right) ```

OrderBy, Offset & Fetch#

<tr>
  <td>
    <strong>Offset &amp; Fetch</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
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);
  </td>
</tr>
Operators Description
Order By
Batch

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

```java Table in = tableEnv.fromDataSet(ds, "a, b, c"); Table result = in.orderBy("a.asc"); ```
<tr>
  <td>
    <strong>Offset &amp; Fetch</strong><br>
    <span class="label label-primary">Batch</span>
  </td>
  <td>
    <p>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.</p>
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)
  </td>
</tr>
Operators Description
Order By
Batch

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

```scala val in = ds.toTable(tableEnv, 'a, 'b, 'c) val result = in.orderBy('a.asc) ```

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.

    <p>Output tables must be registered in the TableEnvironment (see <a href="common.html#register-a-tablesink">Register a TableSink</a>). Moreover, the schema of the registered table must match the schema of the query.</p>
Table orders = tableEnv.scan("Orders");
orders.insertInto("OutOrders");
  </td>
</tr>
Operators Description
Insert Into
Batch Streaming

Similar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.

    <p>Output tables must be registered in the TableEnvironment (see <a href="common.html#register-a-tablesink">Register a TableSink</a>). Moreover, the schema of the registered table must match the schema of the query.</p>
val orders: Table = tableEnv.scan("Orders")
orders.insertInto("OutOrders")
  </td>
</tr>

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: Window) 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.

```java Table table = input .window([Window w].as("w")) // define window with alias w .groupBy("w") // group the table by window w .select("b.sum"); // aggregate ```
```scala val table = input .window([w: Window] as 'w) // define window with alias w .groupBy('w) // group the table by window w .select('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.

```java Table table = input .window([Window 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 ```
```scala val table = input .window([w: Window] as 'w) // define window with alias w .groupBy('w, 'a) // group the table by attribute a and window w .select('a, 'b.sum) // aggregate ```

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

```java Table table = input .window([Window 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 ```
```scala val table = input .window([w: Window] 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 ```

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.
```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"));

</div>

<div data-lang="scala" markdown="1">
```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)

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.
```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"));

</div>

<div data-lang="scala" markdown="1">
```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)

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.
```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"));

</div>

<div data-lang="scala" markdown="1">
```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)

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.

```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 ```
```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 ```

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.

    <p><b>Note:</b> In streaming environments, over window aggregates can only be computed in parallel if the window includes a partition by clause. Without <code>partitionBy(...)</code> the stream is processed by a single, non-parallel task.</p>
  </td>
</tr>
<tr>
  <td><code>orderBy</code></td>
  <td>Required</td>
  <td>
    <p>Defines the order of rows within each partition and thereby the order in which the aggregate functions are applied to rows.</p>

    <p><b>Note:</b> For streaming queries this must be a <a href="streaming.html#time-attributes">declared event-time or processing-time time attribute</a>. Currently, only a single sort attribute is supported.</p>
  </td>
</tr>
<tr>
  <td><code>preceding</code></td>
  <td>Required</td>
  <td>
    <p>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.</p>

    <p><a href="tableApi.html#bounded-over-windows">Bounded over windows</a> are specified with the size of the interval, e.g., <code>10.minutes</code> for a time interval or <code>10.rows</code> for a row-count interval.</p>

    <p><a href="tableApi.html#unbounded-over-windows">Unbounded over windows</a> are specified using a constant, i.e., <code>UNBOUNDED_RANGE</code> for a time interval or <code>UNBOUNDED_ROW</code> for a row-count interval. Unbounded over windows start with the first row of a partition.</p>
  </td>
</tr>
<tr>
  <td><code>following</code></td>
  <td>Optional</td>
  <td>
    <p>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).</p>

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

    <ul>
      <li><code>CURRENT_ROW</code> sets the upper bound of the window to the current row.</li>
      <li><code>CURRENT_RANGE</code> 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.</li>
    </ul>

    <p>If the <code>following</code> clause is omitted, the upper bound of a time interval window is defined as <code>CURRENT_RANGE</code> and the upper bound of a row-count interval window is defined as <code>CURRENT_ROW</code>.</p>
  </td>
</tr>
<tr>
  <td><code>as</code></td>
  <td>Required</td>
  <td>
    <p>Assigns an alias to the over window. The alias is used to reference the over window in the following <code>select()</code> clause.</p>
  </td>
</tr>

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

Unbounded Over Windows#

```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"));

</div>

<div data-lang="scala" markdown="1">
```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)

Bounded Over Windows#

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

</div>

<div data-lang="scala" markdown="1">
```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)

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<String, Integer> for a multiset of String

Generic types and composite types (e.g., POJOs or Tuples) can be fields of a row as well. Generic types are treated as a black box and can be passed on or processed by user-defined functions. Composite types can be accessed with built-in functions (see Value access functions section).

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 = timeIndicator | 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 | nullLiteral | suffixed | atom ;

suffixed = interval | cast | as | if | functionCall ;

interval = timeInterval | rowInterval ;

timeInterval = composite , "." , ("year" | "years" | "month" | "months" | "day" | "days" | "hour" | "hours" | "minute" | "minutes" | "second" | "seconds" | "milli" | "millis") ;

rowInterval = composite , "." , "rows" ;

cast = composite , ".cast(" , 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 , ")" ) ;

as = composite , ".as(" , fieldReference , ")" ;

if = composite , ".?(" , expression , "," , expression , ")" ;

functionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ;

atom = ( "(" , expression , ")" ) | literal | fieldReference ;

fieldReference = "*" | identifier ;

nullLiteral = "Null(" , dataType , ")" ;

timeIntervalUnit = "YEAR" | "YEAR_TO_MONTH" | "MONTH" | "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" ) ;

Here, literal is a valid Java literal, 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. Expressions specified as Strings can also use prefix notation instead of suffix notation to call operators and functions.

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.

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

Built-In Functions

The Table API comes with a set of built-in functions for data transformations. This section gives a brief overview of the available functions.

<tr>
  <td>
    ```java

ANY === ANY

      </td>
      <td>
        <p>Equals.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY !== ANY
  </td>
  <td>
    <p>Not equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY > ANY

      </td>
      <td>
        <p>Greater than.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY >= ANY
  </td>
  <td>
    <p>Greater than or equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY < ANY

      </td>
      <td>
        <p>Less than.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY <= ANY
  </td>
  <td>
    <p>Less than or equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY.isNull

      </td>
      <td>
        <p>Returns true if the given expression is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY.isNotNull
  </td>
  <td>
    <p>Returns true if the given expression is not null.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.like(STRING)

      </td>
      <td>
        <p>Returns true, if a string matches the specified LIKE pattern. E.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n".</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.similar(STRING)
  </td>
  <td>
    <p>Returns true, if a string matches the specified SQL regex pattern. E.g. "A+" matches all strings that consist of at least one "A".</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY.in(ANY, ANY, ...)

      </td>
      <td>
        <p>Returns true if an expression exists in a given list of expressions. This is a shorthand for multiple OR conditions. If the testing set contains null, the result will be null if the element can not be found and true if it can be found. If element is null, the result is always null. E.g. "42.in(1, 2, 3)" leads to false.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY.in(TABLE)
  </td>
  <td>
    <p>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.</p>
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY.between(lowerBound, upperBound)

```
  </td>
  <td>
    <p>Returns true if the given expression is between <i>lowerBound</i> and <i>upperBound</i> (both inclusive). False otherwise. The parameters must be numeric types or identical comparable types.
    </p>
  </td>
</tr>

<tr>
  <td>
    ```java

ANY.notBetween(lowerBound, upperBound)

```
  </td>
  <td>
    <p>Returns true if the given expression is not between <i>lowerBound</i> and <i>upperBound</i> (both inclusive). False otherwise. The parameters must be numeric types or identical comparable types.
    </p>
  </td>
</tr>
Comparison functions Description
<tr>
  <td>
    ```java

boolean1 || boolean2

      </td>
      <td>
        <p>Returns true if <i>boolean1</i> is true or <i>boolean2</i> is true. Supports three-valued logic.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
boolean1 && boolean2
  </td>
  <td>
    <p>Returns true if <i>boolean1</i> and <i>boolean2</i> are both true. Supports three-valued logic.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

!BOOLEAN

      </td>
      <td>
        <p>Returns true if boolean expression is not true; returns null if boolean is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
BOOLEAN.isTrue
  </td>
  <td>
    <p>Returns true if the given boolean expression is true. False otherwise (for null and false).</p>
  </td>
</tr>

<tr>
  <td>
    ```java

BOOLEAN.isFalse

      </td>
      <td>
        <p>Returns true if given boolean expression is false. False otherwise (for null and true).</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
BOOLEAN.isNotTrue
  </td>
  <td>
    <p>Returns true if the given boolean expression is not true (for null and false). False otherwise.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

BOOLEAN.isNotFalse

      </td>
      <td>
        <p>Returns true if given boolean expression is not false (for null and true). False otherwise.</p>
      </td>
    </tr>

  </tbody>
</table>


<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Arithmetic functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

   <tr>
      <td>
        ```java
+ numeric
  </td>
  <td>
    <p>Returns <i>numeric</i>.</p>
  </td>
</tr>

<tr>
  <td>
    ```java
  • numeric

        </td>
        <td>
          <p>Returns negative <i>numeric</i>.</p>
        </td>
      </tr>
    
      <tr>
        <td>
          ```java
    numeric1 + numeric2
    </td>
    <td>
      <p>Returns <i>numeric1</i> plus <i>numeric2</i>.</p>
    </td>
Logical functions Description
```java numeric1 - numeric2 ```

Returns numeric1 minus numeric2.

```java numeric1 * numeric2 ```

Returns numeric1 multiplied by numeric2.

```java numeric1 / numeric2 ```

Returns numeric1 divided by numeric2.

```java numeric1.power(numeric2) ```

Returns numeric1 raised to the power of numeric2.

```java NUMERIC.abs() ```

Calculates the absolute value of given value.

```java numeric1 % numeric2 ```

Returns the remainder (modulus) of numeric1 divided by numeric2. The result is negative only if numeric1 is negative.

```java NUMERIC.sqrt() ```

Calculates the square root of a given value.

```java NUMERIC.ln() ```

Calculates the natural logarithm of given value.

```java NUMERIC.log10() ```

Calculates the base 10 logarithm of given value.

```java numeric1.log() numeric1.log(numeric2) ```

Calculates the logarithm of a given numeric value.

If called without a parameter, this function returns the natural logarithm of numeric1. If called with a parameter numeric2, this function returns the logarithm of numeric1 to the base numeric2. numeric1 must be greater than 0. numeric2 must be greater than 1.

```java NUMERIC.exp() ```

Calculates the Euler's number raised to the given power.

```java NUMERIC.ceil() ```

Calculates the smallest integer greater than or equal to a given number.

```java NUMERIC.floor() ```

Calculates the largest integer less than or equal to a given number.

```java NUMERIC.sin() ```

Calculates the sine of a given number.

```java NUMERIC.cos() ```

Calculates the cosine of a given number.

```java NUMERIC.tan() ```

Calculates the tangent of a given number.

```java NUMERIC.cot() ```

Calculates the cotangent of a given number.

```java NUMERIC.asin() ```

Calculates the arc sine of a given number.

```java NUMERIC.acos() ```

Calculates the arc cosine of a given number.

```java NUMERIC.atan() ```

Calculates the arc tangent of a given number.

```java NUMERIC.degrees() ```

Converts numeric from radians to degrees.

```java NUMERIC.radians() ```

Converts numeric from degrees to radians.

```java NUMERIC.sign() ```

Calculates the signum of a given number.

```java NUMERIC.round(INT) ```

Rounds the given number to integer places right to the decimal point.

```java pi() ```

Returns a value that is closer than any other value to pi.

```java e() ```

Returns a value that is closer than any other value to e.

```java rand() ```

Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive).

```java rand(seed integer) ```

Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive) with a initial seed. Two rand functions will return identical sequences of numbers if they have same initial seed.

```java randInteger(bound integer) ```

Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value (exclusive).

```java randInteger(seed integer, bound integer) ```

Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value (exclusive) with a initial seed. Two randInteger functions will return identical sequences of numbers if they have same initial seed and same bound.

```java NUMERIC.bin() ```

Returns a string representation of an integer numeric value in binary format. Returns null if numeric is null. E.g. "4" leads to "100", "12" leads to "1100".

<tr>
  <td>
    ```java

STRING + STRING

      </td>
      <td>
        <p>Concatenates two character strings.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.charLength()
  </td>
  <td>
    <p>Returns the length of a String.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.upperCase()

      </td>
      <td>
        <p>Returns all of the characters in a string in upper case using the rules of the default locale.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.lowerCase()
  </td>
  <td>
    <p>Returns all of the characters in a string in lower case using the rules of the default locale.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.position(STRING)

      </td>
      <td>
        <p>Returns the position of string in an other string starting at 1. Returns 0 if string could not be found. E.g. <code>'a'.position('bbbbba')</code> leads to 6.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.trim(LEADING, STRING)
STRING.trim(TRAILING, STRING)
STRING.trim(BOTH, STRING)
STRING.trim(BOTH)
STRING.trim()
  </td>
  <td>
    <p>Removes leading and/or trailing characters from the given string. By default, whitespaces at both sides are removed.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.overlay(STRING, INT) STRING.overlay(STRING, INT, INT)

      </td>
      <td>
        <p>Replaces a substring of string with a string starting at a position (starting at 1). An optional length specifies how many characters should be removed. E.g. <code>'xxxxxtest'.overlay('xxxx', 6)</code> leads to "xxxxxxxxx", <code>'xxxxxtest'.overlay('xxxx', 6, 2)</code> leads to "xxxxxxxxxst".</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.substring(INT)
  </td>
  <td>
    <p>Creates a substring of the given string beginning at the given index to the end. The start index starts at 1 and is inclusive.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.substring(INT, INT)

      </td>
      <td>
        <p>Creates a substring of the given string at the given index for the given length. The index starts at 1 and is inclusive, i.e., the character at the index is included in the substring. The substring has the specified length or less.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.initCap()
  </td>

  <td>
    <p>Converts the initial letter of each word in a string to uppercase. Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.lpad(len INT, pad STRING)

      </td>

      <td>
        <p>Returns a string left-padded with the given pad string to a length of len characters. If the string is longer than len, the return value is shortened to len characters. E.g. "hi".lpad(4, '??') returns "??hi",  "hi".lpad(1, '??') returns "h".</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.rpad(len INT, pad STRING)
  </td>

  <td>
    <p>Returns a string right-padded with the given pad string to a length of len characters. If the string is longer than len, the return value is shortened to len characters. E.g. "hi".rpad(4, '??') returns "hi??",  "hi".rpad(1, '??') returns "h".</p>
  </td>
</tr>

<tr>
  <td>
    ```text

concat(string1, string2,...)

      </td>
      <td>
        <p>Returns the string that results from concatenating the arguments. Returns NULL if any argument is NULL. E.g. <code>concat("AA", "BB", "CC")</code> returns <code>AABBCC</code>.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```text
concat_ws(separator, string1, string2,...)
  </td>
  <td>
    <p>Returns the string that results from concatenating the arguments using a separator. The separator is added between the strings to be concatenated. Returns NULL If the separator is NULL. concat_ws() does not skip empty strings. However, it does skip any NULL argument. E.g. <code>concat_ws("~", "AA", "BB", "", "CC")</code> returns <code>AA~BB~~CC</code></p>
  </td>
</tr>
String functions Description
<tr>
  <td>
    ```java

BOOLEAN.?(value1, value2)

      </td>
      <td>
        <p>Ternary conditional operator that decides which of two other expressions should be evaluated based on a evaluated boolean condition. E.g. <code>(42 > 5).?("A", "B")</code> leads to "A".</p>
      </td>
    </tr>

    </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Type conversion functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```java
ANY.cast(TYPE)
  </td>
  <td>
    <p>Converts a value to a given type. E.g. <code>"42".cast(INT)</code> leads to 42.</p>
  </td>
</tr>

</tbody>
Conditional functions Description
<tr>
  <td>
    ```java

NUMERIC.rows

      </td>
      <td>
        <p>Creates an interval of rows.</p>
      </td>
    </tr>

    </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Temporal functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

   <tr>
      <td>
        ```java
STRING.toDate()
  </td>
  <td>
    <p>Parses a date string in the form "yy-mm-dd" to a SQL date.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.toTime()

      </td>
      <td>
        <p>Parses a time string in the form "hh:mm:ss" to a SQL time.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.toTimestamp()
  </td>
  <td>
    <p>Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL timestamp.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

NUMERIC.year NUMERIC.years

      </td>
      <td>
        <p>Creates an interval of months for a given number of years.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
NUMERIC.month
NUMERIC.months
  </td>
  <td>
    <p>Creates an interval of months for a given number of months.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

NUMERIC.day NUMERIC.days

      </td>
      <td>
        <p>Creates an interval of milliseconds for a given number of days.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
NUMERIC.hour
NUMERIC.hours
  </td>
  <td>
    <p>Creates an interval of milliseconds for a given number of hours.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

NUMERIC.minute NUMERIC.minutes

      </td>
      <td>
        <p>Creates an interval of milliseconds for a given number of minutes.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
NUMERIC.second
NUMERIC.seconds
  </td>
  <td>
    <p>Creates an interval of milliseconds for a given number of seconds.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

NUMERIC.milli NUMERIC.millis

      </td>
      <td>
        <p>Creates an interval of milliseconds.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
currentDate()
  </td>
  <td>
    <p>Returns the current SQL date in UTC time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

currentTime()

      </td>
      <td>
        <p>Returns the current SQL time in UTC time zone.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
currentTimestamp()
  </td>
  <td>
    <p>Returns the current SQL timestamp in UTC time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

localTime()

      </td>
      <td>
        <p>Returns the current SQL time in local time zone.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
localTimestamp()
  </td>
  <td>
    <p>Returns the current SQL timestamp in local time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

TEMPORAL.extract(TIMEINTERVALUNIT)

      </td>
      <td>
        <p>Extracts parts of a time point or time interval. Returns the part as a long value. E.g. <code>'2006-06-05'.toDate.extract(DAY)</code> leads to 5 or <code>'2006-06-05'.toDate.extract(QUARTER)</code> leads to 2.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
TIMEPOINT.floor(TIMEINTERVALUNIT)
  </td>
  <td>
    <p>Rounds a time point down to the given unit. E.g. <code>'12:44:31'.toDate.floor(MINUTE)</code> leads to 12:44:00.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

TIMEPOINT.ceil(TIMEINTERVALUNIT)

      </td>
      <td>
        <p>Rounds a time point up to the given unit. E.g. <code>'12:44:31'.toTime.floor(MINUTE)</code> leads to 12:45:00.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
temporalOverlaps(TIMEPOINT, TEMPORAL, TIMEPOINT, TEMPORAL)
  </td>
  <td>
    <p>Determines whether two anchored time intervals overlap. Time point and temporal are transformed into a range defined by two time points (start, end). The function evaluates <code>leftEnd >= rightStart && rightEnd >= leftStart</code>. E.g. <code>temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour)</code> leads to true.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

dateFormat(TIMESTAMP, STRING)

      </td>
      <td>
        <p>Formats <code>timestamp</code> as a string using a specified <code>format</code>. The format must be compatible with MySQL's date formatting syntax as used by the <code>date_parse</code> function. The format specification is given in the <a href="sql.html#date-format-specifier">Date Format Specifier table</a> below.</p>
        <p>For example <code>dateFormat(ts, '%Y, %d %M')</code> results in strings formatted as <code>"2017, 05 May"</code>.</p>
      </td>
    </tr>

    </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Aggregate functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```java
FIELD.count
  </td>
  <td>
    <p>Returns the number of input rows for which the field is not null.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

FIELD.avg

      </td>
      <td>
        <p>Returns the average (arithmetic mean) of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
FIELD.sum
  </td>
  <td>
    <p>Returns the sum of the numeric field across all input values. If all values are null, null is returned.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

FIELD.sum0

      </td>
      <td>
        <p>Returns the sum of the numeric field across all input values. If all values are null, 0 is returned.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
FIELD.max
  </td>
  <td>
    <p>Returns the maximum value of field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

FIELD.min

      </td>
      <td>
        <p>Returns the minimum value of field across all input values.</p>
      </td>
    </tr>


    <tr>
      <td>
        ```java
FIELD.stddevPop
  </td>
  <td>
    <p>Returns the population standard deviation of the numeric field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

FIELD.stddevSamp

      </td>
      <td>
        <p>Returns the sample standard deviation of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
FIELD.varPop
  </td>
  <td>
    <p>Returns the population variance (square of the population standard deviation) of the numeric field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

FIELD.varSamp

      </td>
      <td>
        <p>Returns the sample variance (square of the sample standard deviation) of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
FIELD.collect
  </td>
  <td>
    <p>Returns the multiset aggregate of the input value.</p>
  </td>
</tr>

</tbody>
Value constructor functions Description
<tr>
  <td>
    ```java

COMPOSITE.get(STRING) COMPOSITE.get(INT)

      </td>
      <td>
        <p>Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index or name and returns it's value. E.g. <code>pojo.get('myField')</code> or <code>tuple.get(0)</code>.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ANY.flatten()
  </td>
  <td>
    <p>Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes into a flat representation where every subtype is a separate field. In most cases the fields of the flat representation are named similarly to the original fields but with a dollar separator (e.g. <code>mypojo$mytuple$f0</code>).</p>
  </td>
</tr>
Value access functions Description
<tr>
  <td>
    ```java

array(ANY [, ANY ]*)

      </td>
      <td>
        <p>Creates an array from a list of values. The array will be an array of objects (not primitives).</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ARRAY.cardinality()
  </td>
  <td>
    <p>Returns the number of elements of an array.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

ARRAY.at(INT)

      </td>
      <td>
        <p>Returns the element at a particular position in an array. The index starts at 1.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
ARRAY.element()
  </td>
  <td>
    <p>Returns the sole element of an array with a single element. Returns <code>null</code> if the array is empty. Throws an exception if the array has more than one element.</p>
  </td>
</tr>
Array functions Description
<tr>
  <td>
    ```java

map(ANY, ANY [, ANY, ANY ]*)

      </td>
      <td>
        <p>Creates a map from a list of key-value pairs.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
MAP.cardinality()
  </td>
  <td>
    <p>Returns the number of entries of a map.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

MAP.at(ANY)

      </td>
      <td>
        <p>Returns the value specified by a particular key in a map.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Hash functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```java
STRING.md5()
  </td>
  <td>
    <p>Returns the MD5 hash of the string argument as a string of 32 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.sha1()

      </td>
      <td>
        <p>Returns the SHA-1 hash of the string argument as a string of 40 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.sha224()
  </td>
  <td>
    <p>Returns the SHA-224 hash of the string argument as a string of 56 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.sha256()

      </td>
      <td>
        <p>Returns the SHA-256 hash of the string argument as a string of 64 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.sha384()
  </td>
  <td>
    <p>Returns the SHA-384 hash of the string argument as a string of 96 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```java

STRING.sha512()

      </td>
      <td>
        <p>Returns the SHA-512 hash of the string argument as a string of 128 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```java
STRING.sha2(INT)
  </td>
  <td>
    <p>Returns the hash using the SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384, or SHA-512). The first argument <i>string</i> is the string to be hashed. <i>hashLength</i> is the bit length of the result (either 224, 256, 384, or 512). Returns <i>null</i> if <i>string</i> or <i>hashLength</i> is <i>null</i>.
    </p>
  </td>
</tr>

</tbody>
Map functions Description
<tr>
  <td>
    ```java

row(ANY, [, ANY]*)

      </td>
      <td>
        <p>Creates a row from a list of values. Row is composite type and can be access via <a href="tableApi.html#built-in-functions">value access functions</a>.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Auxiliary functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```java
ANY.as(name [, name ]* )
  </td>
  <td>
    <p>Specifies a name for an expression i.e. a field. Additional names can be specified if the expression expands to multiple fields.</p>
  </td>
</tr>
Row functions Description
 <tr>
  <td>
    ```scala

ANY === ANY

      </td>
      <td>
        <p>Equals.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY !== ANY
  </td>
  <td>
    <p>Not equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY > ANY

      </td>
      <td>
        <p>Greater than.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY >= ANY
  </td>
  <td>
    <p>Greater than or equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY < ANY

      </td>
      <td>
        <p>Less than.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY <= ANY
  </td>
  <td>
    <p>Less than or equal.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY.isNull

      </td>
      <td>
        <p>Returns true if the given expression is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY.isNotNull
  </td>
  <td>
    <p>Returns true if the given expression is not null.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.like(STRING)

      </td>
      <td>
        <p>Returns true, if a string matches the specified LIKE pattern. E.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n".</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.similar(STRING)
  </td>
  <td>
    <p>Returns true, if a string matches the specified SQL regex pattern. E.g. "A+" matches all strings that consist of at least one "A".</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY.in(ANY, ANY, ...)

      </td>
      <td>
        <p>Returns true if an expression exists in a given list of expressions. This is a shorthand for multiple OR conditions. If the testing set contains null, the result will be null if the element can not be found and true if it can be found. If element is null, the result is always null. E.g. "42".in(1, 2, 3) leads to false.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY.in(TABLE)
  </td>
  <td>
    <p>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: This operation is not supported in a streaming environment yet.</p>
    <p><b>Note:</b> 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 <a href="streaming.html">Streaming Concepts</a> for details.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY.between(lowerBound, upperBound)

```
  </td>
  <td>
    <p>Returns true if the given expression is between <i>lowerBound</i> and <i>upperBound</i> (both inclusive). False otherwise. The parameters must be numeric types or identical comparable types.
    </p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ANY.notBetween(lowerBound, upperBound)

```
  </td>
  <td>
    <p>Returns true if the given expression is not between <i>lowerBound</i> and <i>upperBound</i> (both inclusive). False otherwise. The parameters must be numeric types or identical comparable types.
    </p>
  </td>
</tr>
Comparison functions Description
<tr>
  <td>
    ```scala

boolean1 || boolean2

      </td>
      <td>
        <p>Returns true if <i>boolean1</i> is true or <i>boolean2</i> is true. Supports three-valued logic.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
boolean1 && boolean2
  </td>
  <td>
    <p>Returns true if <i>boolean1</i> and <i>boolean2</i> are both true. Supports three-valued logic.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

!BOOLEAN

      </td>
      <td>
        <p>Returns true if boolean expression is not true; returns null if boolean is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
BOOLEAN.isTrue
  </td>
  <td>
    <p>Returns true if the given boolean expression is true. False otherwise (for null and false).</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

BOOLEAN.isFalse

      </td>
      <td>
        <p>Returns true if given boolean expression is false. False otherwise (for null and true).</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
BOOLEAN.isNotTrue
  </td>
  <td>
    <p>Returns true if the given boolean expression is not true (for null and false). False otherwise.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

BOOLEAN.isNotFalse

      </td>
      <td>
        <p>Returns true if given boolean expression is not false (for null and true). False otherwise.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Arithmetic functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

   <tr>
      <td>
        ```scala
+ numeric
  </td>
  <td>
    <p>Returns <i>numeric</i>.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala
  • numeric

        </td>
        <td>
          <p>Returns negative <i>numeric</i>.</p>
        </td>
      </tr>
    
      <tr>
        <td>
          ```scala
    numeric1 + numeric2
    </td>
    <td>
      <p>Returns <i>numeric1</i> plus <i>numeric2</i>.</p>
    </td>
Logical functions Description
```scala numeric1 - numeric2 ```

Returns numeric1 minus numeric2.

```scala numeric1 * numeric2 ```

Returns numeric1 multiplied by numeric2.

```scala numeric1 / numeric2 ```

Returns numeric1 divided by numeric2.

```scala numeric1.power(numeric2) ```

Returns numeric1 raised to the power of numeric2.

```scala NUMERIC.abs() ```

Calculates the absolute value of given value.

```scala numeric1 % numeric2 ```

Returns the remainder (modulus) of numeric1 divided by numeric2. The result is negative only if numeric1 is negative.

```scala NUMERIC.sqrt() ```

Calculates the square root of a given value.

```scala NUMERIC.ln() ```

Calculates the natural logarithm of given value.

```scala NUMERIC.log10() ```

Calculates the base 10 logarithm of given value.

```scala numeric1.log() numeric1.log(numeric2) ```

Calculates the logarithm of a given numeric value.

If called without a parameter, this function returns the natural logarithm of numeric1. If called with a parameter numeric2, this function returns the logarithm of numeric1 to the base numeric2. numeric1 must be greater than 0. numeric2 must be greater than 1.

```scala NUMERIC.exp() ```

Calculates the Euler's number raised to the given power.

```scala NUMERIC.ceil() ```

Calculates the smallest integer greater than or equal to a given number.

```scala NUMERIC.floor() ```

Calculates the largest integer less than or equal to a given number.

```scala NUMERIC.sin() ```

Calculates the sine of a given number.

```scala NUMERIC.cos() ```

Calculates the cosine of a given number.

```scala NUMERIC.tan() ```

Calculates the cotangent of a given number.

```scala NUMERIC.cot() ```

Calculates the arc sine of a given number.

```scala NUMERIC.asin() ```

Calculates the arc cosine of a given number.

```scala NUMERIC.acos() ```

Calculates the arc tangent of a given number.

```scala NUMERIC.atan() ```

Calculates the tangent of a given number.

```scala NUMERIC.degrees() ```

Converts numeric from radians to degrees.

```scala NUMERIC.radians() ```

Converts numeric from degrees to radians.

```scala NUMERIC.sign() ```

Calculates the signum of a given number.

```scala NUMERIC.round(INT) ```

Rounds the given number to integer places right to the decimal point.

```scala pi() ```

Returns a value that is closer than any other value to pi.

```scala e() ```

Returns a value that is closer than any other value to e.

```scala rand() ```

Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive).

```scala rand(seed integer) ```

Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive) with a initial seed. Two rand functions will return identical sequences of numbers if they have same initial seed.

```scala randInteger(bound integer) ```

Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value (exclusive).

```scala randInteger(seed integer, bound integer) ```

Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value (exclusive) with a initial seed. Two randInteger functions will return identical sequences of numbers if they have same initial seed and same bound.

```scala NUMERIC.bin() ```

Returns a string representation of an integer numeric value in binary format. Returns null if numeric is null. E.g. "4" leads to "100", "12" leads to "1100".

<tr>
  <td>
    ```scala

STRING + STRING

      </td>
      <td>
        <p>Concatenates two character strings.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.charLength()
  </td>
  <td>
    <p>Returns the length of a String.</p>
  </td>
</tr> 

<tr>
  <td>
    ```scala

STRING.upperCase()

      </td>
      <td>
        <p>Returns all of the characters in a string in upper case using the rules of the default locale.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.lowerCase()
  </td>
  <td>
    <p>Returns all of the characters in a string in lower case using the rules of the default locale.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.position(STRING)

      </td>
      <td>
        <p>Returns the position of string in an other string starting at 1. Returns 0 if string could not be found. E.g. <code>"a".position("bbbbba")</code> leads to 6.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.trim(
  leading = true,
  trailing = true,
  character = " ")
  </td>
  <td>
    <p>Removes leading and/or trailing characters from the given string.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.overlay(STRING, INT) STRING.overlay(STRING, INT, INT)

      </td>
      <td>
        <p>Replaces a substring of string with a string starting at a position (starting at 1). An optional length specifies how many characters should be removed. E.g. <code>"xxxxxtest".overlay("xxxx", 6)</code> leads to "xxxxxxxxx", <code>"xxxxxtest".overlay('xxxx', 6, 2)</code> leads to "xxxxxxxxxst".</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.substring(INT)
  </td>
  <td>
    <p>Creates a substring of the given string beginning at the given index to the end. The start index starts at 1 and is inclusive.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.substring(INT, INT)

      </td>
      <td>
        <p>Creates a substring of the given string at the given index for the given length. The index starts at 1 and is inclusive, i.e., the character at the index is included in the substring. The substring has the specified length or less.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.initCap()
  </td>

  <td>
    <p>Converts the initial letter of each word in a string to uppercase. Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace.</p>
  </td>
</tr>
Arithmetic functions Description
<tr>
  <td>
    ```java

BOOLEAN.?(value1, value2)

      </td>
      <td>
        <p>Ternary conditional operator that decides which of two other expressions should be evaluated based on a evaluated boolean condition. E.g. <code>(42 > 5).?("A", "B")</code> leads to "A".</p>
      </td>
    </tr>

    </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Type conversion functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```scala
ANY.cast(TYPE)
  </td>
  <td>
    <p>Converts a value to a given type. E.g. <code>"42".cast(Types.INT)</code> leads to 42.</p>
  </td>
</tr>
Conditional functions Description
<tr>
  <td>
    ```scala

NUMERIC.rows

      </td>
      <td>
        <p>Creates an interval of rows.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Temporal functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```scala
STRING.toDate
  </td>
  <td>
    <p>Parses a date string in the form "yy-mm-dd" to a SQL date.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.toTime

      </td>
      <td>
        <p>Parses a time string in the form "hh:mm:ss" to a SQL time.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.toTimestamp
  </td>
  <td>
    <p>Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL timestamp.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

NUMERIC.year NUMERIC.years

      </td>
      <td>
        <p>Creates an interval of months for a given number of years.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
NUMERIC.month
NUMERIC.months
  </td>
  <td>
    <p>Creates an interval of months for a given number of months.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

NUMERIC.day NUMERIC.days

      </td>
      <td>
        <p>Creates an interval of milliseconds for a given number of days.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
NUMERIC.hour
NUMERIC.hours
  </td>
  <td>
    <p>Creates an interval of milliseconds for a given number of hours.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

NUMERIC.minute NUMERIC.minutes

      </td>
      <td>
        <p>Creates an interval of milliseconds for a given number of minutes.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
NUMERIC.second
NUMERIC.seconds
  </td>
  <td>
    <p>Creates an interval of milliseconds for a given number of seconds.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

NUMERIC.milli NUMERIC.millis

      </td>
      <td>
        <p>Creates an interval of milliseconds.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
currentDate()
  </td>
  <td>
    <p>Returns the current SQL date in UTC time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

currentTime()

      </td>
      <td>
        <p>Returns the current SQL time in UTC time zone.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
currentTimestamp()
  </td>
  <td>
    <p>Returns the current SQL timestamp in UTC time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

localTime()

      </td>
      <td>
        <p>Returns the current SQL time in local time zone.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
localTimestamp()
  </td>
  <td>
    <p>Returns the current SQL timestamp in local time zone.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

TEMPORAL.extract(TimeIntervalUnit)

      </td>
      <td>
        <p>Extracts parts of a time point or time interval. Returns the part as a long value. E.g. <code>"2006-06-05".toDate.extract(TimeIntervalUnit.DAY)</code> leads to 5 or <code>'2006-06-05'.toDate.extract(QUARTER)</code> leads to 2.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
TIMEPOINT.floor(TimeIntervalUnit)
  </td>
  <td>
    <p>Rounds a time point down to the given unit. E.g. <code>"12:44:31".toTime.floor(TimeIntervalUnit.MINUTE)</code> leads to 12:44:00.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

TIMEPOINT.ceil(TimeIntervalUnit)

      </td>
      <td>
        <p>Rounds a time point up to the given unit. E.g. <code>"12:44:31".toTime.floor(TimeIntervalUnit.MINUTE)</code> leads to 12:45:00.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
temporalOverlaps(TIMEPOINT, TEMPORAL, TIMEPOINT, TEMPORAL)
  </td>
  <td>
    <p>Determines whether two anchored time intervals overlap. Time point and temporal are transformed into a range defined by two time points (start, end). The function evaluates <code>leftEnd >= rightStart && rightEnd >= leftStart</code>. E.g. <code>temporalOverlaps('2:55:00'.toTime, 1.hour, '3:30:00'.toTime, 2.hours)</code> leads to true.</p>
  </td>
</tr>
Value constructor functions Description
<tr>
  <td>
    ```scala

FIELD.avg

      </td>
      <td>
        <p>Returns the average (arithmetic mean) of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
FIELD.sum
  </td>
  <td>
    <p>Returns the sum of the numeric field across all input values. If all values are null, null is returned.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

FIELD.sum0

      </td>
      <td>
        <p>Returns the sum of the numeric field across all input values. If all values are null, 0 is returned.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
FIELD.max
  </td>
  <td>
    <p>Returns the maximum value of field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

FIELD.min

      </td>
      <td>
        <p>Returns the minimum value of field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
FIELD.stddevPop
  </td>
  <td>
    <p>Returns the population standard deviation of the numeric field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

FIELD.stddevSamp

      </td>
      <td>
        <p>Returns the sample standard deviation of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
FIELD.varPop
  </td>
  <td>
    <p>Returns the population variance (square of the population standard deviation) of the numeric field across all input values.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

FIELD.varSamp

      </td>
      <td>
        <p>Returns the sample variance (square of the sample standard deviation) of the numeric field across all input values.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
FIELD.collect
  </td>
  <td>
    <p>Returns the multiset aggregate of the input value.</p>
  </td>
</tr>
Aggregate functions Description
```scala FIELD.count ```

Returns the number of input rows for which the field is not null.

<tr>
  <td>
    ```scala

COMPOSITE.get(STRING) COMPOSITE.get(INT)

      </td>
      <td>
        <p>Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index or name and returns it's value. E.g. <code>'pojo.get("myField")</code> or <code>'tuple.get(0)</code>.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ANY.flatten()
  </td>
  <td>
    <p>Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes into a flat representation where every subtype is a separate field. In most cases the fields of the flat representation are named similarly to the original fields but with a dollar separator (e.g. <code>mypojo$mytuple$f0</code>).</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

dateFormat(TIMESTAMP, STRING)

      </td>
      <td>
        <p>Formats <code>timestamp</code> as a string using a specified <code>format</code>. The format must be compatible with MySQL's date formatting syntax as used by the <code>date_parse</code> function. The format specification is given in the <a href="sql.html#date-format-specifier">Date Format Specifier table</a> below.</p>
        <p>For example <code>dateFormat('ts, "%Y, %d %M")</code> results in strings formatted as <code>"2017, 05 May"</code>.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Array functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```scala
array(ANY [, ANY ]*)
  </td>
  <td>
    <p>Creates an array from a list of values. The array will be an array of objects (not primitives).</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ARRAY.cardinality()

      </td>
      <td>
        <p>Returns the number of elements of an array.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
ARRAY.at(INT)
  </td>
  <td>
    <p>Returns the element at a particular position in an array. The index starts at 1.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

ARRAY.element()

      </td>
      <td>
        <p>Returns the sole element of an array with a single element. Returns <code>null</code> if the array is empty. Throws an exception if the array has more than one element.</p>
      </td>
    </tr>

  </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Map functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```scala
map(ANY, ANY [, ANY, ANY ]*)
  </td>
  <td>
    <p>Creates a map from a list of key-value pairs.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

MAP.cardinality()

      </td>
      <td>
        <p>Returns the number of entries of a map.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
MAP.at(ANY)
  </td>
  <td>
    <p>Returns the value specified by a particular key in a map.</p>
  </td>
</tr>
Value access functions Description
<tr>
  <td>
    ```scala

STRING.md5()

      </td>
      <td>
        <p>Returns the MD5 hash of the string argument as a string of 32 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.sha1()
  </td>
  <td>
    <p>Returns the SHA-1 hash of the string argument as a string of 40 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.sha224()

      </td>
      <td>
        <p>Returns the SHA-224 hash of the string argument as a string of 56 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>

    <tr>
      <td>
        ```scala
STRING.sha256()
  </td>
  <td>
    <p>Returns the SHA-256 hash of the string argument as a string of 64 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.sha384()

      </td>
      <td>
        <p>Returns the SHA-384 hash of the string argument as a string of 96 hexadecimal digits; null if <i>string</i> is null.</p>
      </td>
    </tr>    

    <tr>
      <td>
        ```scala
STRING.sha512()
  </td>
  <td>
    <p>Returns the SHA-512 hash of the string argument as a string of 128 hexadecimal digits; null if <i>string</i> is null.</p>
  </td>
</tr>

<tr>
  <td>
    ```scala

STRING.sha2(INT)

      </td>
      <td>
        <p>Returns the hash using the SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384, or SHA-512). The first argument <i>string</i> is the string to be hashed. <i>hashLength</i> is the bit length of the result (either 224, 256, 384, or 512). Returns <i>null</i> if <i>string</i> or <i>hashLength</i> is <i>null</i>.
        </p>
      </td>
    </tr>
    </tbody>
</table>

<table class="table table-bordered">
  <thead>
    <tr>
      <th class="text-left" style="width: 40%">Row functions</th>
      <th class="text-center">Description</th>
    </tr>
  </thead>

  <tbody>

    <tr>
      <td>
        ```scala
row(ANY, [, ANY]*)
  </td>
  <td>
    <p>Creates a row from a list of values. Row is composite type and can be access via <a href="tableApi.html#built-in-functions">value access functions</a>.</p>
  </td>
</tr>
Hash functions Description
Auxiliary functions Description
```scala ANY.as(name [, name ]* ) ```

Specifies a name for an expression i.e. a field. Additional names can be specified if the expression expands to multiple fields.

Unsupported Functions#

The following operations are not supported yet:

  • Binary string operators and functions
  • System functions
  • Aggregate functions like REGR_xxx
  • Distinct aggregate functions like COUNT DISTINCT