This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.

SQL

SQL queries are specified with the sqlQuery() method of the TableEnvironment. The method returns the result of the SQL query as a Table. A Table can be used in subsequent SQL and Table API queries, be converted into a DataSet or DataStream, or written to a TableSink). SQL and Table API queries can seamlessly mixed and are holistically optimized and translated into a single program.

In order to access a table in a SQL query, it must be registered in the TableEnvironment. A table can be registered from a TableSource, Table, DataStream, or DataSet. Alternatively, users can also register external catalogs in a TableEnvironment to specify the location of the data sources.

For convenience Table.toString() automatically registers the table under a unique name in its TableEnvironment and returns the name. Hence, Table objects can be directly inlined into SQL queries (by string concatenation) as shown in the examples below.

Note: Flink’s SQL support is not yet feature complete. Queries that include unsupported SQL features cause a TableException. The supported features of SQL on batch and streaming tables are listed in the following sections.

Specifying a Query

The following examples show how to specify a SQL queries on registered and inlined tables.

StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);

// ingest a DataStream from an external source
DataStream<Tuple3<Long, String, Integer>> ds = env.addSource(...);

// SQL query with an inlined (unregistered) table
Table table = tableEnv.fromDataStream(ds, "user, product, amount");
Table result = tableEnv.sqlQuery(
  "SELECT SUM(amount) FROM " + table + " WHERE product LIKE '%Rubber%'");

// SQL query with a registered table
// register the DataStream as table "Orders"
tableEnv.registerDataStream("Orders", ds, "user, product, amount");
// run a SQL query on the Table and retrieve the result as a new Table
Table result2 = tableEnv.sqlQuery(
  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");

// SQL update with a registered table
// create and register a TableSink
TableSink csvSink = new CsvTableSink("/path/to/file", ...);
String[] fieldNames = {"product", "amount"};
TypeInformation[] fieldTypes = {Types.STRING, Types.INT};
tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink);
// run a SQL update query on the Table and emit the result to the TableSink
tableEnv.sqlUpdate(
  "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");
val env = StreamExecutionEnvironment.getExecutionEnvironment
val tableEnv = TableEnvironment.getTableEnvironment(env)

// read a DataStream from an external source
val ds: DataStream[(Long, String, Integer)] = env.addSource(...)

// SQL query with an inlined (unregistered) table
val table = ds.toTable(tableEnv, 'user, 'product, 'amount)
val result = tableEnv.sqlQuery(
  s"SELECT SUM(amount) FROM $table WHERE product LIKE '%Rubber%'")

// SQL query with a registered table
// register the DataStream under the name "Orders"
tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount)
// run a SQL query on the Table and retrieve the result as a new Table
val result2 = tableEnv.sqlQuery(
  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'")

// SQL update with a registered table
// create and register a TableSink
TableSink csvSink = new CsvTableSink("/path/to/file", ...)
val fieldNames: Array[String] = Array("product", "amount")
val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT)
tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink)
// run a SQL update query on the Table and emit the result to the TableSink
tableEnv.sqlUpdate(
  "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'")

Back to top

Supported Syntax

Flink parses SQL using Apache Calcite, which supports standard ANSI SQL. DDL statements are not supported by Flink.

The following BNF-grammar describes the superset of supported SQL features in batch and streaming queries. The Operations section shows examples for the supported features and indicates which features are only supported for batch or streaming queries.

insert:
  INSERT INTO tableReference
  query
  
query:
  values
  | {
      select
      | selectWithoutFrom
      | query UNION [ ALL ] query
      | query EXCEPT query
      | query INTERSECT query
    }
    [ ORDER BY orderItem [, orderItem ]* ]
    [ LIMIT { count | ALL } ]
    [ OFFSET start { ROW | ROWS } ]
    [ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } ONLY]

orderItem:
  expression [ ASC | DESC ]

select:
  SELECT [ ALL | DISTINCT ]
  { * | projectItem [, projectItem ]* }
  FROM tableExpression
  [ WHERE booleanExpression ]
  [ GROUP BY { groupItem [, groupItem ]* } ]
  [ HAVING booleanExpression ]
  [ WINDOW windowName AS windowSpec [, windowName AS windowSpec ]* ]
  
selectWithoutFrom:
  SELECT [ ALL | DISTINCT ]
  { * | projectItem [, projectItem ]* }

projectItem:
  expression [ [ AS ] columnAlias ]
  | tableAlias . *

tableExpression:
  tableReference [, tableReference ]*
  | tableExpression [ NATURAL ] [ LEFT | RIGHT | FULL ] JOIN tableExpression [ joinCondition ]

joinCondition:
  ON booleanExpression
  | USING '(' column [, column ]* ')'

tableReference:
  tablePrimary
  [ [ AS ] alias [ '(' columnAlias [, columnAlias ]* ')' ] ]

tablePrimary:
  [ TABLE ] [ [ catalogName . ] schemaName . ] tableName
  | LATERAL TABLE '(' functionName '(' expression [, expression ]* ')' ')'
  | UNNEST '(' expression ')'

values:
  VALUES expression [, expression ]*

groupItem:
  expression
  | '(' ')'
  | '(' expression [, expression ]* ')'
  | CUBE '(' expression [, expression ]* ')'
  | ROLLUP '(' expression [, expression ]* ')'
  | GROUPING SETS '(' groupItem [, groupItem ]* ')'

windowRef:
    windowName
  | windowSpec

windowSpec:
    [ windowName ]
    '('
    [ ORDER BY orderItem [, orderItem ]* ]
    [ PARTITION BY expression [, expression ]* ]
    [
        RANGE numericOrIntervalExpression {PRECEDING}
      | ROWS numericExpression {PRECEDING}
    ]
    ')'

Flink SQL uses a lexical policy for identifier (table, attribute, function names) similar to Java:

  • The case of identifiers is preserved whether or not they are quoted.
  • After which, identifiers are matched case-sensitively.
  • Unlike Java, back-ticks allow identifiers to contain non-alphanumeric characters (e.g. "SELECT a AS `my field` FROM t").

