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

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


The following commit(s) were added to refs/heads/release-1.12 by this push:
     new a32803a  [FLINK-18897][docs] Add document for maxwell json format
a32803a is described below

commit a32803aafab46334ea5c8ef4963727a88617646a
Author: Shengkai <[email protected]>
AuthorDate: Mon Dec 7 23:20:33 2020 +0800

    [FLINK-18897][docs] Add document for maxwell json format
    
    This closes #14245
---
 docs/_data/sql-connectors.yml                   |   6 +
 docs/dev/table/connectors/formats/index.md      |   5 +
 docs/dev/table/connectors/formats/index.zh.md   |   5 +
 docs/dev/table/connectors/formats/maxwell.md    | 208 ++++++++++++++++++++++++
 docs/dev/table/connectors/formats/maxwell.zh.md | 208 ++++++++++++++++++++++++
 docs/dev/table/connectors/formats/orc.md        |   6 +-
 docs/dev/table/connectors/formats/orc.zh.md     |   6 +-
 docs/dev/table/connectors/formats/parquet.md    |   6 +-
 docs/dev/table/connectors/formats/parquet.zh.md |   6 +-
 docs/dev/table/connectors/formats/raw.md        |   6 +-
 docs/dev/table/connectors/formats/raw.zh.md     |   4 +-
 11 files changed, 449 insertions(+), 17 deletions(-)

diff --git a/docs/_data/sql-connectors.yml b/docs/_data/sql-connectors.yml
index 2683cda..86ed5de 100644
--- a/docs/_data/sql-connectors.yml
+++ b/docs/_data/sql-connectors.yml
@@ -78,6 +78,12 @@ canal:
     category: format
     built-in: true
 
+maxwell:
+    name: Maxwell
+    maven: flink-json
+    category: format
+    built-in: true
+
 csv:
     name: CSV
     maven: flink-csv
diff --git a/docs/dev/table/connectors/formats/index.md 
b/docs/dev/table/connectors/formats/index.md
index cef6162..9c4e1b9 100644
--- a/docs/dev/table/connectors/formats/index.md
+++ b/docs/dev/table/connectors/formats/index.md
@@ -74,6 +74,11 @@ Flink supports the following formats:
            <a href="{% link dev/table/connectors/filesystem.md 
%}">Filesystem</a></td>
         </tr>
         <tr>
+         <td><a href="{% link dev/table/connectors/formats/maxwell.md 
%}">Maxwell CDC</a></td>
+          <td><a href="{% link dev/table/connectors/kafka.md %}">Apache 
Kafka</a>,
+           <a href="{% link dev/table/connectors/filesystem.md 
%}">Filesystem</a></td>
+        </tr>
+        <tr>
          <td><a href="{% link dev/table/connectors/formats/parquet.md 
%}">Apache Parquet</a></td>
          <td><a href="{% link dev/table/connectors/filesystem.md 
%}">Filesystem</a></td>
         </tr>
diff --git a/docs/dev/table/connectors/formats/index.zh.md 
b/docs/dev/table/connectors/formats/index.zh.md
index e97fde5..81a90de 100644
--- a/docs/dev/table/connectors/formats/index.zh.md
+++ b/docs/dev/table/connectors/formats/index.zh.md
@@ -74,6 +74,11 @@ Flink 支持以下格式:
            <a href="{% link dev/table/connectors/filesystem.zh.md 
%}">Filesystem</a></td>
         </tr>
         <tr>
+         <td><a href="{% link dev/table/connectors/formats/maxwell.zh.md 
%}">Maxwell CDC</a></td>
+          <td><a href="{% link dev/table/connectors/kafka.zh.md %}">Apache 
Kafka</a>,
+           <a href="{% link dev/table/connectors/filesystem.zh.md 
%}">Filesystem</a></td>
+        </tr>
+        <tr>
          <td><a href="{% link dev/table/connectors/formats/parquet.zh.md 
%}">Apache Parquet</a></td>
          <td><a href="{% link dev/table/connectors/filesystem.zh.md 
%}">Filesystem</a></td>
         </tr>
