You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/26 10:15:20 UTC

[flink] 01/02: [FLINK-13359][docs] Add documentation for DDL introduction

This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 00b6e8bd3ae3943c24e0538debcae82df35dac4d
Author: yuzhao.cyz <yu...@alibaba-inc.com>
AuthorDate: Tue Aug 6 12:53:35 2019 +0800

    [FLINK-13359][docs] Add documentation for DDL introduction
    
    This closes #9366
---
 docs/dev/table/sql.md    | 163 ++++++++++++++++++++++++++++++++++++-------
 docs/dev/table/sql.zh.md | 177 ++++++++++++++++++++++++++++++++++++++++-------
 2 files changed, 290 insertions(+), 50 deletions(-)

diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md
index e607716..79f0b41 100644
--- a/docs/dev/table/sql.md
+++ b/docs/dev/table/sql.md
@@ -22,19 +22,20 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+This is a complete list of Data Definition Language (DDL) and Data Manipulation Language (DML) constructs supported in Flink.
+* This will be replaced by the TOC
+{:toc} 
+
+## Query
 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](common.html#mixing-table-api-and-sql), be [converted into a DataSet or DataStream](common.html#integration-with-datastream-and-dataset-api), or [written to a TableSink](common.html#emit-a-table)). SQL and Table API queries can be seamlessly mixed and are holistically optimized and tra [...]
 
-In order to access a table in a SQL query, it must be [registered in the TableEnvironment](common.html#register-tables-in-the-catalog). A table can be registered from a [TableSource](common.html#register-a-tablesource), [Table](common.html#register-a-table), [DataStream, or DataSet](common.html#register-a-datastream-or-dataset-as-table). Alternatively, users can also [register external catalogs in a TableEnvironment](common.html#register-an-external-catalog) to specify the location of th [...]
+In order to access a table in a SQL query, it must be [registered in the TableEnvironment](common.html#register-tables-in-the-catalog). A table can be registered from a [TableSource](common.html#register-a-tablesource), [Table](common.html#register-a-table), [CREATE TABLE statement](#create-table), [DataStream, or DataSet](common.html#register-a-datastream-or-dataset-as-table). Alternatively, users can also [register external catalogs in a TableEnvironment](common.html#register-an-extern [...]
 
 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.
 
-* This will be replaced by the TOC
-{:toc}
-
-Specifying a Query
-------------------
+### Specifying a Query
 
 The following examples show how to specify a SQL queries on registered and inlined tables.
 
@@ -130,8 +131,7 @@ table_env \
 
 {% top %}
 
-Supported Syntax
-----------------
+### Supported Syntax
 
 Flink parses SQL using [Apache Calcite](https://calcite.apache.org/docs/reference.html), which supports standard ANSI SQL. DDL statements are not supported by Flink.
 
@@ -276,10 +276,9 @@ String literals must be enclosed in single quotes (e.g., `SELECT 'Hello World'`)
 
 {% top %}
 
-Operations
---------------------
+### Operations
 
-### Show and Use
+#### Show and Use
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -330,7 +329,7 @@ USE mydatabase;
 </table>
 </div>
 
-### Scan, Projection, and Filter
+#### Scan, Projection, and Filter
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -385,7 +384,7 @@ SELECT PRETTY_PRINT(user) FROM Orders
 
 {% top %}
 
-### Aggregations
+#### Aggregations
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -509,7 +508,7 @@ GROUP BY users
 
 {% top %}
 
-### Joins
+#### Joins
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -655,7 +654,7 @@ WHERE
 
 {% top %}
 
-### Set Operations
+#### Set Operations
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -765,7 +764,7 @@ WHERE product EXISTS (
 
 {% top %}
 
-### OrderBy & Limit
+#### OrderBy & Limit
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -813,7 +812,7 @@ LIMIT 3
 
 {% top %}
 
-### Insert
+#### Insert
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -846,7 +845,7 @@ FROM Orders
 
 {% top %}
 
-### Group Windows
+#### 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.
 
@@ -874,13 +873,13 @@ Group windows are defined in the `GROUP BY` clause of a SQL query. Just like que
   </tbody>
 </table>
 
-#### Time Attributes
+##### 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](streaming/time_attributes.html) 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
+##### 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:
 
@@ -1019,7 +1018,7 @@ val result4 = tableEnv.sqlQuery(
 
 {% top %}
 
-### Pattern Recognition
+#### Pattern Recognition
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -1065,8 +1064,119 @@ MATCH_RECOGNIZE (
 
 {% top %}
 
-Data Types
-----------
+## DDL
+
+DDLs are specified with the `sqlUpdate()` method of the `TableEnvironment`. The method returns nothing for a success table creation. A `Table` can be register into the [Catalog](catalogs.html) with a `CREATE TABLE` statement, then can be referenced in SQL queries in method `sqlQuery()` of `TableEnvironment`.
+
+**Note:** Flink's DDL support is not yet feature complete. Queries that include unsupported SQL features cause a `TableException`. The supported features of SQL DDL on batch and streaming tables are listed in the following sections.
+
+### Specifying a DDL
+
+The following examples show how to specify a SQL DDL.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
+
+// SQL query with a registered table
+// register a table named "Orders"
+tableEnv.sqlUpdate("CREATE TABLE Orders (`user` BIGINT, product VARCHAR, amount INT) WITH (...)");
+// run a SQL query on the Table and retrieve the result as a new Table
+Table result = tableEnv.sqlQuery(
+  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");
+
+// SQL update with a registered table
+// register a TableSink
+tableEnv.sqlUpdate("CREATE TABLE RubberOrders(product VARCHAR, amount INT) WITH (...)");
+// 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%'");
+{% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+val tableEnv = StreamTableEnvironment.create(env)
+
+// SQL query with a registered table
+// register a table named "Orders"
+tableEnv.sqlUpdate("CREATE TABLE Orders (`user` BIGINT, product VARCHAR, amount INT) WITH (...)");
+// run a SQL query on the Table and retrieve the result as a new Table
+val result = tableEnv.sqlQuery(
+  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");
+
+// SQL update with a registered table
+// register a TableSink
+tableEnv.sqlUpdate("CREATE TABLE RubberOrders(product VARCHAR, amount INT) WITH ('connector.path'='/path/to/file' ...)");
+// 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%'")
+{% endhighlight %}
+</div>
+
+<div data-lang="python" markdown="1">
+{% highlight python %}
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+
+# SQL update with a registered table
+# register a TableSink
+table_env.sql_update("CREATE TABLE RubberOrders(product VARCHAR, amount INT) with (...)")
+# run a SQL update query on the Table and emit the result to the TableSink
+table_env \
+    .sql_update("INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'")
+{% endhighlight %}
+</div>
+</div>
+
+{% top %}
+
+### Create Table
+
+{% highlight sql %}
+CREATE TABLE [catalog_name.][db_name.]table_name
+  [(col_name1 col_type1 [COMMENT col_comment1], ...)]
+  [COMMENT table_comment]
+  [PARTITIONED BY (col_name1, col_name2, ...)]
+  WITH (key1=val1, key2=val2, ...)
+{% endhighlight %}
+
+Create a table with the given table properties. If a table with the same name already exists in the database, an exception is thrown.
+
+**PARTITIONED BY**
+
+Partition the created table by the specified columns. A directory is created for each partition if this table is used as a filesystem sink.
+
+**WITH OPTIONS**
+
+Table properties used to create a table source/sink. The properties are usually used to find and create the underlying connector.
+
+The key and value of expression `key1=val1` should both be string literal. See details in [Connect to External Systems](connect.html) for all the supported table properties of different connectors.
+
+**Notes:** The table name can be of three formats: 1. `catalog_name.db_name.table_name` 2. `db_name.table_name` 3. `table_name`. For `catalog_name.db_name.table_name`, the table would be registered into metastore with catalog named "catalog_name" and database named "db_name"; for `db_name.table_name`, the table would be registered into the current catalog of the execution table environment and database named "db_name"; for `table_name`, the table would be registered into the current cata [...]
+
+**Notes:** The table registered with `CREATE TABLE` statement can be used as both table source and table sink, we can not decide if it is used as a source or sink until it is referenced in the DMLs.
+
+{% top %}
+
+### Drop Table
+
+{% highlight sql %}
+DROP TABLE [IF EXISTS] [catalog_name.][db_name.]table_name
+{% endhighlight %}
+
+Drop a table with the given table name. If the table to drop does not exist, an exception is thrown.
+
+**IF EXISTS**
+
+If the table does not exist, nothing happens.
+
+{% top %}
+
+## Data Types
 
 Please see the dedicated page about [data types](types.html).
 
@@ -1076,10 +1186,13 @@ Fields of composite types with arbitrary nesting can be accessed with [value acc
 
 Generic types are treated as a black box and can be passed on or processed by [user-defined functions](udfs.html).
 
+For DDLs, we support full data types defined in page [Data Types]({{ site.baseurl }}/dev/table/types.html).
+
+**Notes:** Some of the data types are not supported in the sql query(the cast expression or literals). E.G. `STRING`, `BYTES`, `TIME(p) WITHOUT TIME ZONE`, `TIME(p) WITH LOCAL TIME ZONE`, `TIMESTAMP(p) WITHOUT TIME ZONE`, `TIMESTAMP(p) WITH LOCAL TIME ZONE`, `ARRAY`, `MULTISET`, `ROW`.
+
 {% top %}
 
-Reserved Keywords
------------------
+## 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` ``).
 
diff --git a/docs/dev/table/sql.zh.md b/docs/dev/table/sql.zh.md
index 4efddfb..fa3b050 100644
--- a/docs/dev/table/sql.zh.md
+++ b/docs/dev/table/sql.zh.md
@@ -22,19 +22,20 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+This is a complete list of Data Definition Language (DDL) and Data Manipulation Language (DML) constructs supported in Flink.
+* This will be replaced by the TOC
+{:toc} 
+
+## Query
 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](common.html#mixing-table-api-and-sql), be [converted into a DataSet or DataStream](common.html#integration-with-datastream-and-dataset-api), or [written to a TableSink](common.html#emit-a-table)). SQL and Table API queries can be seamlessly mixed and are holistically optimized and tra [...]
 
-In order to access a table in a SQL query, it must be [registered in the TableEnvironment](common.html#register-tables-in-the-catalog). A table can be registered from a [TableSource](common.html#register-a-tablesource), [Table](common.html#register-a-table), [DataStream, or DataSet](common.html#register-a-datastream-or-dataset-as-table). Alternatively, users can also [register external catalogs in a TableEnvironment](common.html#register-an-external-catalog) to specify the location of th [...]
+In order to access a table in a SQL query, it must be [registered in the TableEnvironment](common.html#register-tables-in-the-catalog). A table can be registered from a [TableSource](common.html#register-a-tablesource), [Table](common.html#register-a-table), [CREATE TABLE statement](#create-table), [DataStream, or DataSet](common.html#register-a-datastream-or-dataset-as-table). Alternatively, users can also [register external catalogs in a TableEnvironment](common.html#register-an-extern [...]
 
 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.
 
-* This will be replaced by the TOC
-{:toc}
-
-Specifying a Query
-------------------
+### Specifying a Query
 
 The following examples show how to specify a SQL queries on registered and inlined tables.
 
@@ -130,8 +131,7 @@ table_env \
 
 {% top %}
 
-Supported Syntax
-----------------
+### Supported Syntax
 
 Flink parses SQL using [Apache Calcite](https://calcite.apache.org/docs/reference.html), which supports standard ANSI SQL. DDL statements are not supported by Flink.
 
@@ -276,10 +276,9 @@ String literals must be enclosed in single quotes (e.g., `SELECT 'Hello World'`)
 
 {% top %}
 
-Operations
---------------------
+### Operations
 
-### Show and Use
+#### Show and Use
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -385,7 +384,7 @@ SELECT PRETTY_PRINT(user) FROM Orders
 
 {% top %}
 
-### Aggregations
+#### Aggregations
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -509,7 +508,7 @@ GROUP BY users
 
 {% top %}
 
-### Joins
+#### Joins
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -655,7 +654,7 @@ WHERE
 
 {% top %}
 
-### Set Operations
+#### Set Operations
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -765,7 +764,7 @@ WHERE product EXISTS (
 
 {% top %}
 
-### OrderBy & Limit
+#### OrderBy & Limit
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -797,7 +796,7 @@ ORDER BY orderTime
         <span class="label label-primary">Batch</span>
       </td>
       <td>
-<b>Note:</b> The LIMIT clause requires an ORDER BY clause.
+<b>Note:</b> The LIMIT clause requires an ORDER BY clause. 
 {% highlight sql %}
 SELECT *
 FROM Orders
@@ -813,7 +812,7 @@ LIMIT 3
 
 {% top %}
 
-### Insert
+#### Insert
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -846,7 +845,7 @@ FROM Orders
 
 {% top %}
 
-### Group Windows
+#### 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.
 
@@ -874,13 +873,13 @@ Group windows are defined in the `GROUP BY` clause of a SQL query. Just like que
   </tbody>
 </table>
 
-#### Time Attributes
+##### 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](streaming/time_attributes.html) 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
+##### 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:
 
@@ -1019,7 +1018,7 @@ val result4 = tableEnv.sqlQuery(
 
 {% top %}
 
-### Pattern Recognition
+#### Pattern Recognition
 
 <div markdown="1">
 <table class="table table-bordered">
@@ -1065,8 +1064,133 @@ MATCH_RECOGNIZE (
 
 {% top %}
 
-Data Types
-----------
+### Drop Table
+
+{% highlight sql %}
+DROP TABLE [IF EXISTS] [catalog_name.][db_name.]table_name
+{% endhighlight %}
+
+Drop a table with the given table name. If the table to drop does not exist, an exception is thrown.
+
+**IF EXISTS**
+
+If the table does not exist, nothing happens.
+
+{% top %}
+
+## DDL
+
+DDLs are specified with the `sqlUpdate()` method of the `TableEnvironment`. The method returns nothing for a success table creation. A `Table` can be register into the [Catalog](catalog.html) with a `CREATE TABLE` statement, then be referenced in the SQL queries in method `sqlQuery()` of `TableEnvironment`.
+
+**Note:** Flink's DDL support is not yet feature complete. Queries that include unsupported SQL features cause a `TableException`. The supported features of SQL DDL on batch and streaming tables are listed in the following sections.
+
+### Specifying a DDL
+
+The following examples show how to specify a SQL DDL.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
+
+// SQL query with a registered table
+// register a table named "Orders"
+tableEnv.sqlUpdate("CREATE TABLE Orders (`user` BIGINT, product VARCHAR, amount INT) WITH (...)");
+// run a SQL query on the Table and retrieve the result as a new Table
+Table result = tableEnv.sqlQuery(
+  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");
+
+// SQL update with a registered table
+// register a TableSink
+tableEnv.sqlUpdate("CREATE TABLE RubberOrders(product VARCHAR, amount INT) WITH (...)");
+// 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%'");
+{% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+val tableEnv = StreamTableEnvironment.create(env)
+
+// SQL query with a registered table
+// register a table named "Orders"
+tableEnv.sqlUpdate("CREATE TABLE Orders (`user` BIGINT, product VARCHAR, amount INT) WITH (...)");
+// run a SQL query on the Table and retrieve the result as a new Table
+val result = tableEnv.sqlQuery(
+  "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'");
+
+// SQL update with a registered table
+// register a TableSink
+tableEnv.sqlUpdate("CREATE TABLE RubberOrders(product VARCHAR, amount INT) WITH ('connector.path'='/path/to/file' ...)");
+// 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%'")
+{% endhighlight %}
+</div>
+
+<div data-lang="python" markdown="1">
+{% highlight python %}
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+
+# SQL update with a registered table
+# register a TableSink
+table_env.sql_update("CREATE TABLE RubberOrders(product VARCHAR, amount INT) with (...)")
+# run a SQL update query on the Table and emit the result to the TableSink
+table_env \
+    .sql_update("INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'")
+{% endhighlight %}
+</div>
+</div>
+
+{% top %}
+
+### Create Table
+
+{% highlight sql %}
+CREATE TABLE [catalog_name.][db_name.]table_name
+  [(col_name1 col_type1 [COMMENT col_comment1], ...)]
+  [COMMENT table_comment]
+  [PARTITIONED BY (col_name1, col_name2, ...)]
+  WITH (key1=val1, key2=val2, ...)
+{% endhighlight %}
+
+Create a table with the given table properties. If a table with the same name already exists in the database, an exception is thrown.
+
+**PARTITIONED BY**
+
+Partition the created table by the specified columns. A directory is created for each partition if this table is used as a filesystem sink.
+
+**WITH OPTIONS**
+
+Table properties used to create a table source/sink. The properties are usually used to find and create the underlying connector.
+
+The key and value of expression `key1=val1` should both be string literal. See details in [Connect to External Systems](connect.html) for all the supported table properties of different connectors.
+
+**Notes:** The table name can be of three formats: 1. `catalog_name.db_name.table_name` 2. `db_name.table_name` 3. `table_name`. For `catalog_name.db_name.table_name`, the table would be registered into metastore with catalog named "catalog_name" and database named "db_name"; for `db_name.table_name`, the table would be registered into the current catalog of the execution table environment and database named "db_name"; for `table_name`, the table would be registered into the current cata [...]
+
+**Notes:** The table registered with `CREATE TABLE` statement can be used as both table source and table sink, we can not decide if it is used as a source or sink until it is referenced in the DMLs.
+
+{% top %}
+
+### Drop Table
+
+{% highlight sql %}
+DROP TABLE [IF EXISTS] [catalog_name.][db_name.]table_name
+{% endhighlight %}
+
+Drop a table with the given table name. If the table to drop does not exist, an exception is thrown.
+
+**IF EXISTS**
+
+If the table does not exist, nothing happens.
+
+{% top %}
+
+## Data Types
 
 Please see the dedicated page about [data types](types.html).
 
@@ -1076,10 +1200,13 @@ Fields of composite types with arbitrary nesting can be accessed with [value acc
 
 Generic types are treated as a black box and can be passed on or processed by [user-defined functions](udfs.html).
 
+For DDLs, we support full data types defined in page [Data Types]({{ site.baseurl }}/dev/table/types.html).
+
+**Notes:** Some of the data types are not supported in the sql query(the cast expression or literals). E.G. `STRING`, `BYTES`, `TIME(p) WITHOUT TIME ZONE`, `TIME(p) WITH LOCAL TIME ZONE`, `TIMESTAMP(p) WITHOUT TIME ZONE`, `TIMESTAMP(p) WITH LOCAL TIME ZONE`, `ARRAY`, `MULTISET`, `ROW`.
+
 {% top %}
 
-Reserved Keywords
------------------
+## 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` ``).