Back to top

Operations

Scan, Projection, and Filter

Operation Description
Scan / Select / As
Batch Streaming
SELECT * FROM Orders

SELECT a, c AS d FROM Orders
Where / Filter
Batch Streaming
SELECT * FROM Orders WHERE b = 'red'

SELECT * FROM Orders WHERE a % 2 = 0
User-defined Scalar Functions (Scalar UDF)
Batch Streaming

UDFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register scalar UDFs.

SELECT PRETTY_PRINT(user) FROM Orders

Back to top

Aggregations

Operation Description
GroupBy Aggregation
Batch Streaming
Result Updating

Note: GroupBy on a streaming table produces an updating result. See the Streaming Concepts page for details.

SELECT a, SUM(b) as d
FROM Orders
GROUP BY a
GroupBy Window Aggregation
Batch Streaming

Use a group window to compute a single result row per group. See Group Windows section for more details.

SELECT user, SUM(amount)
FROM Orders
GROUP BY TUMBLE(rowtime, INTERVAL '1' DAY), user
Over Window aggregation
Streaming

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute

SELECT COUNT(amount) OVER (
  PARTITION BY user
  ORDER BY proctime
  ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)
FROM Orders

SELECT COUNT(amount) OVER w, SUM(amount) OVER w
FROM Orders 
WINDOW w AS (
  PARTITION BY user
  ORDER BY proctime
  ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)  
Distinct
Batch Streaming
Result Updating
SELECT DISTINCT users FROM Orders

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.

Grouping sets, Rollup, Cube
Batch
SELECT SUM(amount)
FROM Orders
GROUP BY GROUPING SETS ((user), (product))
Having
Batch Streaming
SELECT SUM(amount)
FROM Orders
GROUP BY users
HAVING SUM(amount) > 50
User-defined Aggregate Functions (UDAGG)
Batch Streaming

UDAGGs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDAGGs.

SELECT MyAggregate(amount)
FROM Orders
GROUP BY users

Back to top

Joins

Operation Description
Inner Equi-join
Batch Streaming

Currently, only equi-joins are supported, i.e., joins that have at least one conjunctive condition with an equality predicate. Arbitrary cross or theta joins are not supported.

Note: The order of joins is not optimized. Tables are joined in the order in which they are specified in the FROM clause. Make sure to specify tables in an order that does not yield a cross join (Cartesian product) which are not supported and would cause a query to fail.

SELECT *
FROM Orders INNER JOIN Product ON Orders.productId = Product.id

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.

Outer Equi-join
Batch Streaming Result Updating

Currently, only equi-joins are supported, i.e., joins that have at least one conjunctive condition with an equality predicate. Arbitrary cross or theta joins are not supported.

Note: The order of joins is not optimized. Tables are joined in the order in which they are specified in the FROM clause. Make sure to specify tables in an order that does not yield a cross join (Cartesian product) which are not supported and would cause a query to fail.

SELECT *
FROM Orders LEFT JOIN Product ON Orders.productId = Product.id

SELECT *
FROM Orders RIGHT JOIN Product ON Orders.productId = Product.id

SELECT *
FROM Orders FULL OUTER JOIN Product ON Orders.productId = Product.id

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.

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 (<, <=, >=, >), a BETWEEN predicate, or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

For example, the following predicates are valid window join conditions:

  • ltime = rtime
  • ltime >= rtime AND ltime < rtime + INTERVAL '10' MINUTE
  • ltime BETWEEN rtime - INTERVAL '10' SECOND AND rtime + INTERVAL '5' SECOND
SELECT *
FROM Orders o, Shipments s
WHERE o.id = s.orderId AND
      o.ordertime BETWEEN s.shiptime - INTERVAL '4' HOUR AND s.shiptime
The example above will join all orders with their corresponding shipments if the order was shipped four hours after the order was received.
Expanding arrays into a relation
Batch Streaming

Unnesting WITH ORDINALITY is not supported yet.

SELECT users, tag
FROM Orders CROSS JOIN UNNEST(tags) AS t (tag)
Join with Table Function
Batch Streaming

UDTFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDTFs.

Inner Join

SELECT users, tag
FROM Orders, LATERAL TABLE(unnest_udtf(tags)) t AS tag

Left Outer Join

SELECT users, tag
FROM Orders LEFT JOIN LATERAL TABLE(unnest_udtf(tags)) t AS tag ON TRUE

Note: Currently, only literal TRUE is supported as predicate for a left outer join against a lateral table.

Back to top

Set Operations

Operation Description
Union
Batch
SELECT *
FROM (
    (SELECT user FROM Orders WHERE a % 2 = 0)
  UNION
    (SELECT user FROM Orders WHERE b = 0)
)
UnionAll
Batch Streaming
SELECT *
FROM (
    (SELECT user FROM Orders WHERE a % 2 = 0)
  UNION ALL
    (SELECT user FROM Orders WHERE b = 0)
)
Intersect / Except
Batch
SELECT *
FROM (
    (SELECT user FROM Orders WHERE a % 2 = 0)
  INTERSECT
    (SELECT user FROM Orders WHERE b = 0)
)
SELECT *
FROM (
    (SELECT user FROM Orders WHERE a % 2 = 0)
  EXCEPT
    (SELECT user FROM Orders WHERE b = 0)
)
In
Batch Streaming

Returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

SELECT user, amount
FROM Orders
WHERE product IN (
    SELECT product FROM NewProducts
)

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

Exists
Batch Streaming

Returns true if the sub-query returns at least one row. Only supported if the operation can be rewritten in a join and group operation.

SELECT user, amount
FROM Orders
WHERE product EXISTS (
    SELECT product FROM NewProducts
)

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

Back to top

OrderBy & Limit