diff --git a/docs/dev/table/connectors/formats/maxwell.md 
b/docs/dev/table/connectors/formats/maxwell.md
new file mode 100644
index 0000000..ae287e3
--- /dev/null
+++ b/docs/dev/table/connectors/formats/maxwell.md
@@ -0,0 +1,208 @@
+---
+title: "Maxwell Format"
+nav-title: Maxwell
+nav-parent_id: sql-formats
+nav-pos: 6
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-info">Changelog-Data-Capture Format</span>
+<span class="label label-info">Format: Serialization Schema</span>
+<span class="label label-info">Format: Deserialization Schema</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+[Maxwell](https://maxwells-daemon.io/) is a CDC (Changelog Data Capture) tool 
that can stream changes in real-time from MySQL into Kafka, Kinesis and other 
streaming connectors. Maxwell provides a unified format schema for changelog 
and supports to serialize messages using JSON.
+
+Flink supports to interpret Maxwell JSON messages as INSERT/UPDATE/DELETE 
messages into Flink SQL system. This is useful in many cases to leverage this 
feature, such as
+ - synchronizing incremental data from databases to other systems
+ - auditing logs
+ - real-time materialized views on databases
+ - temporal join changing history of a database table and so on.
+
+Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL 
as Maxwell JSON messages, and emit to external systems like Kafka.
+However, currently Flink can't combine UPDATE_BEFORE and UPDATE_AFTER into a 
single UPDATE message. Therefore, Flink encodes UPDATE_BEFORE and UDPATE_AFTER 
as DELETE and INSERT Maxwell messages.
+
+Dependencies
+------------
+
+{% assign connector = site.data.sql-connectors['maxwell'] %}
+{% include sql-connector-download-table.html
+    connector=connector
+%}
+
+*Note: please refer to [Maxwell 
documentation](http://maxwells-daemon.io/quickstart/) about how to synchronize 
changelog to Kafka topics with Maxwell JSON.*
+
+
+How to use Maxwell format
+----------------
+
+Maxwell provides a unified format for changelog, here is a simple example for 
an update operation captured from a MySQL `products` table in JSON format:
+
+```json
+{
+   "database":"test",
+   "table":"e",
+   "type":"insert",
+   "ts":1477053217,
+   "xid":23396,
+   "commit":true,
+   "position":"master.000006:800911",
+   "server_id":23042,
+   "thread_id":108,
+   "primary_key": [1, "2016-10-21 05:33:37.523000"],
+   "primary_key_columns": ["id", "c"],
+   "data":{
+     "id":111,
+     "name":"scooter",
+     "description":"Big 2-wheel scooter",
+     "weight":5.15
+   },
+   "old":{
+     "weight":5.18,
+   }
+}
+```
+
+*Note: please refer to [Maxwell 
documentation](http://maxwells-daemon.io/dataformat/) about the meaning of each 
fields.*
+
+The MySQL `products` table has 4 columns (`id`, `name`, `description` and 
`weight`). The above JSON message is an update change event on the `products` 
table where the `weight` value of the row with `id = 111` is changed from 
`5.18` to `5.15`.
+Assuming this messages is synchronized to Kafka topic `products_binlog`, then 
we can use the following DDL to consume this topic and interpret the change 
events.
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE topic_products (
+  -- schema is totally the same to the MySQL "products" table
+  id BIGINT,
+  name STRING,
+  description STRING,
+  weight DECIMAL(10, 2)
+) WITH (
+ 'connector' = 'kafka',
+ 'topic' = 'products_binlog',
+ 'properties.bootstrap.servers' = 'localhost:9092',
+ 'properties.group.id' = 'testGroup',
+ 'format' = 'maxwell-json'
+)
+{% endhighlight %}
+</div>
+</div>
+
+After registering the topic as a Flink table, then you can consume the Maxwell 
messages as a changelog source.
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+-- a real-time materialized view on the MySQL "products"
+-- which calculate the latest average of weight for the same products
+SELECT name, AVG(weight) FROM topic_products GROUP BY name;
+
+-- synchronize all the data and incremental changes of MySQL "products" table 
to
+-- Elasticsearch "products" index for future searching
+INSERT INTO elasticsearch_products
+SELECT * FROM topic_products;
+{% endhighlight %}
+</div>
+</div>
+
+Format Options
+----------------
+
+<div data-lang="Maxwell Json" markdown="1">
+
+<table class="table table-bordered">
+    <thead>
+      <tr>
+        <th class="text-left" style="width: 25%">Option</th>
+        <th class="text-center" style="width: 8%">Required</th>
+        <th class="text-center" style="width: 7%">Default</th>
+        <th class="text-center" style="width: 10%">Type</th>
+        <th class="text-center" style="width: 50%">Description</th>
+      </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what format to use, here should be 
<code>'maxwell-json'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.ignore-parse-errors</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">false</td>
+      <td>Boolean</td>
+      <td>Skip fields and rows with parse errors instead of failing.
+      Fields are set to null in case of errors.</td>
+    </tr>
+    <tr>
+       <td><h5>maxwell-json.timestamp-format.standard</h5></td>
+       <td>optional</td>
+       <td style="word-wrap: break-word;"><code>'SQL'</code></td>
+       <td>String</td>
+       <td>Specify the input and output timestamp format. Currently supported 
values are <code>'SQL'</code> and <code>'ISO-8601'</code>:
+       <ul>
+         <li>Option <code>'SQL'</code> will parse input timestamp in 
"yyyy-MM-dd HH:mm:ss.s{precision}" format, e.g '2020-12-30 12:13:14.123' and 
output timestamp in the same format.</li>
+         <li>Option <code>'ISO-8601'</code>will parse input timestamp in 
"yyyy-MM-ddTHH:mm:ss.s{precision}" format, e.g '2020-12-30T12:13:14.123' and 
output timestamp in the same format.</li>
+       </ul>
+       </td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.map-null-key.mode</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;"><code>'FAIL'</code></td>
+      <td>String</td>
+      <td>Specify the handling mode when serializing null keys for map data. 
Currently supported values are <code>'FAIL'</code>, <code>'DROP'</code> and 
<code>'LITERAL'</code>:
+      <ul>
+        <li>Option <code>'FAIL'</code> will throw exception when encountering 
map with null key.</li>
+        <li>Option <code>'DROP'</code> will drop null key entries for map 
data.</li>
+        <li>Option <code>'LITERAL'</code> will replace null key with string 
literal. The string literal is defined by 
<code>maxwell-json.map-null-key.literal</code> option.</li>
+      </ul>
+      </td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.map-null-key.literal</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">'null'</td>
+      <td>String</td>
+      <td>Specify string literal to replace null key when 
<code>'maxwell-json.map-null-key.mode'</code> is LITERAL.</td>
+    </tr>
+    </tbody>
+</table>
+
+</div>
+
+Caveats
+----------------
+
+### Duplicate change events
+
+The Maxwell application allows to deliver every change event **exactly-once**. 
Flink works pretty well when consuming Maxwell produced events in this 
situation.
+If Maxwell application works in **at-least-once** delivery, it may deliver 
duplicate change events to Kafka and Flink will get the duplicate events.
+This may cause Flink query to get wrong results or unexpected exceptions. 
Thus, it is recommended to set job configuration 
[`table.exec.source.cdc-events-duplicate`]({% link dev/table/config.md 
%}#table-exec-source-cdc-events-duplicate) to `true` and define PRIMARY KEY on 
the source in this situation.
+Framework will generate an additional stateful operator, and use the primary 
key to deduplicate the change events and produce a normalized changelog stream.
+
+Data Type Mapping
+----------------
+
+Currently, the Maxwell format uses JSON for serialization and deserialization. 
Please refer to [JSON Format documentation]({% link 
dev/table/connectors/formats/json.md %}#data-type-mapping) for more details 
about the data type mapping.
diff --git a/docs/dev/table/connectors/formats/maxwell.zh.md 
b/docs/dev/table/connectors/formats/maxwell.zh.md
new file mode 100644
index 0000000..3aa93fb
--- /dev/null
+++ b/docs/dev/table/connectors/formats/maxwell.zh.md
@@ -0,0 +1,208 @@
+---
+title: "Maxwell Format"
+nav-title: Maxwell
+nav-parent_id: sql-formats
+nav-pos: 6
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-info">Changelog-Data-Capture Format</span>
+<span class="label label-info">Format: Serialization Schema</span>
+<span class="label label-info">Format: Deserialization Schema</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+[Maxwell](https://maxwells-daemon.io/) is a CDC (Changelog Data Capture) tool 
that can stream changes in real-time from MySQL into Kafka, Kinesis and other 
streaming connectors. Maxwell provides a unified format schema for changelog 
and supports to serialize messages using JSON.
+
+Flink supports to interpret Maxwell JSON messages as INSERT/UPDATE/DELETE 
messages into Flink SQL system. This is useful in many cases to leverage this 
feature, such as
+ - synchronizing incremental data from databases to other systems
+ - auditing logs
+ - real-time materialized views on databases
+ - temporal join changing history of a database table and so on.
+
+Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL 
as Maxwell JSON messages, and emit to external systems like Kafka.
+However, currently Flink can't combine UPDATE_BEFORE and UPDATE_AFTER into a 
single UPDATE message. Therefore, Flink encodes UPDATE_BEFORE and UDPATE_AFTER 
as DELETE and INSERT Maxwell messages.
+
+Dependencies
+------------
+
+{% assign connector = site.data.sql-connectors['maxwell'] %}
+{% include sql-connector-download-table.html
+    connector=connector
+%}
+
+*Note: please refer to [Maxwell 
documentation](http://maxwells-daemon.io/quickstart/) about how to synchronize 
changelog to Kafka topics with Maxwell JSON.*
+
+
+How to use Maxwell format
+----------------
+
+Maxwell provides a unified format for changelog, here is a simple example for 
an update operation captured from a MySQL `products` table in JSON format:
+
+```json
+{
+   "database":"test",
+   "table":"e",
+   "type":"insert",
+   "ts":1477053217,
+   "xid":23396,
+   "commit":true,
+   "position":"master.000006:800911",
+   "server_id":23042,
+   "thread_id":108,
+   "primary_key": [1, "2016-10-21 05:33:37.523000"],
+   "primary_key_columns": ["id", "c"],
+   "data":{
+     "id":111,
+     "name":"scooter",
+     "description":"Big 2-wheel scooter",
+     "weight":5.15
+   },
+   "old":{
+     "weight":5.18,
+   }
+}
+```
+
+*Note: please refer to [Maxwell 
documentation](http://maxwells-daemon.io/dataformat/) about the meaning of each 
fields.*
+
+The MySQL `products` table has 4 columns (`id`, `name`, `description` and 
`weight`). The above JSON message is an update change event on the `products` 
table where the `weight` value of the row with `id = 111` is changed from 
`5.18` to `5.15`.
+Assuming this messages is synchronized to Kafka topic `products_binlog`, then 
we can use the following DDL to consume this topic and interpret the change 
events.
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE topic_products (
+  -- schema is totally the same to the MySQL "products" table
+  id BIGINT,
+  name STRING,
+  description STRING,
+  weight DECIMAL(10, 2)
+) WITH (
+ 'connector' = 'kafka',
+ 'topic' = 'products_binlog',
+ 'properties.bootstrap.servers' = 'localhost:9092',
+ 'properties.group.id' = 'testGroup',
+ 'format' = 'maxwell-json'
+)
+{% endhighlight %}
+</div>
+</div>
+
+After registering the topic as a Flink table, then you can consume the Maxwell 
messages as a changelog source.
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+-- a real-time materialized view on the MySQL "products"
+-- which calculate the latest average of weight for the same products
+SELECT name, AVG(weight) FROM topic_products GROUP BY name;
+
+-- synchronize all the data and incremental changes of MySQL "products" table 
to
+-- Elasticsearch "products" index for future searching
+INSERT INTO elasticsearch_products
+SELECT * FROM topic_products;
+{% endhighlight %}
+</div>
+</div>
+
+Format Options
+----------------
+
+<div data-lang="Maxwell Json" markdown="1">
+
+<table class="table table-bordered">
+    <thead>
+      <tr>
+        <th class="text-left" style="width: 25%">Option</th>
+        <th class="text-center" style="width: 8%">Required</th>
+        <th class="text-center" style="width: 7%">Default</th>
+        <th class="text-center" style="width: 10%">Type</th>
+        <th class="text-center" style="width: 50%">Description</th>
+      </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what format to use, here should be 
<code>'maxwell-json'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.ignore-parse-errors</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">false</td>
+      <td>Boolean</td>
+      <td>Skip fields and rows with parse errors instead of failing.
+      Fields are set to null in case of errors.</td>
+    </tr>
+    <tr>
+       <td><h5>maxwell-json.timestamp-format.standard</h5></td>
+       <td>optional</td>
+       <td style="word-wrap: break-word;"><code>'SQL'</code></td>
+       <td>String</td>
+       <td>Specify the input and output timestamp format. Currently supported 
values are <code>'SQL'</code> and <code>'ISO-8601'</code>:
+       <ul>
+         <li>Option <code>'SQL'</code> will parse input timestamp in 
"yyyy-MM-dd HH:mm:ss.s{precision}" format, e.g '2020-12-30 12:13:14.123' and 
output timestamp in the same format.</li>
+         <li>Option <code>'ISO-8601'</code>will parse input timestamp in 
"yyyy-MM-ddTHH:mm:ss.s{precision}" format, e.g '2020-12-30T12:13:14.123' and 
output timestamp in the same format.</li>
+       </ul>
+       </td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.map-null-key.mode</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;"><code>'FAIL'</code></td>
+      <td>String</td>
+      <td>Specify the handling mode when serializing null keys for map data. 
Currently supported values are <code>'FAIL'</code>, <code>'DROP'</code> and 
<code>'LITERAL'</code>:
+      <ul>
+        <li>Option <code>'FAIL'</code> will throw exception when encountering 
map with null key.</li>
+        <li>Option <code>'DROP'</code> will drop null key entries for map 
data.</li>
+        <li>Option <code>'LITERAL'</code> will replace null key with string 
literal. The string literal is defined by 
<code>maxwell-json.map-null-key.literal</code> option.</li>
+      </ul>
+      </td>
+    </tr>
+    <tr>
+      <td><h5>maxwell-json.map-null-key.literal</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">'null'</td>
+      <td>String</td>
+      <td>Specify string literal to replace null key when 
<code>'maxwell-json.map-null-key.mode'</code> is LITERAL.</td>
+    </tr>
+    </tbody>
+</table>
+
+</div>
+
+Caveats
+----------------
+
+### Duplicate change events
+
+The Maxwell application allows to deliver every change event **exactly-once**. 
Flink works pretty well when consuming Maxwell produced events in this 
situation.
+If Maxwell application works in **at-least-once** delivery, it may deliver 
duplicate change events to Kafka and Flink will get the duplicate events.
+This may cause Flink query to get wrong results or unexpected exceptions. 
Thus, it is recommended to set job configuration 
[`table.exec.source.cdc-events-duplicate`]({% link dev/table/config.zh.md 
%}#table-exec-source-cdc-events-duplicate) to `true` and define PRIMARY KEY on 
the source in this situation.
+Framework will generate an additional stateful operator, and use the primary 
key to deduplicate the change events and produce a normalized changelog stream.
+
+Data Type Mapping
+----------------
+
+Currently, the Maxwell format uses JSON for serialization and deserialization. 
Please refer to [JSON Format documentation]({% link 
dev/table/connectors/formats/json.zh.md %}#data-type-mapping) for more details 
about the data type mapping.
diff --git a/docs/dev/table/connectors/formats/orc.md 
b/docs/dev/table/connectors/formats/orc.md
index 3bb695e..6be8010 100644
--- a/docs/dev/table/connectors/formats/orc.md
+++ b/docs/dev/table/connectors/formats/orc.md
@@ -2,7 +2,7 @@
 title: "Orc Format"
 nav-title: Orc
 nav-parent_id: sql-formats
-nav-pos: 6
+nav-pos: 8
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -34,8 +34,8 @@ The [Apache Orc](https://orc.apache.org/) format allows to 
read and write Orc da
 Dependencies
 ------------
 
-{% assign connector = site.data.sql-connectors['orc'] %} 
-{% include sql-connector-download-table.html 
+{% assign connector = site.data.sql-connectors['orc'] %}
+{% include sql-connector-download-table.html
     connector=connector
 %}
 
diff --git a/docs/dev/table/connectors/formats/orc.zh.md 
b/docs/dev/table/connectors/formats/orc.zh.md
index 6e153eb..1bc8203 100644
--- a/docs/dev/table/connectors/formats/orc.zh.md
+++ b/docs/dev/table/connectors/formats/orc.zh.md
@@ -2,7 +2,7 @@
 title: "Orc Format"
 nav-title: Orc
 nav-parent_id: sql-formats
-nav-pos: 6
+nav-pos: 8
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -34,8 +34,8 @@ under the License.
 依赖
 ------------
 
-{% assign connector = site.data.sql-connectors['orc'] %} 
-{% include sql-connector-download-table.html 
+{% assign connector = site.data.sql-connectors['orc'] %}
+{% include sql-connector-download-table.html
     connector=connector
 %}
 
diff --git a/docs/dev/table/connectors/formats/parquet.md 
b/docs/dev/table/connectors/formats/parquet.md
index 5cc57ca..20aab81 100644
--- a/docs/dev/table/connectors/formats/parquet.md
+++ b/docs/dev/table/connectors/formats/parquet.md
@@ -2,7 +2,7 @@
 title: "Parquet Format"
 nav-title: Parquet
 nav-parent_id: sql-formats
-nav-pos: 5
+nav-pos: 7
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -34,8 +34,8 @@ The [Apache Parquet](https://parquet.apache.org/) format 
allows to read and writ
 Dependencies
 ------------
 
-{% assign connector = site.data.sql-connectors['parquet'] %} 
-{% include sql-connector-download-table.html 
+{% assign connector = site.data.sql-connectors['parquet'] %}
+{% include sql-connector-download-table.html
     connector=connector
 %}
 
diff --git a/docs/dev/table/connectors/formats/parquet.zh.md 
b/docs/dev/table/connectors/formats/parquet.zh.md
index 865ad2b..18abae4 100644
--- a/docs/dev/table/connectors/formats/parquet.zh.md
+++ b/docs/dev/table/connectors/formats/parquet.zh.md
@@ -2,7 +2,7 @@
 title: "Parquet 格式"
 nav-title: Parquet
 nav-parent_id: sql-formats
-nav-pos: 5
+nav-pos: 7
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -34,8 +34,8 @@ under the License.
 依赖
 ------------
 
-{% assign connector = site.data.sql-connectors['parquet'] %} 
-{% include sql-connector-download-table.html 
+{% assign connector = site.data.sql-connectors['parquet'] %}
+{% include sql-connector-download-table.html
     connector=connector
 %}
 
diff --git a/docs/dev/table/connectors/formats/raw.md 
b/docs/dev/table/connectors/formats/raw.md
index 4ecad71..a2a7bab 100644
--- a/docs/dev/table/connectors/formats/raw.md
+++ b/docs/dev/table/connectors/formats/raw.md
@@ -2,7 +2,7 @@
 title: "Raw Format"
 nav-title: Raw
 nav-parent_id: sql-formats
-nav-pos: 7
+nav-pos: 9
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -36,8 +36,8 @@ Note: this format encodes `null` values as `null` of `byte[]` 
type. This may hav
 Dependencies
 ------------
 
-{% assign connector = site.data.sql-connectors['raw'] %} 
-{% include sql-connector-download-table.html 
+{% assign connector = site.data.sql-connectors['raw'] %}
+{% include sql-connector-download-table.html
     connector=connector
 %}
 
diff --git a/docs/dev/table/connectors/formats/raw.zh.md 
b/docs/dev/table/connectors/formats/raw.zh.md
index 067fabb..9892996 100644
--- a/docs/dev/table/connectors/formats/raw.zh.md
+++ b/docs/dev/table/connectors/formats/raw.zh.md
@@ -2,7 +2,7 @@
 title: "Raw Format"
 nav-title: Raw
 nav-parent_id: sql-formats
-nav-pos: 7
+nav-pos: 9
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -37,7 +37,7 @@ Raw format 允许读写原始(基于字节)值作为单个列。
 ------------
 
 {% assign connector = site.data.sql-connectors['raw'] %}
-{% include sql-connector-download-table.html 
+{% include sql-connector-download-table.html
     connector=connector
 %}
 

Reply via email to