From d31e9478f760f947df137558264dfa7520681834 Mon Sep 17 00:00:00 2001
From: Carl-Zhou-CN <67902676+Carl-Zhou-CN@users.noreply.github.com>
Date: Wed, 18 Oct 2023 11:09:10 +0800
Subject: [PATCH] [Feature][Jdbc] Supporting more ways to configure connection
parameters. (#5388)
---
docs/en/connector-v2/sink/DB2.md | 45 ++--
docs/en/connector-v2/sink/Jdbc.md | 5 +
docs/en/connector-v2/sink/Mysql.md | 45 ++--
docs/en/connector-v2/sink/OceanBase.md | 41 ++--
docs/en/connector-v2/sink/Oracle.md | 45 ++--
docs/en/connector-v2/sink/PostgreSql.md | 45 ++--
docs/en/connector-v2/sink/Snowflake.md | 163 +++++++------
docs/en/connector-v2/sink/Vertica.md | 43 ++--
docs/en/connector-v2/source/DB2.md | 1 +
docs/en/connector-v2/source/Jdbc.md | 5 +
docs/en/connector-v2/source/Mysql.md | 7 +
docs/en/connector-v2/source/OceanBase.md | 1 +
docs/en/connector-v2/source/Oracle.md | 4 +
docs/en/connector-v2/source/PostgreSQL.md | 1 +
docs/en/connector-v2/source/Snowflake.md | 1 +
docs/en/connector-v2/source/Vertica.md | 1 +
.../jdbc/config/JdbcConnectionConfig.java | 17 ++
.../seatunnel/jdbc/config/JdbcOptions.java | 7 +
.../internal/connection/DataSourceUtils.java | 2 +-
.../SimpleJdbcConnectionProvider.java | 1 +
.../jdbc/internal/dialect/JdbcDialect.java | 16 ++
.../internal/dialect/mysql/MysqlDialect.java | 9 +
.../seatunnel/jdbc/sink/JdbcSink.java | 4 +
.../seatunnel/jdbc/sink/JdbcSinkFactory.java | 4 +
.../seatunnel/jdbc/source/JdbcSource.java | 10 +-
.../jdbc/source/JdbcSourceFactory.java | 8 +-
.../command/SparkTaskExecuteCommand.java | 3 +-
.../seatunnel/jdbc/JdbcMysqlIT.java | 222 ++++++++++++++++++
.../resources/jdbc_mysql_source_and_sink.conf | 12 +-
.../jdbc_mysql_source_and_sink_xa.conf | 8 +
.../resources/jdbc_oracle_source_to_sink.conf | 6 +
31 files changed, 562 insertions(+), 220 deletions(-)
diff --git a/docs/en/connector-v2/sink/DB2.md b/docs/en/connector-v2/sink/DB2.md
index fc0aaca0943..e91912790a9 100644
--- a/docs/en/connector-v2/sink/DB2.md
+++ b/docs/en/connector-v2/sink/DB2.md
@@ -35,7 +35,7 @@ semantics (using XA transaction guarantee).
## Data Type Mapping
| DB2 Data type | SeaTunnel Data type |
-|------------------------------------------------------------------------------------------------------|---------------------|---|
+|------------------------------------------------------------------------------------------------------|---------------------|
| BOOLEAN | BOOLEAN |
| SMALLINT | SHORT |
| INT
INTEGER
| INTEGER |
@@ -52,27 +52,28 @@ semantics (using XA transaction guarantee).
## Sink Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:db2://127.0.0.1:50000/dbname |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use DB2 the value is `com.ibm.db2.jdbc.app.DB2Driver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
-| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, DB2 is `com.db2.cj.jdbc.Db2XADataSource`, and
please refer to appendix for other data sources |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:db2://127.0.0.1:50000/dbname |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use DB2 the value is `com.ibm.db2.jdbc.app.DB2Driver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
+| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, DB2 is `com.db2.cj.jdbc.Db2XADataSource`, and
please refer to appendix for other data sources |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md
index 394fadde801..3b1ae9c0eb8 100644
--- a/docs/en/connector-v2/sink/Jdbc.md
+++ b/docs/en/connector-v2/sink/Jdbc.md
@@ -48,6 +48,7 @@ support `Xa transactions`. You can set `is_exactly_once=true` to enable it.
| transaction_timeout_sec | Int | No | -1 |
| auto_commit | Boolean | No | true |
| field_ide | String | No | - |
+| properties | Map | No | - |
| common-options | | no | - |
### driver [string]
@@ -143,6 +144,10 @@ The field "field_ide" is used to identify whether the field needs to be converte
synchronizing from the source to the sink. "ORIGINAL" indicates no conversion is needed, "UPPERCASE" indicates
conversion to uppercase, and "LOWERCASE" indicates conversion to lowercase.
+### properties
+
+Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL.
+
### common options
Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details
diff --git a/docs/en/connector-v2/sink/Mysql.md b/docs/en/connector-v2/sink/Mysql.md
index 860f071df0e..108b657fdd4 100644
--- a/docs/en/connector-v2/sink/Mysql.md
+++ b/docs/en/connector-v2/sink/Mysql.md
@@ -58,28 +58,29 @@ semantics (using XA transaction guarantee).
## Sink Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/test |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
-| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, mysql is `com.mysql.cj.jdbc.MysqlXADataSource`, and
please refer to appendix for other data sources |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| field_ide | String | No | - | Identify whether the field needs to be converted when synchronizing from the source to the sink. `ORIGINAL` indicates no conversion is needed;`UPPERCASE` indicates conversion to uppercase;`LOWERCASE` indicates conversion to lowercase. |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/test |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
+| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, mysql is `com.mysql.cj.jdbc.MysqlXADataSource`, and
please refer to appendix for other data sources |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| field_ide | String | No | - | Identify whether the field needs to be converted when synchronizing from the source to the sink. `ORIGINAL` indicates no conversion is needed;`UPPERCASE` indicates conversion to uppercase;`LOWERCASE` indicates conversion to lowercase. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/sink/OceanBase.md b/docs/en/connector-v2/sink/OceanBase.md
index 3cea0b5e6e6..d5888dcc3b7 100644
--- a/docs/en/connector-v2/sink/OceanBase.md
+++ b/docs/en/connector-v2/sink/OceanBase.md
@@ -67,26 +67,27 @@ Write data through jdbc. Support Batch mode and Streaming mode, support concurre
## Sink Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:oceanbase://localhost:2883/test |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source, should be `com.oceanbase.jdbc.Driver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| compatible_mode | String | Yes | - | The compatible mode of OceanBase, can be 'mysql' or 'oracle'. |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:oceanbase://localhost:2883/test |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source, should be `com.oceanbase.jdbc.Driver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| compatible_mode | String | Yes | - | The compatible mode of OceanBase, can be 'mysql' or 'oracle'. |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/sink/Oracle.md b/docs/en/connector-v2/sink/Oracle.md
index 151243f318f..753891653ac 100644
--- a/docs/en/connector-v2/sink/Oracle.md
+++ b/docs/en/connector-v2/sink/Oracle.md
@@ -52,28 +52,29 @@ semantics (using XA transaction guarantee).
## Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:oracle:thin:@datasource01:1523:xe |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Oracle the value is `oracle.jdbc.OracleDriver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `batch_interval_ms`
, the data will be flushed into the database |
-| batch_interval_ms | Int | No | 1000 | For batch writing, when the number of buffers reaches the number of `batch_size` or the time reaches `batch_interval_ms`, the data will be flushed into the database |
-| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to. |
-| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, Oracle is `oracle.jdbc.xa.client.OracleXADataSource`, and
please refer to appendix for other data sources |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:oracle:thin:@datasource01:1523:xe |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Oracle the value is `oracle.jdbc.OracleDriver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `batch_interval_ms`
, the data will be flushed into the database |
+| batch_interval_ms | Int | No | 1000 | For batch writing, when the number of buffers reaches the number of `batch_size` or the time reaches `batch_interval_ms`, the data will be flushed into the database |
+| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to. |
+| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, Oracle is `oracle.jdbc.xa.client.OracleXADataSource`, and
please refer to appendix for other data sources |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/sink/PostgreSql.md b/docs/en/connector-v2/sink/PostgreSql.md
index bcc5616f5ea..79f595da329 100644
--- a/docs/en/connector-v2/sink/PostgreSql.md
+++ b/docs/en/connector-v2/sink/PostgreSql.md
@@ -61,28 +61,29 @@ semantics (using XA transaction guarantee).
## Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/test
if you would use json or jsonb type insert please add jdbc url stringtype=unspecified option |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use PostgreSQL the value is `org.postgresql.Driver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to. |
-| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, PostgreSQL is `org.postgresql.xa.PGXADataSource`, and
please refer to appendix for other data sources |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| field_ide | String | No | - | Identify whether the field needs to be converted when synchronizing from the source to the sink. `ORIGINAL` indicates no conversion is needed;`UPPERCASE` indicates conversion to uppercase;`LOWERCASE` indicates conversion to lowercase. |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/test
if you would use json or jsonb type insert please add jdbc url stringtype=unspecified option |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use PostgreSQL the value is `org.postgresql.Driver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to. |
+| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, PostgreSQL is `org.postgresql.xa.PGXADataSource`, and
please refer to appendix for other data sources |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| field_ide | String | No | - | Identify whether the field needs to be converted when synchronizing from the source to the sink. `ORIGINAL` indicates no conversion is needed;`UPPERCASE` indicates conversion to uppercase;`LOWERCASE` indicates conversion to lowercase. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/sink/Snowflake.md b/docs/en/connector-v2/sink/Snowflake.md
index 1dfff5e09c7..2ae6b0fac78 100644
--- a/docs/en/connector-v2/sink/Snowflake.md
+++ b/docs/en/connector-v2/sink/Snowflake.md
@@ -1,14 +1,14 @@
# Snowflake
> JDBC Snowflake Sink Connector
->
-> ## Support those engines
->
+
+## Support Those Engines
+
> Spark
> Flink
> SeaTunnel Zeta
->
- ## Key features
+
+## Key features
- [ ] [exactly-once](../../concept/connector-v2-features.md)
- [x] [cdc](../../concept/connector-v2-features.md)
@@ -27,8 +27,8 @@ Write data through jdbc. Support Batch mode and Streaming mode, support concurre
> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' working directory
> For example Snowflake datasource: cp snowflake-connector-java-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/
->
- ## Data Type Mapping
+
+## Data Type Mapping
| Snowflake Data type | SeaTunnel Data type |
|-----------------------------------------------------------------------------|---------------------|
@@ -48,24 +48,25 @@ Write data through jdbc. Support Batch mode and Streaming mode, support concurre
## Options
-| name | type | required | default value | description |
-|-------------------------------------------|---------|----------|---------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:snowflake://.snowflakecomputing.com |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Snowflake the value is `net.snowflake.client.jdbc.SnowflakeDriver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| name | type | required | default value | description |
+|-------------------------------------------|---------|----------|---------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:snowflake://.snowflakecomputing.com |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Snowflake the value is `net.snowflake.client.jdbc.SnowflakeDriver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
## tips
@@ -76,68 +77,66 @@ Write data through jdbc. Support Batch mode and Streaming mode, support concurre
### simple:
> This example defines a SeaTunnel synchronization task that automatically generates data through FakeSource and sends it to JDBC Sink. FakeSource generates a total of 16 rows of data (row.num=16), with each row having two fields, name (string type) and age (int type). The final target table is test_table will also be 16 rows of data in the table. Before run this job, you need create database test and table test_table in your snowflake database. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job.
->
-> ```
-> # Defining the runtime environment
-> env {
-> # You can set flink configuration here
-> execution.parallelism = 1
-> job.mode = "BATCH"
-> }
-> source {
-> # This is a example source plugin **only for test and demonstrate the feature source plugin**
-> FakeSource {
-> parallelism = 1
-> result_table_name = "fake"
-> row.num = 16
-> schema = {
-> fields {
-> name = "string"
-> age = "int"
-> }
-> }
-> }
-> # If you would like to get more information about how to configure seatunnel and see full list of source plugins,
-> # please go to https://seatunnel.apache.org/docs/category/source-v2
-> }
-> transform {
-> # If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
-> # please go to https://seatunnel.apache.org/docs/category/transform-v2
-> }
-> sink {
-> jdbc {
-> url = "jdbc:snowflake://.snowflakecomputing.com"
-> driver = "net.snowflake.client.jdbc.SnowflakeDriver"
-> user = "root"
-> password = "123456"
-> query = "insert into test_table(name,age) values(?,?)"
-> }
-> # If you would like to get more information about how to configure seatunnel and see full list of sink plugins,
-> # please go to https://seatunnel.apache.org/docs/category/sink-v2
-> }
-> ```
+
+```
+# Defining the runtime environment
+env {
+# You can set flink configuration here
+execution.parallelism = 1
+job.mode = "BATCH"
+}
+source {
+# This is a example source plugin **only for test and demonstrate the feature source plugin**
+FakeSource {
+parallelism = 1
+result_table_name = "fake"
+row.num = 16
+schema = {
+fields {
+name = "string"
+age = "int"
+}
+}
+}
+# If you would like to get more information about how to configure seatunnel and see full list of source plugins,
+# please go to https://seatunnel.apache.org/docs/category/source-v2
+}
+transform {
+# If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
+# please go to https://seatunnel.apache.org/docs/category/transform-v2
+}
+sink {
+jdbc {
+url = "jdbc:snowflake://.snowflakecomputing.com"
+driver = "net.snowflake.client.jdbc.SnowflakeDriver"
+user = "root"
+password = "123456"
+query = "insert into test_table(name,age) values(?,?)"
+}
+# If you would like to get more information about how to configure seatunnel and see full list of sink plugins,
+# please go to https://seatunnel.apache.org/docs/category/sink-v2
+}
+```
### CDC(Change data capture) event
> CDC change data is also supported by us In this case, you need config database, table and primary_keys.
->
-> ```
-> jdbc {
-> url = "jdbc:snowflake://.snowflakecomputing.com"
-> driver = "net.snowflake.client.jdbc.SnowflakeDriver"
-> user = "root"
-> password = "123456"
->
-> ```
-
- # You need to configure both database and table
- database = test
- table = sink_table
- primary_keys = ["id","name"]
-
-}
```
-
+sink {
+ jdbc {
+ url = "jdbc:snowflake://.snowflakecomputing.com"
+ driver = "net.snowflake.client.jdbc.SnowflakeDriver"
+ user = "root"
+ password = "123456"
+ generate_sink_sql = true
+
+
+ # You need to configure both database and table
+ database = test
+ table = sink_table
+ primary_keys = ["id","name"]
+ }
+}
```
diff --git a/docs/en/connector-v2/sink/Vertica.md b/docs/en/connector-v2/sink/Vertica.md
index 9a624407682..c05ac8f6ee5 100644
--- a/docs/en/connector-v2/sink/Vertica.md
+++ b/docs/en/connector-v2/sink/Vertica.md
@@ -54,27 +54,28 @@ semantics (using XA transaction guarantee).
## Sink Options
-| Name | Type | Required | Default | Description |
-|-------------------------------------------|---------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:vertica://localhost:5433/vertica |
-| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Vertical the value is `com.vertica.jdbc.Driver`. |
-| user | String | No | - | Connection instance user name |
-| password | String | No | - | Connection instance password |
-| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
-| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
-| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
-| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
-| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
-| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
-| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
-| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
-| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
-| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, vertical is `com.vertical.cj.jdbc.VerticalXADataSource`, and
please refer to appendix for other data sources |
-| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
-| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
-| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
-| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
+| Name | Type | Required | Default | Description |
+|-------------------------------------------|---------|----------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:vertica://localhost:5433/vertica |
+| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Vertical the value is `com.vertica.jdbc.Driver`. |
+| user | String | No | - | Connection instance user name |
+| password | String | No | - | Connection instance password |
+| query | String | No | - | Use this sql write upstream input datas to database. e.g `INSERT ...`,`query` have the higher priority |
+| database | String | No | - | Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| table | String | No | - | Use database and this table-name auto-generate sql and receive upstream input datas write to database.
This option is mutually exclusive with `query` and has a higher priority. |
+| primary_keys | Array | No | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql. |
+| support_upsert_by_query_primary_key_exist | Boolean | No | false | Choose to use INSERT sql, UPDATE sql to process update events(INSERT, UPDATE_AFTER) based on query primary key exists. This configuration is only used when database unsupport upsert syntax. **Note**: that this method has low performance |
+| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. |
+| max_retries | Int | No | 0 | The number of retries to submit failed (executeBatch) |
+| batch_size | Int | No | 1000 | For batch writing, when the number of buffered records reaches the number of `batch_size` or the time reaches `checkpoint.interval`
, the data will be flushed into the database |
+| is_exactly_once | Boolean | No | false | Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to
set `xa_data_source_class_name`. |
+| generate_sink_sql | Boolean | No | false | Generate sql statements based on the database table you want to write to |
+| xa_data_source_class_name | String | No | - | The xa data source class name of the database Driver, for example, vertical is `com.vertical.cj.jdbc.VerticalXADataSource`, and
please refer to appendix for other data sources |
+| max_commit_attempts | Int | No | 3 | The number of retries for transaction commit failures |
+| transaction_timeout_sec | Int | No | -1 | The timeout after the transaction is opened, the default is -1 (never timeout). Note that setting the timeout may affect
exactly-once semantics |
+| auto_commit | Boolean | No | true | Automatic transaction commit is enabled by default |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
+| common-options | | no | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/source/DB2.md b/docs/en/connector-v2/source/DB2.md
index c9eb6a578b6..c2660535760 100644
--- a/docs/en/connector-v2/source/DB2.md
+++ b/docs/en/connector-v2/source/DB2.md
@@ -67,6 +67,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md
index b86a7b33854..442684351d3 100644
--- a/docs/en/connector-v2/source/Jdbc.md
+++ b/docs/en/connector-v2/source/Jdbc.md
@@ -42,6 +42,7 @@ supports query SQL and can achieve projection effect.
| partition_lower_bound | Long | No | - |
| partition_num | Int | No | job parallelism |
| fetch_size | Int | No | 0 |
+| properties | Map | No | - |
| common-options | | No | - |
### driver [string]
@@ -93,6 +94,10 @@ The number of partition count, only support positive integer. default value is j
For queries that return a large number of objects, you can configure the row fetch size used in the query to
improve performance by reducing the number database hits required to satisfy the selection criteria. Zero means use jdbc default value.
+### properties
+
+Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL.
+
### common options
Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details.
diff --git a/docs/en/connector-v2/source/Mysql.md b/docs/en/connector-v2/source/Mysql.md
index bdac5c0aec6..fc2295e7890 100644
--- a/docs/en/connector-v2/source/Mysql.md
+++ b/docs/en/connector-v2/source/Mysql.md
@@ -73,6 +73,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
@@ -135,6 +136,9 @@ source {
partition_column = "id"
# Number of fragments
partition_num = 10
+ properties {
+ useSSL=false
+ }
}
}
sink {
@@ -162,6 +166,9 @@ source {
# Read end boundary
partition_upper_bound = 500
partition_num = 10
+ properties {
+ useSSL=false
+ }
}
}
```
diff --git a/docs/en/connector-v2/source/OceanBase.md b/docs/en/connector-v2/source/OceanBase.md
index 434e25284dd..5dbb633c109 100644
--- a/docs/en/connector-v2/source/OceanBase.md
+++ b/docs/en/connector-v2/source/OceanBase.md
@@ -85,6 +85,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. Default value is job parallelism. |
| fetch_size | Int | No | 0 | For queries that return a large number of objects, you can configure
the row fetch size used in the query to improve performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/source/Oracle.md b/docs/en/connector-v2/source/Oracle.md
index f191cda9d99..aefd8adfbba 100644
--- a/docs/en/connector-v2/source/Oracle.md
+++ b/docs/en/connector-v2/source/Oracle.md
@@ -67,6 +67,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
@@ -128,6 +129,9 @@ source {
partition_column = "ID"
# Number of fragments
partition_num = 10
+ properties {
+ database.oracle.jdbc.timezoneAsRegion = "false"
+ }
}
}
sink {
diff --git a/docs/en/connector-v2/source/PostgreSQL.md b/docs/en/connector-v2/source/PostgreSQL.md
index 63ddbc25ecf..cb7de77e6c0 100644
--- a/docs/en/connector-v2/source/PostgreSQL.md
+++ b/docs/en/connector-v2/source/PostgreSQL.md
@@ -76,6 +76,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
diff --git a/docs/en/connector-v2/source/Snowflake.md b/docs/en/connector-v2/source/Snowflake.md
index a7835013d58..2ea4c764352 100644
--- a/docs/en/connector-v2/source/Snowflake.md
+++ b/docs/en/connector-v2/source/Snowflake.md
@@ -69,6 +69,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
## tips
diff --git a/docs/en/connector-v2/source/Vertica.md b/docs/en/connector-v2/source/Vertica.md
index df387ac30bf..b4c5d4e05b4 100644
--- a/docs/en/connector-v2/source/Vertica.md
+++ b/docs/en/connector-v2/source/Vertica.md
@@ -69,6 +69,7 @@ Read external data source data through JDBC.
| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. |
| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism |
| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. |
+| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details |
### Tips
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java
index 555963af2cf..eeeff227b2f 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java
@@ -20,6 +20,8 @@
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
import java.util.Optional;
public class JdbcConnectionConfig implements Serializable {
@@ -45,6 +47,8 @@ public class JdbcConnectionConfig implements Serializable {
public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue();
+ private Map properties;
+
public static JdbcConnectionConfig of(ReadonlyConfig config) {
JdbcConnectionConfig.Builder builder = JdbcConnectionConfig.builder();
builder.url(config.get(JdbcOptions.URL));
@@ -63,6 +67,7 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) {
config.getOptional(JdbcOptions.USER).ifPresent(builder::username);
config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password);
+ config.getOptional(JdbcOptions.PROPERTIES).ifPresent(builder::properties);
return builder.build();
}
@@ -114,6 +119,10 @@ public Optional getTransactionTimeoutSec() {
return transactionTimeoutSec < 0 ? Optional.empty() : Optional.of(transactionTimeoutSec);
}
+ public Map getProperties() {
+ return properties;
+ }
+
public static JdbcConnectionConfig.Builder builder() {
return new JdbcConnectionConfig.Builder();
}
@@ -133,6 +142,7 @@ public static final class Builder {
private String xaDataSourceClassName;
private int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue();
private int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue();
+ private Map properties;
private Builder() {}
@@ -201,6 +211,11 @@ public Builder transactionTimeoutSec(int transactionTimeoutSec) {
return this;
}
+ public Builder properties(Map properties) {
+ this.properties = properties;
+ return this;
+ }
+
public JdbcConnectionConfig build() {
JdbcConnectionConfig jdbcConnectionConfig = new JdbcConnectionConfig();
jdbcConnectionConfig.batchSize = this.batchSize;
@@ -215,6 +230,8 @@ public JdbcConnectionConfig build() {
jdbcConnectionConfig.transactionTimeoutSec = this.transactionTimeoutSec;
jdbcConnectionConfig.maxCommitAttempts = this.maxCommitAttempts;
jdbcConnectionConfig.xaDataSourceClassName = this.xaDataSourceClassName;
+ jdbcConnectionConfig.properties =
+ this.properties == null ? new HashMap<>() : this.properties;
return jdbcConnectionConfig;
}
}
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java
index b01fc872f31..91ba5e2030f 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java
@@ -23,6 +23,7 @@
import java.math.BigDecimal;
import java.util.List;
+import java.util.Map;
public interface JdbcOptions {
@@ -161,4 +162,10 @@ public interface JdbcOptions {
.enumType(FieldIdeEnum.class)
.noDefaultValue()
.withDescription("Whether case conversion is required");
+
+ Option