Operation Description
Order By
Batch Streaming
Note: The result of streaming queries must be primarily sorted on an ascending time attribute. Additional sorting attributes are supported.
SELECT *
FROM Orders
ORDER BY orderTime
Limit
Batch
SELECT *
FROM Orders
LIMIT 3

Back to top

Insert

Operation Description
Insert Into
Batch Streaming

Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

INSERT INTO OutputTable
SELECT users, tag
FROM Orders

Back to top

Group Windows

Group windows are defined in the GROUP BY clause of a SQL query. Just like queries with regular GROUP BY clauses, queries with a GROUP BY clause that includes a group window function compute a single result row per group. The following group windows functions are supported for SQL on batch and streaming tables.

Group Window Function Description
TUMBLE(time_attr, interval) Defines a tumbling time window. A tumbling time window assigns rows to non-overlapping, continuous windows with a fixed duration (interval). For example, a tumbling window of 5 minutes groups rows in 5 minutes intervals. Tumbling windows can be defined on event-time (stream + batch) or processing-time (stream).
HOP(time_attr, interval, interval) Defines a hopping time window (called sliding window in the Table API). A hopping time window has a fixed duration (second interval parameter) and hops by a specified hop interval (first interval parameter). If the hop interval is smaller than the window size, hopping windows are overlapping. Thus, rows can be assigned to multiple windows. For example, a hopping window of 15 minutes size and 5 minute hop interval assigns each row to 3 different windows of 15 minute size, which are evaluated in an interval of 5 minutes. Hopping windows can be defined on event-time (stream + batch) or processing-time (stream).
SESSION(time_attr, interval) Defines a session time window. Session time windows do not have a fixed duration but their bounds are defined by a time interval of inactivity, i.e., a session window is closed if no event appears for a defined gap period. For example a session window with a 30 minute gap starts when a row is observed after 30 minutes inactivity (otherwise the row would be added to an existing window) and is closed if no row is added within 30 minutes. Session windows can work on event-time (stream + batch) or processing-time (stream).

Time Attributes

For SQL queries on streaming tables, the time_attr argument of the group window function must refer to a valid time attribute that specifies the processing time or event time of rows. See the documentation of time attributes to learn how to define time attributes.

For SQL on batch tables, the time_attr argument of the group window function must be an attribute of type TIMESTAMP.

Selecting Group Window Start and End Timestamps

The start and end timestamps of group windows as well as time attributes can be selected with the following auxiliary functions:

Auxiliary Function Description
TUMBLE_START(time_attr, interval)
HOP_START(time_attr, interval, interval)
SESSION_START(time_attr, interval)

Returns the timestamp of the inclusive lower bound of the corresponding tumbling, hopping, or session window.

TUMBLE_END(time_attr, interval)
HOP_END(time_attr, interval, interval)
SESSION_END(time_attr, interval)

Returns the timestamp of the exclusive upper bound of the corresponding tumbling, hopping, or session window.

Note: The exclusive upper bound timestamp cannot be used as a rowtime attribute in subsequent time-based operations, such as time-windowed joins and group window or over window aggregations.

TUMBLE_ROWTIME(time_attr, interval)
HOP_ROWTIME(time_attr, interval, interval)
SESSION_ROWTIME(time_attr, interval)

Returns the timestamp of the inclusive upper bound of the corresponding tumbling, hopping, or session window.

The resulting attribute is a rowtime attribute that can be used in subsequent time-based operations such as time-windowed joins and group window or over window aggregations.

TUMBLE_PROCTIME(time_attr, interval)
HOP_PROCTIME(time_attr, interval, interval)
SESSION_PROCTIME(time_attr, interval)

Returns a proctime attribute that can be used in subsequent time-based operations such as time-windowed joins and group window or over window aggregations.

Note: Auxiliary functions must be called with exactly same arguments as the group window function in the GROUP BY clause.

The following examples show how to specify SQL queries with group windows on streaming tables.

StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);

// ingest a DataStream from an external source
DataStream<Tuple3<Long, String, Integer>> ds = env.addSource(...);
// register the DataStream as table "Orders"
tableEnv.registerDataStream("Orders", ds, "user, product, amount, proctime.proctime, rowtime.rowtime");

// compute SUM(amount) per day (in event-time)
Table result1 = tableEnv.sqlQuery(
  "SELECT user, " +
  "  TUMBLE_START(rowtime, INTERVAL '1' DAY) as wStart,  " +
  "  SUM(amount) FROM Orders " +
  "GROUP BY TUMBLE(rowtime, INTERVAL '1' DAY), user");

// compute SUM(amount) per day (in processing-time)
Table result2 = tableEnv.sqlQuery(
  "SELECT user, SUM(amount) FROM Orders GROUP BY TUMBLE(proctime, INTERVAL '1' DAY), user");

// compute every hour the SUM(amount) of the last 24 hours in event-time
Table result3 = tableEnv.sqlQuery(
  "SELECT product, SUM(amount) FROM Orders GROUP BY HOP(rowtime, INTERVAL '1' HOUR, INTERVAL '1' DAY), product");

// compute SUM(amount) per session with 12 hour inactivity gap (in event-time)
Table result4 = tableEnv.sqlQuery(
  "SELECT user, " +
  "  SESSION_START(rowtime, INTERVAL '12' HOUR) AS sStart, " +
  "  SESSION_ROWTIME(rowtime, INTERVAL '12' HOUR) AS snd, " +
  "  SUM(amount) " +
  "FROM Orders " +
  "GROUP BY SESSION(rowtime, INTERVAL '12' HOUR), user");
val env = StreamExecutionEnvironment.getExecutionEnvironment
val tableEnv = TableEnvironment.getTableEnvironment(env)

// read a DataStream from an external source
val ds: DataStream[(Long, String, Int)] = env.addSource(...)
// register the DataStream under the name "Orders"
tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount, 'proctime.proctime, 'rowtime.rowtime)

