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

vinoyang pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/asf-site by this push:
     new db9cb1c  [HUDI-1693] Add document about HUDI Flink integration (#2681)
db9cb1c is described below

commit db9cb1c568d4180125dd654e815afa64fa2aeb2a
Author: Danny Chan <[email protected]>
AuthorDate: Wed Mar 17 14:19:50 2021 +0800

    [HUDI-1693] Add document about HUDI Flink integration (#2681)
---
 docs/_config.yml                                   |   4 +-
 docs/_data/navigation.yml                          |  12 +-
 docs/_docs/0_3_migration_guide.cn.md               |   2 +-
 docs/_docs/0_3_migration_guide.md                  |   2 +-
 ...ide.cn.md => 1_1_spark_quick_start_guide.cn.md} |   2 +-
 ...art_guide.md => 1_1_spark_quick_start_guide.md} |   2 +-
 docs/_docs/1_6_flink_quick_start_guide.md          | 169 +++++++++++++++++++++
 docs/_docs/2_2_writing_data.md                     |  34 +++++
 docs/_docs/2_3_querying_data.md                    |  45 +++++-
 docs/_docs/2_4_configurations.md                   |  53 +++++++
 docs/_layouts/home.html                            |   2 +-
 docs/_pages/contributing.cn.md                     |   2 +-
 docs/_pages/contributing.md                        |   2 +-
 docs/_pages/releases.md                            |   2 +-
 14 files changed, 317 insertions(+), 16 deletions(-)

diff --git a/docs/_config.yml b/docs/_config.yml
index 6d8a6fb..270d8a2 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -12,8 +12,8 @@ version : &version "0.5.1-SNAPSHOT"
 
 previous_docs:
   - version: Latest
-    en: /docs/quick-start-guide.html
-    cn: /cn/docs/quick-start-guide.html
+    en: /docs/spark_quick-start-guide.html
+    cn: /cn/docs/spark_quick-start-guide.html
   - version: 0.7.0
     en: /docs/0.7.0-quick-start-guide.html
     cn: /cn/docs/0.7.0-quick-start-guide.html
diff --git a/docs/_data/navigation.yml b/docs/_data/navigation.yml
index 4054bc8..5803a43 100644
--- a/docs/_data/navigation.yml
+++ b/docs/_data/navigation.yml
@@ -2,7 +2,7 @@
 # main links
 main:
   - title: "Documentation"
-    url: /docs/quick-start-guide.html
+    url: /docs/spark_quick-start-guide.html
   - title: "Community"
     url: /community.html
   - title: "Blog"
@@ -20,8 +20,10 @@ docs:
     children:
       - title: "Overview"
         url: /docs/overview.html
-      - title: "Quick Start"
-        url: /docs/quick-start-guide.html
+      - title: "Quick Start(Spark)"
+        url: /docs/spark_quick-start-guide.html
+      - title: "Quick Start(Flink)"
+        url: /docs/flink-quick-start-guide.html
       - title: "Use Cases"
         url: /docs/use_cases.html
       - title: "Writing Data"
@@ -50,7 +52,7 @@ docs:
 
 cn_main:
   - title: "文档"
-    url: /cn/docs/quick-start-guide.html
+    url: /cn/docs/spark_quick-start-guide.html
   - title: "社区"
     url: /cn/community.html
   - title: "动态"
@@ -65,7 +67,7 @@ cn_docs:
   - title: 入门指南
     children:
       - title: "快速开始"
-        url: /cn/docs/quick-start-guide.html
+        url: /cn/docs/spark_quick-start-guide.html
       - title: "使用案例"
         url: /cn/docs/use_cases.html
       - title: "演讲 & hudi 用户"
