SQL queries are specified with the sql()
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.
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.toTable(ds, "user, product, amount");
Table result = tableEnv.sql(
"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.sql(
"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.sql(
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.sql(
"SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'")
Flink parses SQL using Apache Calcite, which supports standard ANSI SQL. DML and 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.
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 ]
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 ]* ')'
Flink SQL uses a lexical policy for identifier (table, attribute, function names) similar to Java:
"SELECT a AS `my field` FROM t"
).Operation | Description |
---|---|
Scan / Select / As Batch Streaming |
|
Where / Filter Batch Streaming |
|
User-defined Scalar Functions (Scalar UDF) Batch Streaming |
UDFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register scalar UDFs.
|
Operation | Description |
---|---|
GroupBy Aggregation Batch Streaming Result Updating |
Note: GroupBy on a streaming table produces an updating result. See the Streaming Concepts page for details.
|
GroupBy Window Aggregation Batch Streaming |
Use a group window to compute a single result row per group. See Group Windows section for more details.
|
Over Window aggregation Streaming |
Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute
|
Distinct Batch |
|
Grouping sets, Rollup, Cube Batch |
|
Having Batch Streaming |
|
User-defined Aggregate Functions (UDAGG) Batch Streaming |
UDAGGs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDAGGs.
|
Operation | Description |
---|---|
Inner Equi-join / Outer Equi-join Batch |
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.
|
Expanding arrays into a relation Batch Streaming |
Unnesting WITH ORDINALITY is not supported yet.
|
Join with User Defined Table Functions (UDTF) Batch Streaming |
UDTFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDTFs. Inner Join
Left Outer Join
Note: Currently, only literal |
Operation | Description |
---|---|
Union Batch |
|
UnionAll Batch Streaming |
|
Intersect / Except Batch |
|
Operation | Description |
---|---|
Order By Batch |
|
Limit Batch |
|
Group windows are defined in the GROUP BY
clause of a SQL query. Just like queries with regular GROUP BY
clauses, queries with a GROUP BY
clause that includes a group window function compute a single result row per group. The following group windows functions are supported for SQL on batch and streaming tables.
Group Window Function | Description |
---|---|
TUMBLE(time_attr, interval) |
Defines a tumbling time window. A tumbling time window assigns rows to non-overlapping, continuous windows with a fixed duration (interval ). For example, a tumbling window of 5 minutes groups rows in 5 minutes intervals. Tumbling windows can be defined on event-time (stream + batch) or processing-time (stream). |
HOP(time_attr, interval, interval) |
Defines a hopping time window (called sliding window in the Table API). A hopping time window has a fixed duration (second interval parameter) and hops by a specified hop interval (first interval parameter). If the hop interval is smaller than the window size, hopping windows are overlapping. Thus, rows can be assigned to multiple windows. For example, a hopping window of 15 minutes size and 5 minute hop interval assigns each row to 3 different windows of 15 minute size, which are evaluated in an interval of 5 minutes. Hopping windows can be defined on event-time (stream + batch) or processing-time (stream). |
SESSION(time_attr, interval) |
Defines a session time window. Session time windows do not have a fixed duration but their bounds are defined by a time interval of inactivity, i.e., a session window is closed if no event appears for a defined gap period. For example a session window with a 30 minute gap starts when a row is observed after 30 minutes inactivity (otherwise the row would be added to an existing window) and is closed if no row is added within 30 minutes. Session windows can work on event-time (stream + batch) or processing-time (stream). |
For SQL queries on streaming tables, the time_attr
argument of the group window function must refer to a valid time attribute that specifies the processing time or event time of rows. See the documentation of time attributes to learn how to define time attributes.
For SQL on batch tables, the time_attr
argument of the group window function must be an attribute of type TIMESTAMP
.
The start and end timestamps of group windows 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 start timestamp of the corresponding tumbling, hopping, and session window. |
TUMBLE_END(time_attr, interval) HOP_END(time_attr, interval, interval) SESSION_END(time_attr, interval) |
Returns the end timestamp of the corresponding tumbling, hopping, and session window. |
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.sql(
"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.sql(
"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.sql(
"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.sql(
"SELECT user, " +
" SESSION_START(rowtime, INTERVAL '12' HOUR) AS sStart, " +
" SESSION_END(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.sql(
"""
|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.sql(
"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.sql(
"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.sql(
"""
|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)
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.DATE |
DATE |
java.sql.Date |
Types.TIME |
TIME |
java.sql.Time |
Types.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 |
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).
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 |
---|---|
|
Equals. |
|
Not equal. |
|
Greater than. |
|
Greater than or equal. |
|
Less than. |
|
Less than or equal. |
|
Returns TRUE if value is null. |
|
Returns TRUE if value is not null. |
|
Returns TRUE if two values are not equal, treating null values as the same. |
|
Returns TRUE if two values are equal, treating null values as the same. |
|
Returns TRUE if value1 is greater than or equal to value2 and less than or equal to value3. |
|
Returns TRUE if value1 is less than value2 or greater than value3. |
|
Returns TRUE if string1 matches pattern string2. An escape character can be defined if necessary. |
|
Returns TRUE if string1 does not match pattern string2. An escape character can be defined if necessary. |
|
Returns TRUE if string1 matches regular expression string2. An escape character can be defined if necessary. |
|
Returns TRUE if string1 does not match regular expression string2. An escape character can be defined if necessary. |
|
Returns TRUE if value is equal to a value in a list. |
|
Returns TRUE if value is not equal to every value in a list. |
|
Returns TRUE if sub-query returns at least one row. Only supported if the operation can be rewritten in a join and group operation. |
Logical functions | Description |
---|---|
|
Returns TRUE if boolean1 is TRUE or boolean2 is TRUE. Supports three-valued logic. |
|
Returns TRUE if boolean1 and boolean2 are both TRUE. Supports three-valued logic. |
|
Returns TRUE if boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN. |
|
Returns TRUE if boolean is FALSE; returns FALSE if boolean is UNKNOWN. |
|
Returns TRUE if boolean is not FALSE; returns TRUE if boolean is UNKNOWN. |
|
Returns TRUE if boolean is TRUE; returns FALSE if boolean is UNKNOWN. |
|
Returns TRUE if boolean is not TRUE; returns TRUE if boolean is UNKNOWN. |
|
Returns TRUE if boolean is UNKNOWN. |
|
Returns TRUE if boolean is not UNKNOWN. |
Arithmetic functions | Description |
---|---|
|
Returns numeric. |
|
Returns negative numeric. |
|
Returns numeric1 plus numeric2. |
|
Returns numeric1 minus numeric2. |
|
Returns numeric1 multiplied by numeric2. |
|
Returns numeric1 divided by numeric2. |
|
Returns numeric1 raised to the power of numeric2. |
|
Returns the absolute value of numeric. |
|
Returns the remainder (modulus) of numeric1 divided by numeric2. The result is negative only if numeric1 is negative. |
|
Returns the square root of numeric. |
|
Returns the natural logarithm (base e) of numeric. |
|
Returns the base 10 logarithm of numeric. |
|
Returns e raised to the power of numeric. |
|
Rounds numeric up, and returns the smallest number that is greater than or equal to numeric. |
|
Rounds numeric down, and returns the largest number that is less than or equal to numeric. |
|
Calculates the sine of a given number. |
|
Calculates the cosine of a given number. |
|
Calculates the tangent of a given number. |
|
Calculates the cotangent of a given number. |
|
Calculates the arc sine of a given number. |
|
Calculates the arc cosine of a given number. |
|
Calculates the arc tangent of a given number. |
|
Converts numeric from radians to degrees. |
|
Converts numeric from degrees to radians. |
|
Calculates the signum of a given number. |
|
Rounds the given number to integer places right to the decimal point. |
|
Returns a value that is closer than any other value to pi. |
String functions | Description |
---|---|
|
Concatenates two character strings. |
|
Returns the number of characters in a character string. |
|
As CHAR_LENGTH(string). |
|
Returns a character string converted to upper case. |
|
Returns a character string converted to lower case. |
|
Returns the position of the first occurrence of string1 in string2. |
|
Removes leading and/or trailing characters from string2. By default, whitespaces at both sides are removed. |
|
Replaces a substring of string1 with string2. |
|
Returns a substring of a character string starting at a given point. |
|
Returns a substring of a character string starting at a given point with a given length. |
|
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. |
Conditional functions | Description |
---|---|
|
Simple case. |
|
Searched case. |
|
Returns NULL if the values are the same. For example, |
|
Provides a value if the first value is null. For example, |
Type conversion functions | Description |
---|---|
|
Converts a value to a given type. |
Temporal functions | Description |
---|---|
|
Parses a date string in the form "yy-mm-dd" to a SQL date. |
|
Parses a time string in the form "hh:mm:ss" to a SQL time. |
|
Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL timestamp. |
|
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. |
|
Returns the current SQL date in UTC time zone. |
|
Returns the current SQL time in UTC time zone. |
|
Returns the current SQL timestamp in UTC time zone. |
|
Returns the current SQL time in local time zone. |
|
Returns the current SQL timestamp in local time zone. |
|
Extracts parts of a time point or time interval. Returns the part as a long value. E.g. |
|
Rounds a time point down to the given unit. E.g. |
|
Rounds a time point up to the given unit. E.g. |
|
Returns the quarter of a year from a SQL date. E.g. |
|
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 |
Aggregate functions | Description |
---|---|
|
Returns the number of input rows for which value is not null. |
|
Returns the number of input rows. |
|
Returns the average (arithmetic mean) of numeric across all input values. |
|
Returns the sum of numeric across all input values. |
|
Returns the maximum value of value across all input values. |
|
Returns the minimum value of value across all input values. |
|
Returns the population standard deviation of the numeric field across all input values. |
|
Returns the sample standard deviation of the numeric field across all input values. |
|
Returns the population variance (square of the population standard deviation) of the numeric field across all input values. |
|
Returns the sample variance (square of the sample standard deviation) of the numeric field across all input values. |
Grouping functions | Description |
---|---|
|
Returns an integer that uniquely identifies the combination of grouping keys. |
|
Returns 1 if expression is rolled up in the current row’s grouping set, 0 otherwise. |
|
Returns a bit vector of the given grouping expressions. |
Value access functions | Description |
---|---|
|
Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and returns it's value. |
|
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. |
Array functions | Description |
---|---|
|
Creates an array from a list of values. |
|
Returns the number of elements of an array. |
|
Returns the element at a particular position in an array. The index starts at 1. |
|
Returns the sole element of an array with a single element. Returns |
The following functions are not supported yet:
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, CHARACTERISTICTS, 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