// compute SUM(amount) per day (in event-time)
val result1 = tableEnv.sqlQuery(
    """
      |SELECT
      |  user,
      |  TUMBLE_START(rowtime, INTERVAL '1' DAY) as wStart,
      |  SUM(amount)
      | FROM Orders
      | GROUP BY TUMBLE(rowtime, INTERVAL '1' DAY), user
    """.stripMargin)

// compute SUM(amount) per day (in processing-time)
val result2 = tableEnv.sqlQuery(
  "SELECT user, SUM(amount) FROM Orders GROUP BY TUMBLE(proctime, INTERVAL '1' DAY), user")

// compute every hour the SUM(amount) of the last 24 hours in event-time
val result3 = tableEnv.sqlQuery(
  "SELECT product, SUM(amount) FROM Orders GROUP BY HOP(rowtime, INTERVAL '1' HOUR, INTERVAL '1' DAY), product")

// compute SUM(amount) per session with 12 hour inactivity gap (in event-time)
val result4 = tableEnv.sqlQuery(
    """
      |SELECT
      |  user,
      |  SESSION_START(rowtime, INTERVAL '12' HOUR) AS sStart,
      |  SESSION_END(rowtime, INTERVAL '12' HOUR) AS sEnd,
      |  SUM(amount)
      | FROM Orders
      | GROUP BY SESSION(rowtime(), INTERVAL '12' HOUR), user
    """.stripMargin)

Back to top

Data Types

The SQL runtime 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 SQL Types, Table API 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).

Back to top

Built-In Functions

Flink’s SQL support comes with a set of built-in functions for data transformations. This section gives a brief overview of the available functions.

The Flink SQL functions (including their syntax) are a subset of Apache Calcite’s built-in functions. Most of the documentation has been adopted from the Calcite SQL reference.

Comparison functions Description
value1 = value2

Equals.

value1 <> value2

Not equal.

value1 > value2

Greater than.

value1 >= value2

Greater than or equal.

value1 < value2

Less than.

value1 <= value2

Less than or equal.

value IS NULL

Returns TRUE if value is null.

value IS NOT NULL

Returns TRUE if value is not null.

value1 IS DISTINCT FROM value2

Returns TRUE if two values are not equal, treating null values as the same.

value1 IS NOT DISTINCT FROM value2

Returns TRUE if two values are equal, treating null values as the same.

value1 BETWEEN [ASYMMETRIC | SYMMETRIC] value2 AND value3

Returns TRUE if value1 is greater than or equal to value2 and less than or equal to value3.

value1 NOT BETWEEN value2 AND value3

Returns TRUE if value1 is less than value2 or greater than value3.

string1 LIKE string2 [ ESCAPE string3 ]

Returns TRUE if string1 matches pattern string2. An escape character can be defined if necessary.

string1 NOT LIKE string2 [ ESCAPE string3 ]

Returns TRUE if string1 does not match pattern string2. An escape character can be defined if necessary.

string1 SIMILAR TO string2 [ ESCAPE string3 ]

Returns TRUE if string1 matches regular expression string2. An escape character can be defined if necessary.

string1 NOT SIMILAR TO string2 [ ESCAPE string3 ]

Returns TRUE if string1 does not match regular expression string2. An escape character can be defined if necessary.

value IN (value [, value]* )

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 the element is NULL, the result is always NULL. E.g. "42 IN (1, 2, 3)" leads to FALSE.

value NOT IN (value [, value]* )

Returns TRUE if value is not equal to every value in a list.

EXISTS (sub-query)

Returns TRUE if sub-query returns at least one row. Only supported if the operation can be rewritten in a join and group operation.

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

value IN (sub-query)

Returns TRUE if value is equal to a row returned by sub-query.

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

value NOT IN (sub-query)

Returns TRUE if value is not equal to every row returned by sub-query.

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

Logical functions Description
boolean1 OR boolean2

Returns TRUE if boolean1 is TRUE or boolean2 is TRUE. Supports three-valued logic.

boolean1 AND boolean2

Returns TRUE if boolean1 and boolean2 are both TRUE. Supports three-valued logic.

NOT boolean

Returns TRUE if boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.

boolean IS FALSE

Returns TRUE if boolean is FALSE; returns FALSE if boolean is UNKNOWN.

boolean IS NOT FALSE

Returns TRUE if boolean is not FALSE; returns TRUE if boolean is UNKNOWN.

boolean IS TRUE

Returns TRUE if boolean is TRUE; returns FALSE if boolean is UNKNOWN.

boolean IS NOT TRUE

Returns TRUE if boolean is not TRUE; returns TRUE if boolean is UNKNOWN.

boolean IS UNKNOWN

Returns TRUE if boolean is UNKNOWN.

boolean IS NOT UNKNOWN

Returns TRUE if boolean is not UNKNOWN.

Arithmetic functions Description
+ numeric

Returns numeric.

- numeric

Returns negative numeric.

numeric1 + numeric2

Returns numeric1 plus numeric2.

numeric1 - numeric2

Returns numeric1 minus numeric2.

numeric1 * numeric2

Returns numeric1 multiplied by numeric2.

numeric1 / numeric2

Returns numeric1 divided by numeric2.

POWER(numeric1, numeric2)

Returns numeric1 raised to the power of numeric2.

ABS(numeric)

Returns the absolute value of numeric.

MOD(numeric1, numeric2)

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

SQRT(numeric)

Returns the square root of numeric.

LN(numeric)

Returns the natural logarithm (base e) of numeric.

LOG10(numeric)

Returns the base 10 logarithm of numeric.

LOG(x numeric)
LOG(b numeric, x numeric)

Returns the logarithm of a numeric.

If called with one parameter, this function returns the natural logarithm of x. If called with two parameters, this function returns the logarithm of x to the base b. x must be greater than 0. b must be greater than 1.

EXP(numeric)

Returns e raised to the power of numeric.

CEIL(numeric)
CEILING(numeric)