diff --git a/docs/_docs/0_3_migration_guide.cn.md 
b/docs/_docs/0_3_migration_guide.cn.md
index f90229a..95cab06 100644
--- a/docs/_docs/0_3_migration_guide.cn.md
+++ b/docs/_docs/0_3_migration_guide.cn.md
@@ -52,7 +52,7 @@ for partition in [list of partitions in source dataset] {
 
 **Option 3**
 Write your own custom logic of how to load an existing dataset into a Hudi 
managed one. Please read about the RDD API
- [here](/cn/docs/quick-start-guide.html). Using the HDFSParquetImporter Tool. 
Once hudi has been built via `mvn clean install -DskipTests`, the shell can be
+ [here](/cn/docs/spark_quick-start-guide.html). Using the HDFSParquetImporter 
Tool. Once hudi has been built via `mvn clean install -DskipTests`, the shell 
can be
 fired by via `cd hudi-cli && ./hudi-cli.sh`.
 
 ```java
diff --git a/docs/_docs/0_3_migration_guide.md 
b/docs/_docs/0_3_migration_guide.md
index 25c70f6..012abc0 100644
--- a/docs/_docs/0_3_migration_guide.md
+++ b/docs/_docs/0_3_migration_guide.md
@@ -51,7 +51,7 @@ for partition in [list of partitions in source table] {
 
 **Option 3**
 Write your own custom logic of how to load an existing table into a Hudi 
managed one. Please read about the RDD API
- [here](/docs/quick-start-guide.html). Using the HDFSParquetImporter Tool. 
Once hudi has been built via `mvn clean install -DskipTests`, the shell can be
+ [here](/docs/spark_quick-start-guide.html). Using the HDFSParquetImporter 
Tool. Once hudi has been built via `mvn clean install -DskipTests`, the shell 
can be
 fired by via `cd hudi-cli && ./hudi-cli.sh`.
 
 ```java
diff --git a/docs/_docs/1_1_quick_start_guide.cn.md 
b/docs/_docs/1_1_spark_quick_start_guide.cn.md
similarity index 99%
rename from docs/_docs/1_1_quick_start_guide.cn.md
rename to docs/_docs/1_1_spark_quick_start_guide.cn.md
index 9c12c20..dbdb30a 100644
--- a/docs/_docs/1_1_quick_start_guide.cn.md
+++ b/docs/_docs/1_1_spark_quick_start_guide.cn.md
@@ -1,6 +1,6 @@
 ---
 title: "Quick-Start Guide"
-permalink: /cn/docs/quick-start-guide.html
+permalink: /cn/docs/spark_quick-start-guide.html
 toc: true
 last_modified_at: 2019-12-30T15:59:57-04:00
 language: cn
diff --git a/docs/_docs/1_1_quick_start_guide.md 
b/docs/_docs/1_1_spark_quick_start_guide.md
similarity index 99%
rename from docs/_docs/1_1_quick_start_guide.md
rename to docs/_docs/1_1_spark_quick_start_guide.md
index cccf748..fced71d 100644
--- a/docs/_docs/1_1_quick_start_guide.md
+++ b/docs/_docs/1_1_spark_quick_start_guide.md
@@ -1,6 +1,6 @@
 ---
 title: "Quick-Start Guide"
-permalink: /docs/quick-start-guide.html
+permalink: /docs/spark_quick-start-guide.html
 toc: true
 last_modified_at: 2019-12-30T15:59:57-04:00
 ---
diff --git a/docs/_docs/1_6_flink_quick_start_guide.md 
b/docs/_docs/1_6_flink_quick_start_guide.md
new file mode 100644
index 0000000..c16028a
--- /dev/null
+++ b/docs/_docs/1_6_flink_quick_start_guide.md
@@ -0,0 +1,169 @@
+---
+title: "Quick-Start Guide"
+permalink: /docs/flink-quick-start-guide.html
+toc: true
+last_modified_at: 2020-03-16T11:40:57+08:00
+---
+
+This guide provides a quick peek at Hudi's capabilities using flink SQL 
client. Using flink SQL, we will walk through 
+code snippets that allows you to insert and update a Hudi table of default 
table type: 
+[Copy on Write](/docs/concepts.html#copy-on-write-table) and [Merge On 
Read](/docs/concepts.html#merge-on-read-table). 
+After each write operation we will also show how to read the data snapshot 
(incrementally read is already on the roadmap).
+
+## Setup
+
+We use the [Flink Sql 
Client](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sqlClient.html)
 because it's a good
+quick start tool for SQL users.
+
+### Step.1 download flink jar
+Hudi works with Flink-1.11.x version. You can follow instructions 
[here](https://flink.apache.org/downloads.html) for setting up flink.
+The hudi-flink-bundle jar is archived with scala 2.11, so it’s recommended to 
use flink 1.11 bundled with scala 2.11.
+
+### Step.2 start flink cluster
+Start a standalone flink cluster within hadoop environment.
+Before you start up the cluster, we suggest to config the cluster as follows:
+
+- in `$FLINK_HOME/conf/flink-conf.yaml`, add config option 
`taskmanager.numberOfTaskSlots: 4`
+- in `$FLINK_HOME/conf/workers`, add item `localhost` as 4 lines so that there 
are 4 workers on the local cluster
+
+Now starts the cluster:
+
+```bash
+# HADOOP_HOME is your hadoop root directory after unpack the binary package.
+export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`
+
+# Start the flink standalone cluster
+./bin/start-cluster.sh
+```
+### Step.3 start flink SQL client
+
+Hudi has a prepared bundle jar for flink, which should be loaded in the flink 
SQL Client when it starts up.
+You can build the jar manually under path 
`hudi-source-dir/packaging/hudi-flink-bundle`, or download it from the
+[Apache Official 
Repository](https://repo.maven.apache.org/maven2/org/apache/hudi/hudi-flink-bundle_2.11/).
+
+Now starts the SQL CLI:
+
+```bash
+# HADOOP_HOME is your hadoop root directory after unpack the binary package.
+export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`
+
+./bin/sql-client.sh embedded -j .../hudi-flink-bundle_2.1?-*.*.*.jar shell
+```
+
+<div class="notice--info">
+  <h4>Please note the following: </h4>
+<ul>
+  <li>We suggest hadoop 2.9.x+ version because some of the object storage has 
filesystem implementation only after that</li>
+  <li>The flink-parquet and flink-avro formats are already packaged into the 
hudi-flink-bundle jar</li>
+</ul>
+</div>
+
+Setup table name, base path and operate using SQL for this guide.
+The SQL CLI only executes the SQL line by line.
+
+## Insert data
+
+Creates a flink hudi table first and insert data into the Hudi table using SQL 
`VALUES` as below.
+
+```sql
+-- sets up the result mode to tableau to show the results directly in the CLI
+set execution.result-mode=tableau;
+
+CREATE TABLE t1(
+  uuid VARCHAR(20),
+  name VARCHAR(10),
+  age INT,
+  ts TIMESTAMP(3),
+  `partition` VARCHAR(20)
+)
+PARTITIONED BY (`partition`)
+WITH (
+  'connector' = 'hudi',
+  'path' = 'schema://base-path',
+  'table.type' = 'MERGE_ON_READ' -- this creates a MERGE_ON_READ table, by 
default is COPY_ON_WRITE
+);
+
+-- insert data using values
+INSERT INTO t1 VALUES
+  ('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),
+  ('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),
+  ('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),
+  ('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),
+  ('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),
+  ('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),
+  ('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),
+  ('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4');
+```
+
+## Query data
+
+```sql
+-- query from the hudi table
+select * from t1;
+```
+
+This query provides snapshot querying of the ingested data. 
+Refer to [Table types and queries](/docs/concepts#table-types--queries) for 
more info on all table types and query types supported.
+{: .notice--info}
+
+## Update data
+
+This is similar to inserting new data.
+
+```sql
+-- this would update the record with key 'id1'
+insert into t1 values
+  ('id1','Danny',27,TIMESTAMP '1970-01-01 00:00:01','par1');
+```
+
+Notice that the save mode is now `Append`. In general, always use append mode 
unless you are trying to create the table for the first time.
+[Querying](#query-data) the data again will now show updated records. Each 
write operation generates a new [commit](/docs/concepts.html) 
+denoted by the timestamp. Look for changes in `_hoodie_commit_time`, `age` 
fields for the same `_hoodie_record_key`s in previous commit. 
+{: .notice--info}
+
+## Streaming query
+
+Hudi flink also provides capability to obtain a stream of records that changed 
since given commit timestamp. 
+This can be achieved using Hudi's streaming querying and providing a start 
time from which changes need to be streamed. 
+We do not need to specify endTime, if we want all changes after the given 
commit (as is the common case). 
+
+```sql
+CREATE TABLE t1(
+  uuid VARCHAR(20),
+  name VARCHAR(10),
+  age INT,
+  ts TIMESTAMP(3),
+  `partition` VARCHAR(20)
+)
+PARTITIONED BY (`partition`)
+WITH (
+  'connector' = 'hudi',
+  'path' = 'oss://vvr-daily/hudi/t1',
+  'table.type' = 'MERGE_ON_READ',
+  'read.streaming.enabled' = 'true',  -- this option enable the streaming read
+  'read.streaming.start-commit' = '20210316134557' -- specifies the start 
commit instant time
+  'read.streaming.check-interval' = '4' -- specifies the check interval for 
finding new source commits, default 60s.
+);
+
+-- Then query the table in stream mode
+select * from t1;
+``` 
+
+This will give all changes that happened after the 
`read.streaming.start-commit` commit. The unique thing about this
+feature is that it now lets you author streaming pipelines on streaming or 
batch data source.
+{: .notice--info}
+
+## Delete data {#deletes}
+
+When consuming data in streaming query, hudi flink source can also accepts the 
change logs from the underneath data source,
+it can then applies the UPDATE and DELETE by per-row level. You can then sync 
a NEAR-REAL-TIME snapshot on hudi for all kinds
+of RDBMS.
+
+## Where to go from here?
+
+We used flink here to show case the capabilities of Hudi. However, Hudi can 
support multiple table types/query types and 
+Hudi tables can be queried from query engines like Hive, Spark, Flink, Presto 
and much more. We have put together a 
+[demo video](https://www.youtube.com/watch?v=VhNgUsxdrD0) that show cases all 
of this on a docker based setup with all 
+dependent systems running locally. We recommend you replicate the same setup 
and run the demo yourself, by following 
+steps [here](/docs/docker_demo.html) to get a taste for it. Also, if you are 
looking for ways to migrate your existing data 
+to Hudi, refer to [migration guide](/docs/migration_guide.html). 
diff --git a/docs/_docs/2_2_writing_data.md b/docs/_docs/2_2_writing_data.md
index 6b51878..affb731 100644
--- a/docs/_docs/2_2_writing_data.md
+++ b/docs/_docs/2_2_writing_data.md
@@ -262,6 +262,40 @@ inputDF.write()
        .save(basePath);
 ```
 
+## Flink SQL Writer
+The hudi-flink module defines the Flink SQL connector for both hudi source and 
sink.
+There are a number of options available for the sink table:
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| path | Y | N/A | Base path for the target hoodie table. The path would be 
created if it does not exist, otherwise a hudi table expects to be initialized 
successfully |
+| table.type  | N | COPY_ON_WRITE | Type of table to write. COPY_ON_WRITE (or) 
MERGE_ON_READ |
+| write.operation | N | upsert | The write operation, that this write should 
do (insert or upsert is supported) |
+| write.precombine.field | N | ts | Field used in preCombining before actual 
write. When two records have the same key value, we will pick the one with the 
largest value for the precombine field, determined by Object.compareTo(..) |
+| write.payload.class | N | OverwriteWithLatestAvroPayload.class | Payload 
class used. Override this, if you like to roll your own merge logic, when 
upserting/inserting. This will render any value set for the option in-effective 
|
+| write.insert.drop.duplicates | N | false | Flag to indicate whether to drop 
duplicates upon insert. By default insert will accept duplicates, to gain extra 
performance |
+| write.ignore.failed | N | true | Flag to indicate whether to ignore any non 
exception error (e.g. writestatus error). within a checkpoint batch. By default 
true (in favor of streaming progressing over data integrity) |
+| hoodie.datasource.write.recordkey.field | N | uuid | Record key field. Value 
to be used as the `recordKey` component of `HoodieKey`. Actual value will be 
obtained by invoking .toString() on the field value. Nested fields can be 
specified using the dot notation eg: `a.b.c` |
+| hoodie.datasource.write.keygenerator.class | N | 
SimpleAvroKeyGenerator.class | Key generator class, that implements will 
extract the key out of incoming record |
+| write.tasks | N | 4 | Parallelism of tasks that do actual write, default is 
4 |
+| write.batch.size.MB | N | 128 | Batch buffer size in MB to flush data into 
the underneath filesystem |
+
+If the table type is MERGE_ON_READ, you can also specify the asynchronous 
compaction strategy through options:
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| compaction.async.enabled | N | true | Async Compaction, enabled by default 
for MOR |
+| compaction.trigger.strategy | N | num_commits | Strategy to trigger 
compaction, options are 'num_commits': trigger compaction when reach N delta 
commits; 'time_elapsed': trigger compaction when time elapsed > N seconds since 
last compaction; 'num_and_time': trigger compaction when both NUM_COMMITS and 
TIME_ELAPSED are satisfied; 'num_or_time': trigger compaction when NUM_COMMITS 
or TIME_ELAPSED is satisfied. Default is 'num_commits' |
+| compaction.delta_commits | N | 5 | Max delta commits needed to trigger 
compaction, default 5 commits |
+| compaction.delta_seconds | N | 3600 | Max delta seconds time needed to 
trigger compaction, default 1 hour |
+
+You can write the data using the SQL `INSERT INTO` statements:
+```sql
+INSERT INTO hudi_table select ... from ...; 
+```
+
+**Note**: INSERT OVERWRITE is not supported yet but already on the roadmap.
+
 ## Key Generation
 
 Hudi maintains hoodie keys (record key + partition path) for uniquely 
identifying a particular record. Key generator class will extract these out of 
incoming record. Both the tools above have configs to specify the 
diff --git a/docs/_docs/2_3_querying_data.md b/docs/_docs/2_3_querying_data.md
index 0af3418..ed6752d 100644
--- a/docs/_docs/2_3_querying_data.md
+++ b/docs/_docs/2_3_querying_data.md
@@ -40,6 +40,7 @@ Following tables show whether a given query is supported on 
specific query engin
 |**Hive**|Y|Y|
 |**Spark SQL**|Y|Y|
 |**Spark Datasource**|Y|Y|
+|**Flink SQL**|Y|N|
 |**PrestoDB**|Y|N|
 |**Impala**|Y|N|
 
@@ -53,6 +54,7 @@ Note that `Read Optimized` queries are not applicable for 
COPY_ON_WRITE tables.
 |**Hive**|Y|Y|Y|
 |**Spark SQL**|Y|Y|Y|
 |**Spark Datasource**|Y|N|Y|
+|**Flink SQL**|Y|Y|Y|
 |**PrestoDB**|Y|N|Y|
 |**Impala**|N|N|Y|
 
@@ -165,7 +167,7 @@ 
hudiIncQueryDF.createOrReplaceTempView("hudi_trips_incremental")
 spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from  
hudi_trips_incremental where fare > 20.0").show()
 ```
 
-For examples, refer to [Setup spark-shell in 
quickstart](/docs/quick-start-guide.html#setup-spark-shell). 
+For examples, refer to [Setup spark-shell in 
quickstart](/docs/spark_quick-start-guide.html#setup-spark-shell). 
 Please refer to [configurations](/docs/configurations.html#spark-datasource) 
section, to view all datasource options.
 
 Additionally, `HoodieReadClient` offers the following functionality using 
Hudi's implicit indexing.
@@ -176,6 +178,47 @@ Additionally, `HoodieReadClient` offers the following 
functionality using Hudi's
 | filterExists() | Filter out already existing records from the provided 
`RDD[HoodieRecord]`. Useful for de-duplication |
 | checkExists(keys) | Check if the provided keys exist in a Hudi table |
 
+## Flink SQL
+Once the flink Hudi tables have been registered to the Flink catalog, it can 
be queried using the Flink SQL. It supports all query types across both Hudi 
table types,
+relying on the custom Hudi input formats again like Hive. Typically notebook 
users and Flink SQL CLI users leverage flink sql for querying Hudi tables. 
Please add hudi-flink-bundle as described above via --jars.
+
+By default, Flink SQL will try to use its own parquet reader instead of Hive 
SerDe when reading from Hive metastore parquet tables.
+
+```bash
+# HADOOP_HOME is your hadoop root directory after unpack the binary package.
+export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`
+
+./bin/sql-client.sh embedded -j .../hudi-flink-bundle_2.1?-*.*.*.jar shell
+```
+
+```sql
+-- this defines a COPY_ON_WRITE table named 't1'
+CREATE TABLE t1(
+  uuid VARCHAR(20),
+  name VARCHAR(10),
+  age INT,
+  ts TIMESTAMP(3),
+  `partition` VARCHAR(20)
+)
+PARTITIONED BY (`partition`)
+WITH (
+  'connector' = 'hudi',
+  'path' = 'schema://base-path'
+);
+
+-- query the data
+select * from t1 where `partition` = 'par1';
+```
+
+Flink's built-in support parquet is used for both COPY_ON_WRITE and 
MERGE_ON_READ tables,
+additionally partition prune is applied by Flink engine internally if a 
partition path is specified
+in the filter. Filters push down is not supported yet (already on the roadmap).
+
+For MERGE_ON_READ table, in order to query hudi table as a streaming, you need 
to add option `'read.streaming.enabled' = 'true'`,
+when querying the table, a Flink streaming pipeline starts and never ends 
until the user cancel the job manually.
+You can specify the start commit with option `read.streaming.start-commit` and 
source monitoring interval with option
+`read.streaming.check-interval`.
+
 ## PrestoDB
 
 PrestoDB is a popular query engine, providing interactive query performance. 
PrestoDB currently supports snapshot querying on COPY_ON_WRITE tables. 
diff --git a/docs/_docs/2_4_configurations.md b/docs/_docs/2_4_configurations.md
index ff25998..ec35e64 100644
--- a/docs/_docs/2_4_configurations.md
+++ b/docs/_docs/2_4_configurations.md
@@ -11,6 +11,7 @@ This page covers the different ways of configuring your job 
to write/read Hudi t
 At a high level, you can control behaviour at few levels. 
 
 - **[Spark Datasource Configs](#spark-datasource)** : These configs control 
the Hudi Spark Datasource, providing ability to define keys/partitioning, pick 
out the write operation, specify how to merge records or choosing query type to 
read.
+- **[Flink SQL Configs](#flink-options)** : These configs control the Hudi 
Flink SQL source/sink connectors, providing ability to define record keys, pick 
out the write operation, specify how to merge records, enable/disable 
asynchronous compaction or choosing query type to read.
 - **[WriteClient Configs](#writeclient-configs)** : Internally, the Hudi 
datasource uses a RDD based `HoodieWriteClient` api to actually perform writes 
to storage. These configs provide deep control over lower level aspects like 
    file sizing, compression, parallelism, compaction, write schema, cleaning 
etc. Although Hudi provides sane defaults, from time-time these configs may 
need to be tweaked to optimize for specific workloads.
 - **[RecordPayload Config](#PAYLOAD_CLASS_OPT_KEY)** : This is the lowest 
level of customization offered by Hudi. Record payloads define how to produce 
new values to upsert based on incoming new record and 
@@ -171,6 +172,58 @@ Property: `hoodie.datasource.read.end.instanttime`, 
Default: latest instant (i.e
 Property: `hoodie.datasource.read.schema.use.end.instanttime`, Default: false 
<br/>
 <span style="color:grey"> Uses end instant schema when incrementally fetched 
data to. Default: users latest instant schema. </span>
 
+## Flink SQL Config Options {#flink-options}
+
+Flink jobs using the SQL can be configured through the options in `WITH` 
clause.
+The actual datasource level configs are listed below.
+
+### Write Options
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| `path` | Y | N/A | <span style="color:grey"> Base path for the target hoodie 
table. The path would be created if it does not exist, otherwise a hudi table 
expects to be initialized successfully </span> |
+| `table.type`  | N | COPY_ON_WRITE | <span style="color:grey"> Type of table 
to write. COPY_ON_WRITE (or) MERGE_ON_READ </span> |
+| `write.operation` | N | upsert | <span style="color:grey"> The write 
operation, that this write should do (insert or upsert is supported) </span> |
+| `write.precombine.field` | N | ts | <span style="color:grey"> Field used in 
preCombining before actual write. When two records have the same key value, we 
will pick the one with the largest value for the precombine field, determined 
by Object.compareTo(..) </span> |
+| `write.payload.class` | N | OverwriteWithLatestAvroPayload.class | <span 
style="color:grey"> Payload class used. Override this, if you like to roll your 
own merge logic, when upserting/inserting. This will render any value set for 
the option in-effective </span> |
+| `write.insert.drop.duplicates` | N | false | <span style="color:grey"> Flag 
to indicate whether to drop duplicates upon insert. By default insert will 
accept duplicates, to gain extra performance </span> |
+| `write.ignore.failed` | N | true | <span style="color:grey"> Flag to 
indicate whether to ignore any non exception error (e.g. writestatus error). 
within a checkpoint batch. By default true (in favor of streaming progressing 
over data integrity) </span> |
+| `hoodie.datasource.write.recordkey.field` | N | uuid | <span 
style="color:grey"> Record key field. Value to be used as the `recordKey` 
component of `HoodieKey`. Actual value will be obtained by invoking .toString() 
on the field value. Nested fields can be specified using the dot notation eg: 
`a.b.c` </span> |
+| `hoodie.datasource.write.keygenerator.class` | N | 
SimpleAvroKeyGenerator.class | <span style="color:grey"> Key generator class, 
that implements will extract the key out of incoming record </span> |
+| `write.tasks` | N | 4 | <span style="color:grey"> Parallelism of tasks that 
do actual write, default is 4 </span> |
+| `write.batch.size.MB` | N | 128 | <span style="color:grey"> Batch buffer 
size in MB to flush data into the underneath filesystem </span> |
+
+If the table type is MERGE_ON_READ, you can also specify the asynchronous 
compaction strategy through options:
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| `compaction.async.enabled` | N | true | <span style="color:grey"> Async 
Compaction, enabled by default for MOR </span> |
+| `compaction.trigger.strategy` | N | num_commits | <span style="color:grey"> 
Strategy to trigger compaction, options are 'num_commits': trigger compaction 
when reach N delta commits; 'time_elapsed': trigger compaction when time 
elapsed > N seconds since last compaction; 'num_and_time': trigger compaction 
when both NUM_COMMITS and TIME_ELAPSED are satisfied; 'num_or_time': trigger 
compaction when NUM_COMMITS or TIME_ELAPSED is satisfied. Default is 
'num_commits' </span> |
+| `compaction.delta_commits` | N | 5 | <span style="color:grey"> Max delta 
commits needed to trigger compaction, default 5 commits </span> |
+| `compaction.delta_seconds` | N | 3600 | <span style="color:grey"> Max delta 
seconds time needed to trigger compaction, default 1 hour </span> |
+
+### Read Options
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| `path` | Y | N/A | <span style="color:grey"> Base path for the target hoodie 
table. The path would be created if it does not exist, otherwise a hudi table 
expects to be initialized successfully </span> |
+| `table.type`  | N | COPY_ON_WRITE | <span style="color:grey"> Type of table 
to write. COPY_ON_WRITE (or) MERGE_ON_READ </span> |
+| `read.tasks` | N | 4 | <span style="color:grey"> Parallelism of tasks that 
do actual read, default is 4 </span> |
+| `read.avro-schema.path` | N | N/A | <span style="color:grey"> Avro schema 
file path, the parsed schema is used for deserialization, if not specified, the 
avro schema is inferred from the table DDL </span> |
+| `read.avro-schema` | N | N/A | <span style="color:grey"> Avro schema string, 
the parsed schema is used for deserialization, if not specified, the avro 
schema is inferred from the table DDL </span> |
+| `hoodie.datasource.query.type` | N | snapshot | <span style="color:grey"> 
Decides how data files need to be read, in 1) Snapshot mode (obtain latest 
view, based on row & columnar data); 2) incremental mode (new data since an 
instantTime), not supported yet; 3) Read Optimized mode (obtain latest view, 
based on columnar data). Default: snapshot </span> |
+| `hoodie.datasource.merge.type` | N | payload_combine | <span 
style="color:grey"> For Snapshot query on merge on read table. Use this key to 
define how the payloads are merged, in 1) skip_merge: read the base file 
records plus the log file records; 2) payload_combine: read the base file 
records first, for each record in base file, checks whether the key is in the 
log file records(combines the two records with same key for base and log file 
records), then read the left log file records < [...]
+| `hoodie.datasource.hive_style_partition` | N | false | <span 
style="color:grey"> Whether the partition path is with Hive style, e.g. 
'{partition key}={partition value}', default false </span> |
+| `read.utc-timezone` | N | true | <span style="color:grey"> Use UTC timezone 
or local timezone to the conversion between epoch time and LocalDateTime. Hive 
0.x/1.x/2.x use local timezone. But Hive 3.x use UTC timezone, by default true 
</span> |
+
+If the table type is MERGE_ON_READ, streaming read is supported through 
options:
+
+|  Option Name  | Required | Default | Remarks |
+|  -----------  | -------  | ------- | ------- |
+| `read.streaming.enabled` | N | false | <span style="color:grey"> Whether to 
read as streaming source, default false </span> |
+| `read.streaming.check-interval` | N | 60 | <span style="color:grey"> Check 
interval for streaming read of SECOND, default 1 minute </span> |
+| `read.streaming.start-commit` | N | N/A | <span style="color:grey"> Start 
commit instant for streaming read, the commit time format should be 
'yyyyMMddHHmmss', by default reading from the latest instant </span> |
+
 ## WriteClient Configs {#writeclient-configs}
 
 Jobs programming directly against the RDD level apis can build a 
`HoodieWriteConfig` object and pass it in to the `HoodieWriteClient` 
constructor. 
diff --git a/docs/_layouts/home.html b/docs/_layouts/home.html
index a5463a1..bbbfb14 100644
--- a/docs/_layouts/home.html
+++ b/docs/_layouts/home.html
@@ -20,7 +20,7 @@ layout: home
 
         <p class="page__lead">{{ page.excerpt }}</p>
         <p>
-          <a href="/docs/quick-start-guide.html" class="btn btn--light-outline 
btn--large"><i class="fa fa-paper-plane"></i> Get Started</a>
+          <a href="/docs/spark_quick-start-guide.html" class="btn 
btn--light-outline btn--large"><i class="fa fa-paper-plane"></i> Get Started</a>
         </p>
       </div>
     </div>
diff --git a/docs/_pages/contributing.cn.md b/docs/_pages/contributing.cn.md
index ca4b011..6c4e4e6 100644
--- a/docs/_pages/contributing.cn.md
+++ b/docs/_pages/contributing.cn.md
@@ -25,7 +25,7 @@ To contribute code, you need
 
 To contribute, you would need to do the following
  
- - Fork the Hudi code on Github & then clone your own fork locally. Once 
cloned, we recommend building as per instructions on 
[quickstart](/docs/quick-start-guide.html)
+ - Fork the Hudi code on Github & then clone your own fork locally. Once 
cloned, we recommend building as per instructions on [spark 
quickstart](/docs/spark_quick-start-guide.html) or [flink 
quickstart](/docs/flink-quick-start-guide.html)
  - [Recommended] We have embraced the code style largely based on [google 
format](https://google.github.io/styleguide/javaguide.html). Please setup your 
IDE with style files from 
[here](https://github.com/apache/hudi/tree/master/style).
 These instructions have been tested on IntelliJ. 
  - [Recommended] Set up the [Save Action 
Plugin](https://plugins.jetbrains.com/plugin/7642-save-actions) to auto format 
& organize imports on save. The Maven Compilation life-cycle will fail if there 
are checkstyle violations.
diff --git a/docs/_pages/contributing.md b/docs/_pages/contributing.md
index 2d30563..c8147f3 100644
--- a/docs/_pages/contributing.md
+++ b/docs/_pages/contributing.md
@@ -24,7 +24,7 @@ To contribute code, you need
 
 To contribute, you would need to do the following
  
-- Fork the Hudi code on Github & then clone your own fork locally. Once 
cloned, we recommend building as per instructions on 
[quickstart](/docs/quick-start-guide.html)
+- Fork the Hudi code on Github & then clone your own fork locally. Once 
cloned, we recommend building as per instructions on [spark 
quickstart](/docs/spark_quick-start-guide.html) or [flink 
quickstart](/docs/flink-quick-start-guide.html)
 
 - \[Recommended\] We have embraced the code style largely based on [google 
format](https://google.github.io/styleguide/javaguide.html). Please setup your 
IDE with style files from [\<project 
root\>/style/](https://github.com/apache/hudi/tree/master/style). These 
instructions have been tested on IntelliJ.
 
diff --git a/docs/_pages/releases.md b/docs/_pages/releases.md
index 7aa98ec..d2212c5 100644
--- a/docs/_pages/releases.md
+++ b/docs/_pages/releases.md
@@ -66,7 +66,7 @@ Specifically, the `HoodieFlinkStreamer` allows for Hudi 
Copy-On-Write table to b
   derived/ETL pipelines similar to data 
[sensors](https://airflow.apache.org/docs/apache-airflow/stable/_api/airflow/sensors/index.html)
 in Apache Airflow.
 - **Insert Overwrite/Insert Overwrite Table**: We have added these two new 
write operation types, predominantly to help existing batch ETL jobs, which 
typically overwrite entire 
   tables/partitions each run. These operations are much cheaper, than having 
to issue upserts, given they are bulk replacing the target table.
-  Check [here](/docs/quick-start-guide.html#insert-overwrite-table) for 
examples.
+  Check [here](/docs/spark_quick-start-guide.html#insert-overwrite-table) for 
examples.
 - **Delete Partition**: For users of WriteClient/RDD level apis, we have added 
an API to delete an entire partition, again without issuing deletes at the 
record level.
 - The current default `OverwriteWithLatestAvroPayload` will overwrite the 
value in storage, even if for e.g the upsert was reissued for an older value of 
the key.
   Added a new `DefaultHoodieRecordPayload` and a new payload config 
`hoodie.payload.ordering.field` helps specify a field, that the incoming upsert 
record can be compared with

Reply via email to