Rounds numeric up, and returns the smallest number that is greater than or equal to numeric.

FLOOR(numeric)

Rounds numeric down, and returns the largest number that is less than or equal to numeric.

SIN(numeric)

Calculates the sine of a given number.

COS(numeric)

Calculates the cosine of a given number.

TAN(numeric)

Calculates the tangent of a given number.

COT(numeric)

Calculates the cotangent of a given number.

ASIN(numeric)

Calculates the arc sine of a given number.

ACOS(numeric)

Calculates the arc cosine of a given number.

ATAN(numeric)

Calculates the arc tangent of a given number.

DEGREES(numeric)

Converts numeric from radians to degrees.

RADIANS(numeric)

Converts numeric from degrees to radians.

SIGN(numeric)

Calculates the signum of a given number.

ROUND(numeric, int)

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

PI()

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

E()

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

RAND()

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

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.

RAND_INTEGER(bound integer)

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

RAND_INTEGER(seed integer, bound integer)

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

BIN(numeric)
      

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

String functions Description
string || string

Concatenates two character strings.

CHAR_LENGTH(string)

Returns the number of characters in a character string.

CHARACTER_LENGTH(string)

As CHAR_LENGTH(string).

UPPER(string)

Returns a character string converted to upper case.

LOWER(string)

Returns a character string converted to lower case.

POSITION(string1 IN string2)

Returns the position of the first occurrence of string1 in string2.

TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)

Removes leading and/or trailing characters from string2. By default, whitespaces at both sides are removed.

OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])

Replaces a substring of string1 with string2.

SUBSTRING(string FROM integer)

Returns a substring of a character string starting at a given point.

SUBSTRING(string FROM integer FOR integer)

Returns a substring of a character string starting at a given point with a given length.

INITCAP(string)

Returns string with the first letter of each word converter to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters.

CONCAT(string1, string2,...)

Returns the string that results from concatenating the arguments. Returns NULL if any argument is NULL. E.g. CONCAT("AA", "BB", "CC") returns AABBCC.

CONCAT_WS(separator, string1, string2,...)

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. CONCAT_WS("~", "AA", "BB", "", "CC") returns AA~BB~~CC

LPAD(text string, len integer, pad string)

Returns the string text left-padded with the string pad to a length of len characters. If text is longer than len, the return value is shortened to len characters. E.g. LPAD('hi',4,'??') returns ??hi, LPAD('hi',1,'??') returns h.

RPAD(text string, len integer, pad string)

Returns the string text right-padded with the string pad to a length of len characters. If text is longer than len, the return value is shortened to len characters. E.g. RPAD('hi',4,'??') returns hi??, RPAD('hi',1,'??') returns h.

Conditional functions Description
CASE value
WHEN value1 [, value11 ]* THEN result1
[ WHEN valueN [, valueN1 ]* THEN resultN ]*
[ ELSE resultZ ]
END

Simple case.

CASE
WHEN condition1 THEN result1
[ WHEN conditionN THEN resultN ]*
[ ELSE resultZ ]
END

Searched case.

NULLIF(value, value)

Returns NULL if the values are the same. For example, NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5.

COALESCE(value, value [, value ]* )

Provides a value if the first value is null. For example, COALESCE(NULL, 5) returns 5.

Type conversion functions Description
CAST(value AS type)

Converts a value to a given type.

Temporal functions Description
DATE string

Parses a date string in the form "yy-mm-dd" to a SQL date.

TIME string

Parses a time string in the form "hh:mm:ss" to a SQL time.

TIMESTAMP string

Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL timestamp.

INTERVAL string range

Parses an interval string in the form "dd hh:mm:ss.fff" for SQL intervals of milliseconds or "yyyy-mm" for SQL intervals of months. An interval range might be e.g. DAY, MINUTE, DAY TO HOUR, or DAY TO SECOND for intervals of milliseconds; YEAR or YEAR TO MONTH for intervals of months. E.g. INTERVAL '10 00:00:00.004' DAY TO SECOND, INTERVAL '10' DAY, or INTERVAL '2-10' YEAR TO MONTH return intervals.

CURRENT_DATE

Returns the current SQL date in UTC time zone.

CURRENT_TIME

Returns the current SQL time in UTC time zone.

CURRENT_TIMESTAMP

Returns the current SQL timestamp in UTC time zone.

LOCALTIME

Returns the current SQL time in local time zone.

LOCALTIMESTAMP

Returns the current SQL timestamp in local time zone.

EXTRACT(timeintervalunit FROM temporal)

Extracts parts of a time point or time interval. Returns the part as a long value. E.g. EXTRACT(DAY FROM DATE '2006-06-05') leads to 5.

YEAR(date)

Returns the year from a SQL date. Equivalent to EXTRACT(YEAR FROM date). E.g. YEAR(DATE '1994-09-27') leads to 1994.

FLOOR(timepoint TO timeintervalunit)

Rounds a time point down to the given unit. E.g. FLOOR(TIME '12:44:31' TO MINUTE) leads to 12:44:00.

CEIL(timepoint TO timeintervalunit)

Rounds a time point up to the given unit. E.g. CEIL(TIME '12:44:31' TO MINUTE) leads to 12:45:00.

QUARTER(date)

Returns the quarter of a year from a SQL date (an integer between 1 and 4). Equivalent to EXTRACT(QUARTER FROM date). E.g. QUARTER(DATE '1994-09-27') leads to 3.

MONTH(date)

Returns the month of a year from a SQL date (an integer between 1 and 12). Equivalent to EXTRACT(MONTH FROM date). E.g. MONTH(DATE '1994-09-27') leads to 9.

WEEK(date)

Returns the week of a year from a SQL date (an integer between 1 and 53). Equivalent to EXTRACT(WEEK FROM date). E.g. WEEK(DATE '1994-09-27') leads to 39.

DAYOFYEAR(date)

Returns the day of a year from a SQL date (an integer between 1 and 366). Equivalent to EXTRACT(DOY FROM date). E.g. DAYOFYEAR(DATE '1994-09-27') leads to 270.

DAYOFMONTH(date)

Returns the day of a month from a SQL date (an integer between 1 and 31). Equivalent to EXTRACT(DAY FROM date). E.g. DAYOFMONTH(DATE '1994-09-27') leads to 27.

DAYOFWEEK(date)

Returns the day of a week from a SQL date (an integer between 1 and 7; Sunday = 1). Equivalent to EXTRACT(DOW FROM date). E.g. DAYOFWEEK(DATE '1994-09-27') leads to 3.

HOUR(timestamp)

Returns the hour of a day from a SQL timestamp (an integer between 0 and 23). Equivalent to EXTRACT(HOUR FROM timestamp). E.g. HOUR(TIMESTAMP '1994-09-27 13:14:15') leads to 13.

MINUTE(timestamp)

Returns the minute of an hour from a SQL timestamp (an integer between 0 and 59). Equivalent to EXTRACT(MINUTE FROM timestamp). E.g. MINUTE(TIMESTAMP '1994-09-27 13:14:15') leads to 14.

SECOND(timestamp)

Returns the second of a minute from a SQL timestamp (an integer between 0 and 59). Equivalent to EXTRACT(SECOND FROM timestamp). E.g. SECOND(TIMESTAMP '1994-09-27 13:14:15') leads to 15.

(timepoint, temporal) OVERLAPS (timepoint, temporal)

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 leftEnd >= rightStart && rightEnd >= leftStart. E.g. (TIME '2:55:00', INTERVAL '1' HOUR) OVERLAPS (TIME '3:30:00', INTERVAL '2' HOUR) leads to true; (TIME '9:00:00', TIME '10:00:00') OVERLAPS (TIME '10:15:00', INTERVAL '3' HOUR) leads to false.

DATE_FORMAT(timestamp, format)

Attention This function has serious bugs and should not be used for now. Please implement a custom UDF instead or use EXTRACT as a workaround.

TIMESTAMPADD(unit, interval, timestamp)

Adds a (signed) integer interval to a timestamp. The unit for the interval is given by the unit argument, which should be one of the following values: SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR. E.g. TIMESTAMPADD(WEEK, 1, '2003-01-02') leads to 2003-01-09.

Aggregate functions Description
COUNT(value [, value]* )

Returns the number of input rows for which value is not null. Use COUNT(DISTINCT value) for the number of unique values in the column or expression.

COUNT(*)

Returns the number of input rows.

AVG(numeric)

Returns the average (arithmetic mean) of numeric across all input values.

SUM(numeric)

Returns the sum of numeric across all input values. Use SUM(DISTINCT value) for the sum of unique values in the column or expression.

MAX(value)

Returns the maximum value of value across all input values.

MIN(value)

Returns the minimum value of value across all input values.

STDDEV_POP(value)

Returns the population standard deviation of the numeric field across all input values.

STDDEV_SAMP(value)

Returns the sample standard deviation of the numeric field across all input values.

VAR_POP(value)

Returns the population variance (square of the population standard deviation) of the numeric field across all input values.

VAR_SAMP(value)

Returns the sample variance (square of the sample standard deviation) of the numeric field across all input values.

COLLECT(value)

Returns a multiset of the values. null input value will be ignored. Return an empty multiset if only null values are added.

Grouping functions Description
GROUP_ID()

Returns an integer that uniquely identifies the combination of grouping keys.

GROUPING(expression)

Returns 1 if expression is rolled up in the current row’s grouping set, 0 otherwise.

GROUPING_ID(expression [, expression]* )

Returns a bit vector of the given grouping expressions.

Value access functions Description
tableName.compositeType.field

Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and returns it's value.

tableName.compositeType.*

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.

Value constructor functions Description
(value, [, value]*)

Creates a row from a list of values.

ROW(value, [, value]*)

Creates a row from a list of values.

ARRAY ‘[’ value [, value ]* ‘]’

Creates an array from a list of values.

MAP ‘[’ key, value [, key, value ]* ‘]’

Creates a map from a list of key-value pairs.

Array functions Description
CARDINALITY(ARRAY)

Returns the number of elements of an array.

array ‘[’ index ‘]’

Returns the element at a particular position in an array. The index starts at 1.

ELEMENT(ARRAY)

Returns the sole element of an array with a single element. Returns null if the array is empty. Throws an exception if the array has more than one element.

Map functions Description
CARDINALITY(MAP)

Returns the number of entries of a map.

map ‘[’ key ‘]’

Returns the value specified by a particular key in a map.

Hash functions Description
MD5(string)

Returns the MD5 hash of the string argument as a string of 32 hexadecimal digits; null if string is null.

SHA1(string)

Returns the SHA-1 hash of the string argument as a string of 40 hexadecimal digits; null if string is null.

SHA224(string)

Returns the SHA-224 hash of the string argument as a string of 56 hexadecimal digits; null if string is null.

SHA256(string)

Returns the SHA-256 hash of the string argument as a string of 64 hexadecimal digits; null if string is null.

SHA384(string)

Returns the SHA-384 hash of the string argument as a string of 96 hexadecimal digits; null if string is null.

SHA512(string)

Returns the SHA-512 hash of the string argument as a string of 128 hexadecimal digits; null if string is null.

SHA2(string, hashLength)

Returns the hash using the SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384, or SHA-512). The first argument string is the string to be hashed. hashLength is the bit length of the result (either 224, 256, 384, or 512). Returns null if string or hashLength is null.

Unsupported Functions

The following functions are not supported yet:

  • Binary string operators and functions
  • System functions

Back to top

Reserved Keywords

Although not every SQL feature is implemented yet, some string combinations are already reserved as keywords for future use. If you want to use one of the following strings as a field name, make sure to surround them with backticks (e.g. `value`, `count`).

A, ABS, ABSOLUTE, ACTION, ADA, ADD, ADMIN, AFTER, ALL, ALLOCATE, ALLOW, ALTER, ALWAYS, AND, ANY, ARE, ARRAY, AS, ASC, ASENSITIVE, ASSERTION, ASSIGNMENT, ASYMMETRIC, AT, ATOMIC, ATTRIBUTE, ATTRIBUTES, AUTHORIZATION, AVG, BEFORE, BEGIN, BERNOULLI, BETWEEN, BIGINT, BINARY, BIT, BLOB, BOOLEAN, BOTH, BREADTH, BY, C, CALL, CALLED, CARDINALITY, CASCADE, CASCADED, CASE, CAST, CATALOG, CATALOG_NAME, CEIL, CEILING, CENTURY, CHAIN, CHAR, CHARACTER, CHARACTERISTICS, CHARACTERS, CHARACTER_LENGTH, CHARACTER_SET_CATALOG, CHARACTER_SET_NAME, CHARACTER_SET_SCHEMA, CHAR_LENGTH, CHECK, CLASS_ORIGIN, CLOB, CLOSE, COALESCE, COBOL, COLLATE, COLLATION, COLLATION_CATALOG, COLLATION_NAME, COLLATION_SCHEMA, COLLECT, COLUMN, COLUMN_NAME, COMMAND_FUNCTION, COMMAND_FUNCTION_CODE, COMMIT, COMMITTED, CONDITION, CONDITION_NUMBER, CONNECT, CONNECTION, CONNECTION_NAME, CONSTRAINT, CONSTRAINTS, CONSTRAINT_CATALOG, CONSTRAINT_NAME, CONSTRAINT_SCHEMA, CONSTRUCTOR, CONTAINS, CONTINUE, CONVERT, CORR, CORRESPONDING, COUNT, COVAR_POP, COVAR_SAMP, CREATE, CROSS, CUBE, CUME_DIST, CURRENT, CURRENT_CATALOG, CURRENT_DATE, CURRENT_DEFAULT_TRANSFORM_GROUP, CURRENT_PATH, CURRENT_ROLE, CURRENT_SCHEMA, CURRENT_TIME, CURRENT_TIMESTAMP, CURRENT_TRANSFORM_GROUP_FOR_TYPE, CURRENT_USER, CURSOR, CURSOR_NAME, CYCLE, DATA, DATABASE, DATE, DATETIME_INTERVAL_CODE, DATETIME_INTERVAL_PRECISION, DAY, DEALLOCATE, DEC, DECADE, DECIMAL, DECLARE, DEFAULT, DEFAULTS, DEFERRABLE, DEFERRED, DEFINED, DEFINER, DEGREE, DELETE, DENSE_RANK, DEPTH, DEREF, DERIVED, DESC, DESCRIBE, DESCRIPTION, DESCRIPTOR, DETERMINISTIC, DIAGNOSTICS, DISALLOW, DISCONNECT, DISPATCH, DISTINCT, DOMAIN, DOUBLE, DOW, DOY, DROP, DYNAMIC, DYNAMIC_FUNCTION, DYNAMIC_FUNCTION_CODE, EACH, ELEMENT, ELSE, END, END-EXEC, EPOCH, EQUALS, ESCAPE, EVERY, EXCEPT, EXCEPTION, EXCLUDE, EXCLUDING, EXEC, EXECUTE, EXISTS, EXP, EXPLAIN, EXTEND, EXTERNAL, EXTRACT, FALSE, FETCH, FILTER, FINAL, FIRST, FIRST_VALUE, FLOAT, FLOOR, FOLLOWING, FOR, FOREIGN, FORTRAN, FOUND, FRAC_SECOND, FREE, FROM, FULL, FUNCTION, FUSION, G, GENERAL, GENERATED, GET, GLOBAL, GO, GOTO, GRANT, GRANTED, GROUP, GROUPING, HAVING, HIERARCHY, HOLD, HOUR, IDENTITY, IMMEDIATE, IMPLEMENTATION, IMPORT, IN, INCLUDING, INCREMENT, INDICATOR, INITIALLY, INNER, INOUT, INPUT, INSENSITIVE, INSERT, INSTANCE, INSTANTIABLE, INT, INTEGER, INTERSECT, INTERSECTION, INTERVAL, INTO, INVOKER, IS, ISOLATION, JAVA, JOIN, K, KEY, KEY_MEMBER, KEY_TYPE, LABEL, LANGUAGE, LARGE, LAST, LAST_VALUE, LATERAL, LEADING, LEFT, LENGTH, LEVEL, LIBRARY, LIKE, LIMIT, LN, LOCAL, LOCALTIME, LOCALTIMESTAMP, LOCATOR, LOWER, M, MAP, MATCH, MATCHED, MAX, MAXVALUE, MEMBER, MERGE, MESSAGE_LENGTH, MESSAGE_OCTET_LENGTH, MESSAGE_TEXT, METHOD, MICROSECOND, MILLENNIUM, MIN, MINUTE, MINVALUE, MOD, MODIFIES, MODULE, MONTH, MORE, MULTISET, MUMPS, NAME, NAMES, NATIONAL, NATURAL, NCHAR, NCLOB, NESTING, NEW, NEXT, NO, NONE, NORMALIZE, NORMALIZED, NOT, NULL, NULLABLE, NULLIF, NULLS, NUMBER, NUMERIC, OBJECT, OCTETS, OCTET_LENGTH, OF, OFFSET, OLD, ON, ONLY, OPEN, OPTION, OPTIONS, OR, ORDER, ORDERING, ORDINALITY, OTHERS, OUT, OUTER, OUTPUT, OVER, OVERLAPS, OVERLAY, OVERRIDING, PAD, PARAMETER, PARAMETER_MODE, PARAMETER_NAME, PARAMETER_ORDINAL_POSITION, PARAMETER_SPECIFIC_CATALOG, PARAMETER_SPECIFIC_NAME, PARAMETER_SPECIFIC_SCHEMA, PARTIAL, PARTITION, PASCAL, PASSTHROUGH, PATH, PERCENTILE_CONT, PERCENTILE_DISC, PERCENT_RANK, PLACING, PLAN, PLI, POSITION, POWER, PRECEDING, PRECISION, PREPARE, PRESERVE, PRIMARY, PRIOR, PRIVILEGES, PROCEDURE, PUBLIC, QUARTER, RANGE, RANK, READ, READS, REAL, RECURSIVE, REF, REFERENCES, REFERENCING, REGR_AVGX, REGR_AVGY, REGR_COUNT, REGR_INTERCEPT, REGR_R2, REGR_SLOPE, REGR_SXX, REGR_SXY, REGR_SYY, RELATIVE, RELEASE, REPEATABLE, RESET, RESTART, RESTRICT, RESULT, RETURN, RETURNED_CARDINALITY, RETURNED_LENGTH, RETURNED_OCTET_LENGTH, RETURNED_SQLSTATE, RETURNS, REVOKE, RIGHT, ROLE, ROLLBACK, ROLLUP, ROUTINE, ROUTINE_CATALOG, ROUTINE_NAME, ROUTINE_SCHEMA, ROW, ROWS, ROW_COUNT, ROW_NUMBER, SAVEPOINT, SCALE, SCHEMA, SCHEMA_NAME, SCOPE, SCOPE_CATALOGS, SCOPE_NAME, SCOPE_SCHEMA, SCROLL, SEARCH, SECOND, SECTION, SECURITY, SELECT, SELF, SENSITIVE, SEQUENCE, SERIALIZABLE, SERVER, SERVER_NAME, SESSION, SESSION_USER, SET, SETS, SIMILAR, SIMPLE, SIZE, SMALLINT, SOME, SOURCE, SPACE, SPECIFIC, SPECIFICTYPE, SPECIFIC_NAME, SQL, SQLEXCEPTION, SQLSTATE, SQLWARNING, SQL_TSI_DAY, SQL_TSI_FRAC_SECOND, SQL_TSI_HOUR, SQL_TSI_MICROSECOND, SQL_TSI_MINUTE, SQL_TSI_MONTH, SQL_TSI_QUARTER, SQL_TSI_SECOND, SQL_TSI_WEEK, SQL_TSI_YEAR, SQRT, START, STATE, STATEMENT, STATIC, STDDEV_POP, STDDEV_SAMP, STREAM, STRUCTURE, STYLE, SUBCLASS_ORIGIN, SUBMULTISET, SUBSTITUTE, SUBSTRING, SUM, SYMMETRIC, SYSTEM, SYSTEM_USER, TABLE, TABLESAMPLE, TABLE_NAME, TEMPORARY, THEN, TIES, TIME, TIMESTAMP, TIMESTAMPADD, TIMESTAMPDIFF, TIMEZONE_HOUR, TIMEZONE_MINUTE, TINYINT, TO, TOP_LEVEL_COUNT, TRAILING, TRANSACTION, TRANSACTIONS_ACTIVE, TRANSACTIONS_COMMITTED, TRANSACTIONS_ROLLED_BACK, TRANSFORM, TRANSFORMS, TRANSLATE, TRANSLATION, TREAT, TRIGGER, TRIGGER_CATALOG, TRIGGER_NAME, TRIGGER_SCHEMA, TRIM, TRUE, TYPE, UESCAPE, UNBOUNDED, UNCOMMITTED, UNDER, UNION, UNIQUE, UNKNOWN, UNNAMED, UNNEST, UPDATE, UPPER, UPSERT, USAGE, USER, USER_DEFINED_TYPE_CATALOG, USER_DEFINED_TYPE_CODE, USER_DEFINED_TYPE_NAME, USER_DEFINED_TYPE_SCHEMA, USING, VALUE, VALUES, VARBINARY, VARCHAR, VARYING, VAR_POP, VAR_SAMP, VERSION, VIEW, WEEK, WHEN, WHENEVER, WHERE, WIDTH_BUCKET, WINDOW, WITH, WITHIN, WITHOUT, WORK, WRAPPER, WRITE, XML, YEAR, ZONE

Date Format Specifier

Specifier Description
%a
Abbreviated weekday name (Sun .. Sat)
%b
Abbreviated month name (Jan .. Dec)
%c
Month, numeric (1 .. 12)
%D
Day of the month with English suffix (0th, 1st, 2nd, 3rd, ...)
%d
Day of the month, numeric (01 .. 31)
%e
Day of the month, numeric (1 .. 31)
%f
Fraction of second (6 digits for printing: 000000 .. 999000; 1 - 9 digits for parsing: 0 .. 999999999) (Timestamp is truncated to milliseconds.)
%H
Hour (00 .. 23)
%h
Hour (01 .. 12)
%I
Hour (01 .. 12)
%i
Minutes, numeric (00 .. 59)
%j
Day of year (001 .. 366)
%k
Hour (0 .. 23)
%l
Hour (1 .. 12)
%M
Month name (January .. December)
%m
Month, numeric (01 .. 12)
%p
AM or PM
%r
Time, 12-hour (hh:mm:ss followed by AM or PM)
%S
Seconds (00 .. 59)
%s
Seconds (00 .. 59)
%T
Time, 24-hour (hh:mm:ss)
%U
Week (00 .. 53), where Sunday is the first day of the week
%u
Week (00 .. 53), where Monday is the first day of the week
%V
Week (01 .. 53), where Sunday is the first day of the week; used with %X
%v
Week (01 .. 53), where Monday is the first day of the week; used with %x
%W
Weekday name (Sunday .. Saturday)
%w
Day of the week (0 .. 6), where Sunday is the first day of the week
%X
Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V
%x
Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v
%Y
Year, numeric, four digits
%y
Year, numeric (two digits)
%%
A literal % character
%x
x, for any x not listed above

Back to top